Change logical plan to apply filter from 2ndary index

- Changes the IntroduceLSMComponentFilterRule to
replace the constant filter value from the query to the value
carried from 2ndary index search.
- Can use 2ndary index filter even the query doens't contain
any filter related condition.

Change-Id: I0e2fe0208662e5dcd49d1a22bfb58f96533e9497
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1727
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
BAD: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <buyingyi@gmail.com>
diff --git a/.gitignore b/.gitignore
index 295d874..569eb3d 100644
--- a/.gitignore
+++ b/.gitignore
@@ -27,3 +27,4 @@
 *.swp
 .m2*
 ß
+
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 9dd57d5..2fd9079 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
@@ -115,6 +115,7 @@
 
         int[] minFilterFieldIndexes = getKeyIndexes(unnestMap.getMinFilterVars(), inputSchemas);
         int[] maxFilterFieldIndexes = getKeyIndexes(unnestMap.getMaxFilterVars(), inputSchemas);
+        boolean propagateFilter = unnestMap.propagateIndexFilter();
 
         MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
         Dataset dataset = metadataProvider.findDataset(jobGenParams.getDataverseName(), jobGenParams.getDatasetName());
@@ -124,7 +125,7 @@
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> btreeSearch = metadataProvider.buildBtreeRuntime(
                 builder.getJobSpec(), opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainMissing,
                 dataset, jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes, jobGenParams.isLowKeyInclusive(),
-                jobGenParams.isHighKeyInclusive(), minFilterFieldIndexes, maxFilterFieldIndexes);
+                jobGenParams.isHighKeyInclusive(), propagateFilter, minFilterFieldIndexes, maxFilterFieldIndexes);
 
         builder.contributeHyracksOperator(unnestMap, btreeSearch.first);
         builder.contributeAlgebricksPartitionConstraint(btreeSearch.first, btreeSearch.second);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java
index ce43480..9f46e6a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java
@@ -63,7 +63,7 @@
         IDataSource<?> ds = idx.getDataSource();
         IDataSourcePropertiesProvider dspp = ds.getPropertiesProvider();
         AbstractScanOperator as = (AbstractScanOperator) op;
-        deliveredProperties = dspp.computePropertiesVector(as.getVariables());
+        deliveredProperties = dspp.computePropertiesVector(as.getScanVariables());
     }
 
     protected int[] getKeyIndexes(List<LogicalVariable> keyVarList, IOperatorSchema[] inputSchemas) {
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 50c762e..213c60b 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
@@ -135,13 +135,16 @@
             AbstractUnnestMapOperator unnestMap, IOperatorSchema opSchema, boolean retainInput, boolean retainMissing,
             String datasetName, Dataset dataset, String indexName, ATypeTag searchKeyType, int[] keyFields,
             SearchModifierType searchModifierType, IAlgebricksConstantValue similarityThreshold,
-            int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean isFullTextSearchQuery)
-            throws AlgebricksException {
+            int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
+            boolean isFullTextSearchQuery) throws AlgebricksException {
         try {
+
+            boolean propagateIndexFilter = unnestMap.propagateIndexFilter();
             IAObject simThresh = ((AsterixConstantValue) similarityThreshold).getObject();
             int numPrimaryKeys = dataset.getPrimaryKeys().size();
-            Index secondaryIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
-                    dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+            Index secondaryIndex = MetadataManager.INSTANCE
+                    .getIndex(metadataProvider.getMetadataTxnContext(), dataset.getDataverseName(),
+                            dataset.getDatasetName(), indexName);
             if (secondaryIndex == null) {
                 throw new AlgebricksException(
                         "Code generation error: no index " + indexName + " for dataset " + datasetName);
@@ -160,13 +163,15 @@
             IIndexDataflowHelperFactory dataflowHelperFactory =
                     new IndexDataflowHelperFactory(metadataProvider.getStorageComponentProvider().getStorageManager(),
                             secondarySplitsAndConstraint.first);
-            LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
-                    jobSpec, outputRecDesc, queryField, dataflowHelperFactory, queryTokenizerFactory,
-                    searchModifierFactory, retainInput, retainMissing, context.getMissingWriterFactory(),
-                    dataset.getSearchCallbackFactory(metadataProvider.getStorageComponentProvider(), secondaryIndex,
-                            ((JobEventListenerFactory) jobSpec.getJobletEventListenerFactory()).getJobId(),
-                            IndexOperation.SEARCH, null),
-                    minFilterFieldIndexes, maxFilterFieldIndexes, isFullTextSearchQuery, numPrimaryKeys, false);
+            LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp =
+                    new LSMInvertedIndexSearchOperatorDescriptor(jobSpec, outputRecDesc, queryField,
+                            dataflowHelperFactory, queryTokenizerFactory, searchModifierFactory, retainInput,
+                            retainMissing, context.getMissingWriterFactory(),
+                            dataset.getSearchCallbackFactory(metadataProvider.getStorageComponentProvider(),
+                                    secondaryIndex,
+                                    ((JobEventListenerFactory) jobSpec.getJobletEventListenerFactory()).getJobId(),
+                                    IndexOperation.SEARCH, null), minFilterFieldIndexes, maxFilterFieldIndexes,
+                            isFullTextSearchQuery, numPrimaryKeys, propagateIndexFilter);
             return new Pair<>(invIndexSearchOp, secondarySplitsAndConstraint.second);
         } catch (MetadataException e) {
             throw new AlgebricksException(e);
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 f9d4c80..733e62f 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
@@ -66,7 +66,7 @@
     @Override
     public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
             IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
-                    throws AlgebricksException {
+            throws AlgebricksException {
         AbstractUnnestMapOperator unnestMap = (AbstractUnnestMapOperator) op;
         ILogicalExpression unnestExpr = unnestMap.getExpressionRef().getValue();
         if (unnestExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
@@ -81,6 +81,7 @@
         jobGenParams.readFromFuncArgs(unnestFuncExpr.getArguments());
         int[] keyIndexes = getKeyIndexes(jobGenParams.getKeyVarList(), inputSchemas);
 
+        boolean propagateIndexFilter = unnestMap.propagateIndexFilter();
         int[] minFilterFieldIndexes = getKeyIndexes(unnestMap.getMinFilterVars(), inputSchemas);
         int[] maxFilterFieldIndexes = getKeyIndexes(unnestMap.getMaxFilterVars(), inputSchemas);
 
@@ -97,9 +98,10 @@
             // By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching tuples.
             retainNull = true;
         }
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> rtreeSearch = mp.buildRtreeRuntime(
-                builder.getJobSpec(), outputVars, opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainNull,
-                dataset, jobGenParams.getIndexName(), keyIndexes, minFilterFieldIndexes, maxFilterFieldIndexes);
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> rtreeSearch =
+                mp.buildRtreeRuntime(builder.getJobSpec(), outputVars, opSchema, typeEnv, context,
+                        jobGenParams.getRetainInput(), retainNull, dataset, jobGenParams.getIndexName(), keyIndexes,
+                        propagateIndexFilter, minFilterFieldIndexes, maxFilterFieldIndexes);
 
         builder.contributeHyracksOperator(unnestMap, rtreeSearch.first);
         builder.contributeAlgebricksPartitionConstraint(rtreeSearch.first, rtreeSearch.second);
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 83b277d..95f0de9 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
@@ -19,11 +19,16 @@
 package org.apache.asterix.optimizer.rules.am;
 
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Queue;
+import java.util.logging.Logger;
 
 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.DatasetDataSource;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -56,6 +61,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.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IntersectOperator;
 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;
@@ -63,6 +69,8 @@
 
 public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
 
+    static final Logger LOGGER = Logger.getLogger(IntroduceLSMComponentFilterRule.class.getName());
+
     protected IVariableTypeEnvironment typeEnvironment = null;
 
     @Override
@@ -80,12 +88,6 @@
         }
 
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
-        typeEnvironment = context.getOutputTypeEnvironment(op);
-        ILogicalExpression condExpr = ((SelectOperator) op).getCondition().getValue();
-        AccessMethodAnalysisContext analysisCtx = analyzeCondition(condExpr, context, typeEnvironment);
-        if (analysisCtx.getMatchedFuncExprs().isEmpty()) {
-            return false;
-        }
 
         Dataset dataset = getDataset(op, context);
         List<String> filterFieldName = null;
@@ -101,22 +103,33 @@
         if (filterFieldName == null || recType == null) {
             return false;
         }
-        List<Index> datasetIndexes = ((MetadataProvider) context.getMetadataProvider())
-                .getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
+
+        IAType filterType = recType.getSubFieldType(filterFieldName);
+
+        typeEnvironment = context.getOutputTypeEnvironment(op);
+        ILogicalExpression condExpr = ((SelectOperator) op).getCondition().getValue();
+        AccessMethodAnalysisContext analysisCtx = analyzeCondition(condExpr, context, typeEnvironment);
 
         List<IOptimizableFuncExpr> optFuncExprs = new ArrayList<>();
 
-        for (int i = 0; i < analysisCtx.getMatchedFuncExprs().size(); i++) {
-            IOptimizableFuncExpr optFuncExpr = analysisCtx.getMatchedFuncExpr(i);
-            boolean found = findMacthedExprFieldName(optFuncExpr, op, dataset, recType, datasetIndexes, context);
-            if (found && optFuncExpr.getFieldName(0).equals(filterFieldName)) {
-                optFuncExprs.add(optFuncExpr);
+        if (!analysisCtx.getMatchedFuncExprs().isEmpty()) {
+            List<Index> datasetIndexes = ((MetadataProvider) context.getMetadataProvider())
+                    .getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
+
+            for (int i = 0; i < analysisCtx.getMatchedFuncExprs().size(); i++) {
+                IOptimizableFuncExpr optFuncExpr = analysisCtx.getMatchedFuncExpr(i);
+                boolean found = findMacthedExprFieldName(optFuncExpr, op, dataset, recType, datasetIndexes, context);
+                if (found && optFuncExpr.getFieldName(0).equals(filterFieldName)) {
+                    optFuncExprs.add(optFuncExpr);
+                }
             }
         }
+
         if (optFuncExprs.isEmpty()) {
-            return false;
+            assignFilterFromSecondaryUnnestMap(op, dataset, context, filterType);
+        } else {
+            assignFilterFromQuery(optFuncExprs, op, dataset, context, filterType);
         }
-        changePlan(optFuncExprs, op, dataset, context);
 
         OperatorPropertiesUtil.typeOpRec(opRef, context);
         context.addToDontApplySet(this, op);
@@ -147,9 +160,11 @@
         return new AssignOperator(assignKeyVarList, assignKeyExprList);
     }
 
-    private void changePlan(List<IOptimizableFuncExpr> optFuncExprs, AbstractLogicalOperator op, Dataset dataset,
-            IOptimizationContext context) throws AlgebricksException {
+    private void assignFilterFromQuery(List<IOptimizableFuncExpr> optFuncExprs, AbstractLogicalOperator op,
+            Dataset dataset, IOptimizationContext context, IAType filterType) throws AlgebricksException {
 
+        List<UnnestMapOperator> primaryUnnestMapOps = new ArrayList<>();
+        boolean hasSecondaryIndexMap = false;
         Queue<Mutable<ILogicalOperator>> queue = new LinkedList<>(op.getInputs());
         while (!queue.isEmpty()) {
             AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) queue.poll().getValue();
@@ -176,8 +191,7 @@
 
                     dataSourceScanOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
 
-                    assignOp.getInputs()
-                            .add(new MutableObject<>(dataSourceScanOp.getInputs().get(0).getValue()));
+                    assignOp.getInputs().add(new MutableObject<>(dataSourceScanOp.getInputs().get(0).getValue()));
                     dataSourceScanOp.getInputs().get(0).setValue(assignOp);
                 }
             } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
@@ -207,14 +221,153 @@
                                     .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
                         }
                         unnestMapOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
-                        assignOp.getInputs()
-                                .add(new MutableObject<>(unnestMapOp.getInputs().get(0).getValue()));
+                        assignOp.getInputs().add(new MutableObject<>(unnestMapOp.getInputs().get(0).getValue()));
                         unnestMapOp.getInputs().get(0).setValue(assignOp);
+
+                        if (jobGenParams.isPrimaryIndex) {
+                            primaryUnnestMapOps.add(unnestMapOp);
+                        } else {
+                            hasSecondaryIndexMap = true;
+                        }
                     }
                 }
             }
             queue.addAll(descendantOp.getInputs());
         }
+        if (hasSecondaryIndexMap && !primaryUnnestMapOps.isEmpty()) {
+            propagateFilterToPrimaryIndex(primaryUnnestMapOps, filterType, context);
+        }
+    }
+
+    private void propagateFilterToPrimaryIndex(List<UnnestMapOperator> primaryUnnestMapOps, IAType filterType,
+            IOptimizationContext context) throws AlgebricksException {
+        for (UnnestMapOperator primaryOp : primaryUnnestMapOps) {
+            Mutable<ILogicalOperator> assignOrOrderOrIntersect = primaryOp.getInputs().get(0);
+            Mutable<ILogicalOperator> intersectOrSort = assignOrOrderOrIntersect;
+
+            if (assignOrOrderOrIntersect.getValue().getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+                intersectOrSort = assignOrOrderOrIntersect.getValue().getInputs().get(0);
+            }
+
+            switch (intersectOrSort.getValue().getOperatorTag()) {
+                case INTERSECT:
+                    IntersectOperator intersect = (IntersectOperator) (intersectOrSort.getValue());
+                    List<List<LogicalVariable>> filterVars = new ArrayList<>(intersect.getInputs().size());
+                    for (Mutable<ILogicalOperator> mutableOp : intersect.getInputs()) {
+                        ILogicalOperator child = mutableOp.getValue();
+                        while (!child.getOperatorTag().equals(LogicalOperatorTag.UNNEST_MAP)) {
+                            child = child.getInputs().get(0).getValue();
+                        }
+                        UnnestMapOperator unnestMap = (UnnestMapOperator) child;
+                        propagateFilterInSecondaryUnnsetMap(unnestMap, filterType, context);
+
+                        List<LogicalVariable> extraVars = Arrays.asList(unnestMap.getPropagateIndexMinFilterVar(),
+                                unnestMap.getPropagateIndexMaxFilterVar());
+                        filterVars.add(extraVars);
+                    }
+                    if (!filterVars.isEmpty()) {
+                        List<LogicalVariable> outputFilterVars = new ArrayList<>(filterVars.get(0));
+                        IntersectOperator intersectWithFilter =
+                                createIntersectWithFilter(outputFilterVars, filterVars, intersect);
+
+                        intersectOrSort.setValue(intersectWithFilter);
+                        context.computeAndSetTypeEnvironmentForOperator(intersectWithFilter);
+                        setPrimaryFilterVar(primaryOp, outputFilterVars.get(0), outputFilterVars.get(1), context);
+                    }
+                    break;
+                case ORDER:
+                    ILogicalOperator child = intersectOrSort.getValue().getInputs().get(0).getValue();
+                    if (child.getOperatorTag().equals(LogicalOperatorTag.UNNEST_MAP)) {
+                        UnnestMapOperator secondaryMap = (UnnestMapOperator) child;
+
+                        propagateFilterInSecondaryUnnsetMap(secondaryMap, filterType, context);
+
+                        setPrimaryFilterVar(primaryOp, secondaryMap.getPropagateIndexMinFilterVar(),
+                                secondaryMap.getPropagateIndexMaxFilterVar(), context);
+                    }
+                    break;
+                default:
+                    throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+                            intersectOrSort.getValue().getOperatorTag().toString());
+            }
+        }
+    }
+
+    private IntersectOperator createIntersectWithFilter(List<LogicalVariable> outputFilterVars,
+            List<List<LogicalVariable>> filterVars, IntersectOperator intersect) throws AlgebricksException {
+        List<LogicalVariable> outputVars = new ArrayList<>();
+        outputVars.addAll(intersect.getOutputVars());
+        outputVars.addAll(outputFilterVars);
+
+        List<List<LogicalVariable>> compareVars = new ArrayList<>(intersect.getNumInput());
+        for (int i = 0; i < intersect.getNumInput(); i++) {
+            compareVars.add(new ArrayList<>(intersect.getCompareVariables(i)));
+        }
+
+        IntersectOperator intersectWithFilter = new IntersectOperator(outputVars, compareVars, filterVars);
+        intersectWithFilter.getInputs().addAll(intersect.getInputs());
+        return intersectWithFilter;
+    }
+
+    private void propagateFilterInSecondaryUnnsetMap(UnnestMapOperator secondaryUnnest, IAType filterType,
+            IOptimizationContext context) throws AlgebricksException {
+
+        LogicalVariable minIndexFilterVar = context.newVar();
+        LogicalVariable maxIndexFilterVar = context.newVar();
+        secondaryUnnest.markPropagageIndexFilter();
+        secondaryUnnest.getVariables().add(minIndexFilterVar);
+        secondaryUnnest.getVariableTypes().add(filterType);
+        secondaryUnnest.getVariables().add(maxIndexFilterVar);
+        secondaryUnnest.getVariableTypes().add(filterType);
+
+        context.computeAndSetTypeEnvironmentForOperator(secondaryUnnest);
+    }
+
+    private void setPrimaryFilterVar(UnnestMapOperator primaryOp, LogicalVariable minFilterVar,
+            LogicalVariable maxFilterVar, IOptimizationContext context) throws AlgebricksException {
+        primaryOp.setMinFilterVars(Collections.singletonList(minFilterVar));
+        primaryOp.setMaxFilterVars(Collections.singletonList(maxFilterVar));
+
+        List<Mutable<ILogicalExpression>> indexFilterExpression =
+                Arrays.asList(new MutableObject<>(new VariableReferenceExpression(minFilterVar)),
+                        new MutableObject<>(new VariableReferenceExpression(maxFilterVar)));
+
+        primaryOp.setAdditionalFilteringExpressions(indexFilterExpression);
+        context.computeAndSetTypeEnvironmentForOperator(primaryOp);
+    }
+
+    private void assignFilterFromSecondaryUnnestMap(AbstractLogicalOperator op, Dataset dataset,
+            IOptimizationContext context, IAType filterType) throws AlgebricksException {
+        List<UnnestMapOperator> primaryUnnestMapOps = new ArrayList<>();
+        boolean hasSecondaryIndexMap = false;
+        Queue<Mutable<ILogicalOperator>> queue = new LinkedList<>(op.getInputs());
+        while (!queue.isEmpty()) {
+            ILogicalOperator descendantOp = queue.poll().getValue();
+            if (descendantOp.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+                UnnestMapOperator unnestMapOp = (UnnestMapOperator) descendantOp;
+                ILogicalExpression unnestExpr = unnestMapOp.getExpressionRef().getValue();
+                if (unnestExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                    AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
+                    FunctionIdentifier fid = f.getFunctionIdentifier();
+                    if (!fid.equals(BuiltinFunctions.INDEX_SEARCH)) {
+                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, fid.getName());
+                    }
+                    AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
+                    jobGenParams.readFromFuncArgs(f.getArguments());
+                    if (dataset.getDatasetName().compareTo(jobGenParams.datasetName) == 0) {
+                        if (jobGenParams.isPrimaryIndex) {
+                            primaryUnnestMapOps.add(unnestMapOp);
+                        } else {
+                            hasSecondaryIndexMap = true;
+                        }
+                    }
+                }
+            }
+            queue.addAll(descendantOp.getInputs());
+        }
+        if (hasSecondaryIndexMap && !primaryUnnestMapOps.isEmpty()) {
+            propagateFilterToPrimaryIndex(primaryUnnestMapOps, filterType, context);
+        }
     }
 
     private Dataset getDataset(AbstractLogicalOperator op, IOptimizationContext context) throws AlgebricksException {
diff --git a/asterixdb/asterix-app/data/twitter/real.2.adm b/asterixdb/asterix-app/data/twitter/real.2.adm
new file mode 100644
index 0000000..815676c
--- /dev/null
+++ b/asterixdb/asterix-app/data/twitter/real.2.adm
@@ -0,0 +1,5000 @@
+{ "create_at": datetime("2016-01-02T10:00:00.000Z"), "id": 683347019176501248, "text": "@e_tomkiewicz o I know", "in_reply_to_status": 683306019192176642, "in_reply_to_user": 379427210, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 379427210 }}, "user": { "id": 51597840, "name": "Sydney Isroff", "screen_name": "SydneyIsroff", "lang": "en", "location": "null", "create_at": date("2009-06-27"), "description": "Kids are allowed but not welcomed", "followers_count": 408, "friends_count": 78, "statues_count": 13278 }, "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-02T10:00:00.000Z"), "id": 683347019247824896, "text": "Ripley SW Limestone Co. Temp: 46.8°F Wind:0.7mph Pressure: 1005.8mb 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": 46922 }, "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-02T10:00:00.000Z"), "id": 683347019323146240, "text": "@KeepnUpWithElla check yo dm", "in_reply_to_status": -1, "in_reply_to_user": 839418744, "favorite_count": 0, "coordinate": point("-95.2051999,29.9768285"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 839418744 }}, "user": { "id": 95297344, "name": "Hooper$Ambition$", "screen_name": "ChasinDreams414", "lang": "en", "location": "Followin A Dream ", "create_at": date("2009-12-07"), "description": "Young Nigga Tryna Make It Out 14 #SixDouble0 #FreeHk Milwaukee414✈️HTX", "followers_count": 1726, "friends_count": 2025, "statues_count": 83854 }, "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-01-02T10:00:00.000Z"), "id": 683347019839205378, "text": "Low key miss playing battlefield", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 514951505, "name": "Mason Nation ™", "screen_name": "MasoNation757", "lang": "en", "location": "Same", "create_at": date("2012-03-04"), "description": "Somewhere between psychotic and at sonic", "followers_count": 1030, "friends_count": 275, "statues_count": 41730 }, "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-02T10:00:00.000Z"), "id": 683347019985899525, "text": "Back in the saddle again. A Fort Hays State basketball double header vs. Pittsburg State. Women tip at 2pm. Men to follow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 290150645, "name": "Garrett Sager", "screen_name": "TheGarrettSager", "lang": "en", "location": "Hays, America", "create_at": date("2011-04-29"), "description": "As seen on TV. 4 time Award Winning Broadcaster. Award Winning Journalist. Fort Hays State Alumn. Fantasy Football League Commissioner.", "followers_count": 223, "friends_count": 291, "statues_count": 12290 }, "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-02T10:00:00.000Z"), "id": 683347020002783232, "text": "Dad: You know a lot of people, you collect people. Me: it's not like I keep them in a basement.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.27268259,42.37047653"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 92148124, "name": "My-Ishia Cason-Brown", "screen_name": "Myishiacb", "lang": "en", "location": "Hollywood! DTLA!!", "create_at": date("2009-11-23"), "description": "Actress that hop, skipped and jumped from D-town to the glitz, hustle & grind of LA http://www.imdb.me/My-IshiaCason-Brown", "followers_count": 986, "friends_count": 2008, "statues_count": 10985 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redford, MI", "id": "c6637a26ad26239d", "name": "Redford", "place_type": "city", "bounding_box": rectangle("-83.316839,42.355177 -83.266134,42.442875") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-01-02T10:00:00.000Z"), "id": 683347020069912576, "text": "❤️mwaaaa love u too RT @michhh_boss: Wishing my biff @fernyyy12 & pic @jennysmallls a very happy birthday! Love you both ��❤️", "in_reply_to_status": 683286100580896768, "in_reply_to_user": 229995706, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 229995706, 232365647, 159879657 }}, "user": { "id": 159879657, "name": "Jennifer", "screen_name": "jennysmallls", "lang": "en", "location": "Wade County", "create_at": date("2010-06-26"), "description": "null", "followers_count": 1016, "friends_count": 605, "statues_count": 36126 }, "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("2016-01-02T10:00:00.000Z"), "id": 683347020170436608, "text": "Never cool when your dog eats your food.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1862047700, "name": "Keaton Van Dusen", "screen_name": "keaton_van", "lang": "en", "location": "null", "create_at": date("2013-09-13"), "description": "UTEP basketball", "followers_count": 170, "friends_count": 233, "statues_count": 1198 }, "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-02T10:00:00.000Z"), "id": 683347020401217537, "text": "Wind 1.3 mph NE. Barometer 30.332 in, Falling. Temperature 48.7 °F. Rain today 0.00 in. Humidity 56%", "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": 55658 }, "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-02T10:00:00.000Z"), "id": 683347020459970561, "text": "Just dropped my son off at the airport ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 362280590, "name": "Amy-blockhead4life", "screen_name": "whitegirlsmooth", "lang": "en", "location": "New Jersey", "create_at": date("2011-08-25"), "description": "Blockhead for life! Donnie girl, Mom to an amazing son with Cystic Fibrosis. Colts fan", "followers_count": 1983, "friends_count": 1610, "statues_count": 15965 }, "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-02T10:00:00.000Z"), "id": 683347020606619652, "text": "See our latest #TriadArea, NC #job and click to apply: Dental Hygienist - https://t.co/uwp8EOmkJp #Healthcare #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.244216,36.0998596"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TriadArea", "job", "Healthcare", "Hiring", "CareerArc" }}, "user": { "id": 27563554, "name": "Triad Healthcare", "screen_name": "tmj_nct_health", "lang": "en", "location": "Triad Area, NC", "create_at": date("2009-03-29"), "description": "Follow this account for geo-targeted Healthcare job tweets in Triad Area, NC. Need help? Tweet us at @CareerArc!", "followers_count": 662, "friends_count": 328, "statues_count": 264 }, "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-02T10:00:00.000Z"), "id": 683347020917035008, "text": "First class ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 459647579, "name": "BURROLA", "screen_name": "MrBurrola", "lang": "en", "location": "AZ✈️NY", "create_at": date("2012-01-09"), "description": "United States Army", "followers_count": 1110, "friends_count": 859, "statues_count": 41269 }, "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-02T10:00:00.000Z"), "id": 683347020954923008, "text": "@Risssssa13 smooth! I am sure you gave him your number after that!", "in_reply_to_status": 683345003507257345, "in_reply_to_user": 2266589804, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2266589804 }}, "user": { "id": 3238932719, "name": "wagz", "screen_name": "FINSUPWAGZ", "lang": "en", "location": "null", "create_at": date("2015-05-06"), "description": "null", "followers_count": 368, "friends_count": 632, "statues_count": 1772 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Union, OH", "id": "dd9778f306e5d993", "name": "Union", "place_type": "city", "bounding_box": rectangle("-84.338263,39.890999 -84.293224,39.922831") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39113, "countyName": "Montgomery", "cityID": 3978470, "cityName": "Union" } }
+{ "create_at": datetime("2016-01-02T10:00:00.000Z"), "id": 683347020975833089, "text": "Literally you don't even follow me, you're always the first person to watch my stories, back off PLZ you were left in 2015", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287090492, "name": "Broadway Barbie™", "screen_name": "TaylorLinCam", "lang": "en", "location": "Broadway, NYC", "create_at": date("2011-04-24"), "description": "I'm a Barbie girl, in a broadway world. #DCP15 #jewishamericanprincess", "followers_count": 708, "friends_count": 347, "statues_count": 42758 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Buena Vista, FL", "id": "01c6165c7783155f", "name": "Lake Buena Vista", "place_type": "city", "bounding_box": rectangle("-81.538267,28.362991 -81.502787,28.398695") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1237625, "cityName": "Lake Buena Vista" } }
+{ "create_at": datetime("2016-01-02T10:00:00.000Z"), "id": 683347021034577920, "text": "PM @JustinTrudeau & @CanadaTourist - you all are the #beesknees. Also, please consider updates to the #canadianpavilion @DisneyEpcotUS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "beesknees", "canadianpavilion" }}, "user_mentions": {{ 14260960, 2291894478, 743215490 }}, "user": { "id": 34003756, "name": "Brendan Kelly", "screen_name": "goBK", "lang": "en", "location": "El Barrio: DC; CT; NY; NJ", "create_at": date("2009-04-21"), "description": "Politics, Design and Tortillas: Daily Rider and DC Flâneur. Opinions are mine.", "followers_count": 439, "friends_count": 705, "statues_count": 1476 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Sandwich, MA", "id": "6b3ba923b47fb25d", "name": "East Sandwich", "place_type": "city", "bounding_box": rectangle("-70.494736,41.710506 -70.383191,41.75594") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25001, "countyName": "Barnstable", "cityID": 2520380, "cityName": "East Sandwich" } }
+{ "create_at": datetime("2016-01-02T10:00:00.000Z"), "id": 683347021173026816, "text": "One of those days where I really could care less to see or talk to anyone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325089736, "name": "Kay", "screen_name": "Knephew", "lang": "en", "location": "Farmington, MO", "create_at": date("2011-06-27"), "description": "19 • MAC • @JakeMarshall12 ❤️", "followers_count": 531, "friends_count": 669, "statues_count": 6562 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Farmington, MO", "id": "94d9e8f4ac8454bf", "name": "Farmington", "place_type": "city", "bounding_box": rectangle("-90.458548,37.75164 -90.389145,37.820258") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29187, "countyName": "St. Francois", "cityID": 2923752, "cityName": "Farmington" } }
+{ "create_at": datetime("2016-01-02T10:00:00.000Z"), "id": 683347021324021760, "text": "If '16 is \"year of the creator,\" I rly want to learn more from creators. @CarlosGil83, have u met my fave Viner @LadieFantastic ? @Social545", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 34079088, 1014185676, 4169858779 }}, "user": { "id": 116060961, "name": "Chris Strub", "screen_name": "ChrisStrub", "lang": "en", "location": "Watertown, NY", "create_at": date("2010-02-20"), "description": "First to live-stream in all 50 -- & helped nonprofits along the way. Snap / IG / Vine / Periscope / Meerkat (Top70) @ChrisStrub. Inquiries: chrisstrub@gmail.com", "followers_count": 2697, "friends_count": 289, "statues_count": 45278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Watertown, NY", "id": "d4930fd3a1388ef1", "name": "Watertown", "place_type": "city", "bounding_box": rectangle("-75.958806,43.917677 -75.855647,44.014324") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36045, "countyName": "Jefferson", "cityID": 3678608, "cityName": "Watertown" } }
+{ "create_at": datetime("2016-01-02T10:00:00.000Z"), "id": 683347021340753925, "text": "Irving though ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 279157237, "name": "Danny Schmid", "screen_name": "DSchmid21", "lang": "en", "location": "Hull | Westfield State '18", "create_at": date("2011-04-08"), "description": "FTB", "followers_count": 571, "friends_count": 368, "statues_count": 10290 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hull, MA", "id": "6d0bf4ccf087b5f8", "name": "Hull", "place_type": "city", "bounding_box": rectangle("-70.922775,42.258661 -70.825676,42.310371") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25023, "countyName": "Plymouth", "cityID": 2531680, "cityName": "Hull" } }
+{ "create_at": datetime("2016-01-02T10:00:00.000Z"), "id": 683347021512732676, "text": "this cold ain't 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": 2180065839, "name": "genesis", "screen_name": "_itsgenesissss", "lang": "en", "location": "TPA", "create_at": date("2013-11-13"), "description": "SEU'19 || M.", "followers_count": 185, "friends_count": 164, "statues_count": 4544 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gibsonton, FL", "id": "00c23e64064f65f5", "name": "Gibsonton", "place_type": "city", "bounding_box": rectangle("-82.412765,27.777215 -82.326561,27.868938") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1225900, "cityName": "Gibsonton" } }
+{ "create_at": datetime("2016-01-02T10:00:00.000Z"), "id": 683347021571428352, "text": "Trusting you again soon afterwards can", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3093698424, "name": "Mac Stephenson", "screen_name": "MacSeniors2016", "lang": "en", "location": "San Francisco, CA ", "create_at": date("2015-03-17"), "description": "Danbury Senior #52 / this is my best friend is @Austin14_", "followers_count": 2656, "friends_count": 5004, "statues_count": 25837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marblehead, OH", "id": "00b0e2b53b9ae871", "name": "Marblehead", "place_type": "city", "bounding_box": rectangle("-82.741681,41.510744 -82.711397,41.545162") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39123, "countyName": "Ottawa", "cityID": 3947502, "cityName": "Marblehead" } }
+{ "create_at": datetime("2016-01-02T10:00:00.000Z"), "id": 683347021630066690, "text": "Naw but forreal 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": 588747155, "name": "GoofyGates", "screen_name": "LaDaishaBriAnte", "lang": "en", "location": "null", "create_at": date("2012-05-23"), "description": "Numb. Love is stronger than pride ❗️#LongLiveKushyyyy", "followers_count": 2302, "friends_count": 1850, "statues_count": 73371 }, "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-01-02T10:00:00.000Z"), "id": 683347021818822656, "text": "You know what would make the day better? A hello from @BQQuinn. My day would rock after that. :)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 454237315 }}, "user": { "id": 28194070, "name": "Stacey Crowder", "screen_name": "poothead", "lang": "en", "location": "Indiana ", "create_at": date("2009-04-01"), "description": "Computer geek, web designer, music fanatic, blogger,bookworm, animal lover, parent of childhood leukemia survivor. Impractical Jokers Rob Thomas #teamQ #bqquinn", "followers_count": 1536, "friends_count": 1899, "statues_count": 2624 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jasper, IN", "id": "aae8ae43f0fd518e", "name": "Jasper", "place_type": "city", "bounding_box": rectangle("-87.002013,38.362725 -86.894516,38.440426") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18037, "countyName": "Dubois", "cityID": 1837782, "cityName": "Jasper" } }
+{ "create_at": datetime("2016-01-02T10:00:00.000Z"), "id": 683347021902696448, "text": "A pixie sprit just told me that I make her feel safe. Can any mortal man ever hope to hear a better compliment in this life or the next?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 405161384, "name": "the mean reds", "screen_name": "soverylittlenow", "lang": "en", "location": "null", "create_at": date("2011-11-04"), "description": "if you light me on fire, i will in fact burn", "followers_count": 513, "friends_count": 357, "statues_count": 15883 }, "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-01-02T10:00:00.000Z"), "id": 683347021902790657, "text": "Who 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": 490537307, "name": "Ye", "screen_name": "NanahPejay", "lang": "en", "location": "null", "create_at": date("2012-02-12"), "description": "null", "followers_count": 799, "friends_count": 653, "statues_count": 10581 }, "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-01-02T10:00:00.000Z"), "id": 683347022208970752, "text": "The walk. #thewalk #dogs #puppies #puppack @ Villebois Community https://t.co/3pXjc9xWna", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.79623921,45.30943208"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thewalk", "dogs", "puppies", "puppack" }}, "user": { "id": 1184212243, "name": "Luis Centeno", "screen_name": "Batou323", "lang": "en", "location": "null", "create_at": date("2013-02-15"), "description": "null", "followers_count": 6, "friends_count": 35, "statues_count": 504 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wilsonville, OR", "id": "b08b39af5eca8b4c", "name": "Wilsonville", "place_type": "city", "bounding_box": rectangle("-122.807286,45.281815 -122.742838,45.340517") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41005, "countyName": "Clackamas", "cityID": 4182800, "cityName": "Wilsonville" } }
+{ "create_at": datetime("2016-01-02T10:00:00.000Z"), "id": 683347022225604608, "text": "Birthday in 21 days ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1427040950, "name": "kkkeeeiiilllaaa", "screen_name": "keila_adrianaa", "lang": "en", "location": "Yay Area ", "create_at": date("2013-05-13"), "description": "{university of cali - davis} {nicaragüense}", "followers_count": 485, "friends_count": 324, "statues_count": 17597 }, "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-02T10:00:00.000Z"), "id": 683347022242447360, "text": "#CareerArc #Retail #Job alert: Retail Store Positions | CVS Health | #Knoxville, TN https://t.co/crXJ87nVMc #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.927451,36.028355"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CareerArc", "Retail", "Job", "Knoxville", "Jobs", "Hiring" }}, "user": { "id": 59989371, "name": "TMJ-TN Retail Jobs", "screen_name": "tmj_tn_retail", "lang": "en", "location": "Tennessee", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in Tennessee Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 401, "friends_count": 300, "statues_count": 759 }, "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-02T10:00:00.000Z"), "id": 683347022674554880, "text": "By sapping the will & opportunity 2 find work welfare perpetuates the poverty the war on poverty was 2 end. https://t.co/n2gSInRu4Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "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": 4363, "friends_count": 4790, "statues_count": 49188 }, "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-02T10:00:00.000Z"), "id": 683347023014199296, "text": "@TylerDreiling What's your missed call count up to now? Gotta be double digits.", "in_reply_to_status": -1, "in_reply_to_user": 43013485, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 43013485 }}, "user": { "id": 2464931059, "name": "Sean Moran", "screen_name": "powercatsean", "lang": "en", "location": "null", "create_at": date("2014-04-26"), "description": "null", "followers_count": 38, "friends_count": 287, "statues_count": 38 }, "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-02T10:00:00.000Z"), "id": 683347023089762305, "text": "Not even sorry for all my Ryan tweets because I'm obsessed with him", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 746068316, "name": "Donna✨", "screen_name": "Donna_Marello", "lang": "en", "location": "null", "create_at": date("2012-08-08"), "description": "null", "followers_count": 311, "friends_count": 347, "statues_count": 27198 }, "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-02T10:00:01.000Z"), "id": 683347023500677120, "text": "@ThatCrazyMJ beautiful pictures Chica", "in_reply_to_status": 683133397112229890, "in_reply_to_user": 3438986832, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3438986832 }}, "user": { "id": 4519972311, "name": "Stephanie Sarah", "screen_name": "Steffis50", "lang": "en", "location": "Tacoma, WA", "create_at": date("2015-12-17"), "description": "Trans woman, transgender at 50 years of age. Starting HRT January 15th 2016, and Loving it.", "followers_count": 115, "friends_count": 183, "statues_count": 157 }, "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-02T10:00:01.000Z"), "id": 683347023672795137, "text": "https://t.co/MWLvrDdtpQ kaysey coleman cavoc jada louie kalynn 6-15-2016 johnathon wingfield @CharmingCharlie CAVOC https://t.co/CivOA1iVgE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 44379956 }}, "user": { "id": 3199413230, "name": "Preciosa", "screen_name": "5e3e26ad31bc4a1", "lang": "en", "location": "null", "create_at": date("2015-05-17"), "description": "null", "followers_count": 65, "friends_count": 174, "statues_count": 2908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Cleveland, OH", "id": "36ba7375e4498732", "name": "East Cleveland", "place_type": "city", "bounding_box": rectangle("-81.598231,41.515406 -81.555538,41.548046") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3923380, "cityName": "East Cleveland" } }
+{ "create_at": datetime("2016-01-02T10:00:01.000Z"), "id": 683347023894974464, "text": "OMM!!!!! https://t.co/DXHgQTVrGa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 506420712, "name": "JAN21♒️", "screen_name": "Breyanniaa", "lang": "en", "location": "TANGLEWOOD,LA", "create_at": date("2012-02-27"), "description": "I'm worth it fam.", "followers_count": 1845, "friends_count": 1472, "statues_count": 69418 }, "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-01-02T10:00:01.000Z"), "id": 683347024138350592, "text": "Temp: 68.9°F | Humidity: 66% | Wind: NE @ 0.0 mph | Barometer: 30.10 in | Dewpoint: 57.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": 164183 }, "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-02T10:00:01.000Z"), "id": 683347024440340481, "text": "Been waiting to get treated at the ER for almost 2 hours now. Really glad I'm not in mortal peril", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16602986, "name": "Liz Schanke", "screen_name": "lizschanke", "lang": "en", "location": "Kenosha", "create_at": date("2008-10-05"), "description": "photographer. artist. coffee. jesus. music. dreams. midwest girl with a heart for travel.", "followers_count": 187, "friends_count": 168, "statues_count": 33364 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pleasant Prairie, WI", "id": "01ce73922fe260b2", "name": "Pleasant Prairie", "place_type": "city", "bounding_box": rectangle("-87.952181,42.492766 -87.801335,42.574318") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55059, "countyName": "Kenosha", "cityID": 5563300, "cityName": "Pleasant Prairie" } }
+{ "create_at": datetime("2016-01-02T10:00:01.000Z"), "id": 683347024473927680, "text": "Wind 0 mph --. Barometer 30.34 in, Falling. Temperature 46.6 °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": 19921 }, "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-02T10:00:01.000Z"), "id": 683347024897429505, "text": "@Maldita_Elo ��������", "in_reply_to_status": -1, "in_reply_to_user": 62702940, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 62702940 }}, "user": { "id": 61366106, "name": "Kenny Bout It, Not U", "screen_name": "Flyer_thanu2", "lang": "en", "location": "Chicago/Blue Island, IL", "create_at": date("2009-07-29"), "description": "28 years old #80sBaby I'm only here for laughs, sports, and music. Forget a debate #49ers #Lakers #OKC Instagram: Kenny_DatDude", "followers_count": 4939, "friends_count": 3583, "statues_count": 549782 }, "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-01-02T10:00:01.000Z"), "id": 683347025140801536, "text": "#HolaSebastian tweeteame una selfie tuya de ahora! @SEBTSB x24", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "HolaSebastian" }}, "user_mentions": {{ 1631128196 }}, "user": { "id": 2621545545, "name": "sarah", "screen_name": "confusemendes", "lang": "en", "location": "New York, USA", "create_at": date("2014-06-18"), "description": "* ✧ · . ☾ *| njh | jw | sm |* ✧ ·. ☾ *", "followers_count": 2704, "friends_count": 2079, "statues_count": 22264 }, "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-01-02T10:00:01.000Z"), "id": 683347025270804480, "text": "@TesterTrish what?", "in_reply_to_status": 683346705866186752, "in_reply_to_user": 3722566695, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3722566695 }}, "user": { "id": 3222587749, "name": "Nicole Tester", "screen_name": "nicole_tester", "lang": "en", "location": "null", "create_at": date("2015-05-21"), "description": "Michigan. January 26th leave date for Air Force Basic", "followers_count": 302, "friends_count": 198, "statues_count": 4102 }, "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-01-02T10:00:01.000Z"), "id": 683347025337778176, "text": "Want to work in #Atlanta, GA? View our latest opening: https://t.co/Y8cEAxjYfY #IT #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.3883717,33.755711"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Atlanta", "IT", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 21298824, "name": "Atlanta IT Jobs", "screen_name": "tmj_atl_it", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-02-19"), "description": "Follow this account for geo-targeted Software Dev. - General/IT job tweets in Atlanta, GA. Need help? Tweet us at @CareerArc!", "followers_count": 906, "friends_count": 361, "statues_count": 536 }, "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-02T10:00:01.000Z"), "id": 683347025337954304, "text": "Big surprise, it's fair and 28ºF. #MorrisWeather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.9,45.59"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MorrisWeather" }}, "user": { "id": 1931463259, "name": "Morris Weather", "screen_name": "MorrisMNWeather", "lang": "en", "location": "Morris, MN", "create_at": date("2013-10-03"), "description": "A place to find the current weather in Morris, MN. Run by @bman4789\n\nScheduled tweets are at 7 AM, 12 PM, 3 PM, 6 PM, and 10 PM. Daily forecast at 6 AM.", "followers_count": 21, "friends_count": 1, "statues_count": 1767 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morris, MN", "id": "dc943406a248a125", "name": "Morris", "place_type": "city", "bounding_box": rectangle("-95.933929,45.564348 -95.883709,45.607334") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27149, "countyName": "Stevens", "cityID": 2744242, "cityName": "Morris" } }
+{ "create_at": datetime("2016-01-02T10:00:01.000Z"), "id": 683347025363103744, "text": "they call me short, but i started at the top", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 938225714, "name": "Han Solo", "screen_name": "TrappHam", "lang": "en", "location": "Fenton, MI", "create_at": date("2012-11-09"), "description": "I make bad jokes and can touch my tongue to my nose", "followers_count": 236, "friends_count": 324, "statues_count": 4592 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Blanc, MI", "id": "0022acf0052352cf", "name": "Grand Blanc", "place_type": "city", "bounding_box": rectangle("-83.697158,42.851374 -83.570346,42.961471") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2633280, "cityName": "Grand Blanc" } }
+{ "create_at": datetime("2016-01-02T10:00:01.000Z"), "id": 683347025404928000, "text": "The Big 12 continues to let the officiating of the conference be sub par", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 197275367, "name": "hunter severn", "screen_name": "htsevern", "lang": "en", "location": "Manhattan, Kansas", "create_at": date("2010-09-30"), "description": "null", "followers_count": 190, "friends_count": 1049, "statues_count": 5491 }, "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-02T10:00:01.000Z"), "id": 683347025509797888, "text": "#park #conservation #water #huronriver #land #firsthike #ypsireal #ypsi #outdoors #ypsilanti… https://t.co/3JZnMgdItL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.67430841,42.23181481"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "park", "conservation", "water", "huronriver", "land", "firsthike", "ypsireal", "ypsi", "outdoors", "ypsilanti" }}, "user": { "id": 3759815295, "name": "Kaylan Petrie", "screen_name": "bywaycreative", "lang": "en", "location": "Ypsilanti, MI", "create_at": date("2015-09-24"), "description": "Design • Vintage • Handmade", "followers_count": 14, "friends_count": 53, "statues_count": 71 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsfield, MI", "id": "00d73c6d35bf8b5a", "name": "Pittsfield", "place_type": "city", "bounding_box": rectangle("-83.785083,42.142414 -83.641471,42.259412") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26161, "countyName": "Washtenaw" } }
+{ "create_at": datetime("2016-01-02T10:00:01.000Z"), "id": 683347025669271552, "text": "Josh is playing his rabbits songs by The Weeknd and Usher in hopes that they'll make more little rabbits ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 164506265, "name": "Ivan", "screen_name": "whitegrapecigar", "lang": "en", "location": "336 Saxapahaw", "create_at": date("2010-07-08"), "description": "nada man", "followers_count": 328, "friends_count": 320, "statues_count": 13882 }, "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-01-02T10:00:01.000Z"), "id": 683347025950187520, "text": "Taking over until 5! Anyone joining me? https://t.co/7BX5OLMHwv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4355563221, "name": "Zack", "screen_name": "ZackWCTY", "lang": "en", "location": "Norwich, CT", "create_at": date("2015-12-02"), "description": "Weekend radio host for 97.7 WCTY, and Intern for Jimmy & Shelly!", "followers_count": 45, "friends_count": 82, "statues_count": 106 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwich, CT", "id": "ca059e12283afa1e", "name": "Norwich", "place_type": "city", "bounding_box": rectangle("-72.175195,41.490724 -72.030241,41.607853") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9011, "countyName": "New London", "cityID": 956200, "cityName": "Norwich" } }
+{ "create_at": datetime("2016-01-02T10:00:01.000Z"), "id": 683347026268913665, "text": "Need to check out Jauan's film.....he added Maryland late but I need to check him out when I get back", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 291506241, "name": "Jasmine B.", "screen_name": "lovejazzy94", "lang": "en", "location": "PG County,MD", "create_at": date("2011-05-01"), "description": "21. God 1st @WMUCSports Volleyball & Basketball reporter for the MarylandTerrapins|#BrooksWeekly writer| #UMD|", "followers_count": 2414, "friends_count": 1514, "statues_count": 244920 }, "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.965622,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-01-02T10:00:01.000Z"), "id": 683347026306809856, "text": "Carried them to the airport bar. Boots are on. As well as a Crown Royal double.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24539756, "name": "amanda .", "screen_name": "manderrx27", "lang": "en", "location": "301.", "create_at": date("2009-03-15"), "description": "live & let live.", "followers_count": 277, "friends_count": 213, "statues_count": 40042 }, "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-02T10:00:01.000Z"), "id": 683347026323582976, "text": "Wind 4.0 mph WSW. Barometer 30.223 in, Falling. Temperature 41.1 °F. Rain today 0.00 in. Humidity 61%", "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": 16578 }, "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-02T10:00:01.000Z"), "id": 683347026407325696, "text": "Just finished the first episode of #MakingAMurderer and I am so disillusioned and mad right now I could cry.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MakingAMurderer" }}, "user": { "id": 406484638, "name": "KathleenInCLE", "screen_name": "KathleenInCLE", "lang": "en", "location": "Cleveland, Ohio", "create_at": date("2011-11-06"), "description": "West Side dweller, East Side worker, Downtown lover. Opinions are mine alone. Feel free to tune me out. My entire family does. #happyinCLE", "followers_count": 991, "friends_count": 1543, "statues_count": 9114 }, "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-02T10:00:01.000Z"), "id": 683347026428473344, "text": "13:00 37.6°F Feels:28.8°F (Hi37.8°F/Lo28.8°F) Hum:57% Wnd:W 14.8MPH Baro:30.12in. 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": 161, "friends_count": 261, "statues_count": 24227 }, "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-02T10:00:01.000Z"), "id": 683347026654965766, "text": "Emma isn't answering me and I need to rant on and on to her������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2718606633, "name": "Lex↟", "screen_name": "alevsxo", "lang": "en", "location": "Massachusetts, USA", "create_at": date("2014-07-20"), "description": "Leo♌️ // MA // Gbaby 6.10.12❤️", "followers_count": 271, "friends_count": 590, "statues_count": 11711 }, "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-01-02T10:00:01.000Z"), "id": 683347026721902592, "text": "https://t.co/AHyNgJnUji", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 89137546, "name": "Jamie Alquiza", "screen_name": "jamiealquiza", "lang": "en", "location": "Boulder, CO", "create_at": date("2009-11-11"), "description": "Mountain-bound technologist. Avoids clusters with less than 1,000 cores. Believer that ideas are terrible.", "followers_count": 324, "friends_count": 300, "statues_count": 4284 }, "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-02T10:00:01.000Z"), "id": 683347026759823360, "text": "Halftime:@JMUMBasketball 35, @DelawareMBB 20.\n\nWinston Grays leading the Dukes with 13 points at the break.\nAnthony Mosley has 10 for UD.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 875831407, 3299387830 }}, "user": { "id": 2500301656, "name": "Travon Miles", "screen_name": "TMiles47ABC", "lang": "en", "location": "Maryland/Delaware", "create_at": date("2014-04-21"), "description": "Dad. Sports Anchor/Reporter @47ABC. Co-Host of the Delmarva Sports Insider. Son of Salisbury. Wi-Hi/Salisbury U Grad. Living out my dream! Opinions are my own.", "followers_count": 1177, "friends_count": 499, "statues_count": 6469 }, "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-01-02T10:00:01.000Z"), "id": 683347026780762112, "text": "01/02@13:00 - Temp 42.3F, WC 38.9F. Wind 5.3mph WSW, Gust 11.0mph. Bar 30.083in, Falling. Rain 0.25in. Hum 57%. UV 1.7. SolarRad 439.", "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": 45978 }, "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-02T10:00:01.000Z"), "id": 683347026805932032, "text": "Is it weird my mother in law calls my phone before anyone else ? Lol ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3759766572, "name": "Kimberly Alexis", "screen_name": "kimberlylexisss", "lang": "en", "location": "McKeesport, PA", "create_at": date("2015-10-02"), "description": "9.29.15 ✨❣", "followers_count": 26, "friends_count": 41, "statues_count": 448 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McKeesport, PA", "id": "2c9a7524524ac60b", "name": "McKeesport", "place_type": "city", "bounding_box": rectangle("-79.885623,40.31858 -79.815855,40.364977") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4246256, "cityName": "McKeesport" } }
+{ "create_at": datetime("2016-01-02T10:00:01.000Z"), "id": 683347027221180416, "text": "Hate when bitches think they allat ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2544914336, "name": "Ron ", "screen_name": "almiighty_ron", "lang": "en", "location": " looking up to dolo ❤️", "create_at": date("2014-06-03"), "description": "Matthew 19:26 |", "followers_count": 232, "friends_count": 196, "statues_count": 521 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodlawn, MD", "id": "bf64a5dd9b5f1cf2", "name": "Woodlawn", "place_type": "city", "bounding_box": rectangle("-76.915396,38.938828 -76.711092,39.327477") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2486525, "cityName": "Woodlawn" } }
+{ "create_at": datetime("2016-01-02T10:00:01.000Z"), "id": 683347027258806272, "text": "https://t.co/0MINN9Ngcl take a look at this great tee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4168351522, "name": "MAK S4DISTIC JOKER", "screen_name": "S4DISTICJOKER", "lang": "en", "location": "Gotham City", "create_at": date("2015-11-11"), "description": "xb1 S4DISTIC JOKER MAK team member,sponsored by @Spidey_Grips, @brimanscustoms, and @westcoastchill. Gamer,streamer, father, http://youtube.com/channel/UCNW52…", "followers_count": 711, "friends_count": 681, "statues_count": 10283 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenfield, WI", "id": "7f42154616cee8fa", "name": "Greenfield", "place_type": "city", "bounding_box": rectangle("-88.069295,42.930385 -87.948504,42.988186") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5531175, "cityName": "Greenfield" } }
+{ "create_at": datetime("2016-01-02T10:00:02.000Z"), "id": 683347027833565184, "text": "We climb mountains in our free time (he also promised me a donut after ��) @ Top of Cowels… https://t.co/dl67WmOtb0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.03196664,32.81269525"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28920930, "name": "Elizabeth", "screen_name": "LizStout30", "lang": "en", "location": "Raleigh be the city", "create_at": date("2009-04-04"), "description": "NCSU educated. XΩ alum. Follower of Christ. Clear eyes, full hearts, can't lose!", "followers_count": 274, "friends_count": 319, "statues_count": 5041 }, "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-02T10:00:02.000Z"), "id": 683347027875336192, "text": "Money can't buy you love cuz' it's over priced", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 966079052, "name": "Ashlie☾", "screen_name": "ashlieeeeee__", "lang": "en", "location": "Pennsylvania, USA", "create_at": date("2012-11-23"), "description": "null", "followers_count": 1497, "friends_count": 1991, "statues_count": 9348 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mohnton, PA", "id": "3858590dcead9a42", "name": "Mohnton", "place_type": "city", "bounding_box": rectangle("-76.00032,40.277759 -75.976028,40.295263") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42011, "countyName": "Berks", "cityID": 4250272, "cityName": "Mohnton" } }
+{ "create_at": datetime("2016-01-02T10:00:02.000Z"), "id": 683347028110258176, "text": "If you're a #Hospitality professional in #Milwaukee, WI, check out this #job: https://t.co/Cz7nZNmc8z #Hiring https://t.co/JIzEzO6KuG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.9064736,43.0389025"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "Milwaukee", "job", "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": 375, "friends_count": 460, "statues_count": 2968 }, "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-02T10:00:02.000Z"), "id": 683347028374589440, "text": "Took a stroll through Heavenly Hills today... I can definitely get used to this! �������� @ Heavenly… https://t.co/rVmQ27cb1D", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.64632634,38.70565484"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 104426262, "name": "Stef Azzarello", "screen_name": "TheGr8Bambina", "lang": "en", "location": "St Louis", "create_at": date("2010-01-13"), "description": "Girl. Short. Italian. Descriptive words followed by periods. NFL. Dogs. Ppl. (in that order)..Home Cookin. Outdoors. Laughing. Get Out.", "followers_count": 106, "friends_count": 423, "statues_count": 883 }, "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": 17027, "countyName": "Clinton" } }
+{ "create_at": datetime("2016-01-02T10:00:02.000Z"), "id": 683347028684869633, "text": "Temp: 28.8°F - Dew Point: 18.6° - Wind: 14.7 mph - Gust: 19.7 - Rain Today: 0.00in. - Pressure: 30.03in, - 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": 11423 }, "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-02T10:00:02.000Z"), "id": 683347028882010113, "text": "PAINTER - King's Creek Plantation: (#Williamsburg, VA) https://t.co/VQZvreDRS0 #Construction #va #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.6350285,37.2668356"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Williamsburg", "Construction", "va", "Job", "Jobs", "Hiring" }}, "user": { "id": 3015648286, "name": "King's Creek Jobs", "screen_name": "KingsCreekJobs", "lang": "en", "location": "null", "create_at": date("2015-02-03"), "description": "Check here for current job opportunities with King's Creek Plantation, in beautiful Williamsburg, VA.", "followers_count": 21, "friends_count": 46, "statues_count": 5 }, "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": 51199, "countyName": "York" } }
+{ "create_at": datetime("2016-01-02T10:00:02.000Z"), "id": 683347028970201089, "text": "2h 5m 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": 374, "friends_count": 888, "statues_count": 2486 }, "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-01-02T10:00:02.000Z"), "id": 683347029188329473, "text": "The Mighty Ducks is one of the greatest sports movies ever", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 607604724, "name": "Kevdogg Lulu", "screen_name": "benthebaker11", "lang": "en", "location": "PENNRIDGE '17", "create_at": date("2012-06-13"), "description": "what do you mean?", "followers_count": 253, "friends_count": 521, "statues_count": 1023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Perkasie, PA", "id": "384bf31d2ff3c68e", "name": "Perkasie", "place_type": "city", "bounding_box": rectangle("-75.311441,40.352315 -75.274748,40.388979") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42017, "countyName": "Bucks", "cityID": 4259384, "cityName": "Perkasie" } }
+{ "create_at": datetime("2016-01-02T10:00:02.000Z"), "id": 683347029280616448, "text": "Temp: 45.6°F Wind:1.8mph Pressure: 30.354hpa 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": 55659 }, "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-02T10:00:02.000Z"), "id": 683347029553102848, "text": "@Airykah1382 omg I have to find it and send it to you", "in_reply_to_status": 683344432993800192, "in_reply_to_user": 32275557, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 32275557 }}, "user": { "id": 533675851, "name": "Aliyya⚜", "screen_name": "Whats_Good_Li", "lang": "en", "location": "Southern TRILLinois| STL", "create_at": date("2012-03-22"), "description": "You could be the king, but watch the queen conquer #Nursing #GoNecks #WIU17", "followers_count": 1323, "friends_count": 1019, "statues_count": 38089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glen Carbon, IL", "id": "e3d941a68af042e7", "name": "Glen Carbon", "place_type": "city", "bounding_box": rectangle("-90.026138,38.738099 -89.92845,38.787788") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17119, "countyName": "Madison", "cityID": 1729639, "cityName": "Glen Carbon" } }
+{ "create_at": datetime("2016-01-02T10:00:02.000Z"), "id": 683347030329049089, "text": "Want to work in #PLANO, TX? View our latest opening: https://t.co/0JwzUGq3l4 #Hospitality #Job #Jobs #Hiring #CareerArc", "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": {{ "PLANO", "Hospitality", "Job", "Jobs", "Hiring", "CareerArc" }}, "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": 605, "friends_count": 294, "statues_count": 3116 }, "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-02T10:00:02.000Z"), "id": 683347030425509889, "text": "Valley Health #Healthcare #Job: Risk Manager #VirginiaHealthCareJobs (#Winchester, VA) https://t.co/ftb0r0CX9w #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.15,39.19"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "Job", "VirginiaHealthCareJobs", "Winchester", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 128213036, "name": "TMJ-VAW Health Jobs", "screen_name": "tmj_VAW_health", "lang": "en", "location": "Winchester, VA", "create_at": date("2010-03-31"), "description": "Follow this account for geo-targeted Healthcare job tweets in Winchester, VA. Need help? Tweet us at @CareerArc!", "followers_count": 386, "friends_count": 330, "statues_count": 226 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winchester, VA", "id": "b4add85f41a3e371", "name": "Winchester", "place_type": "city", "bounding_box": rectangle("-78.201069,39.13086 -78.137654,39.204922") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51840, "countyName": "Winchester", "cityID": 5186720, "cityName": "Winchester" } }
+{ "create_at": datetime("2016-01-02T10:00:02.000Z"), "id": 683347030475866112, "text": "Liberty Mutual Insurance #internship #Job: Sales Internship Program – Hawaii (#Kailua, HI) https://t.co/1Wl1NHJ7W8 #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.7394444,21.4022222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "internship", "Job", "Kailua", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 154601054, "name": "TMJ-HON Intern Jobs", "screen_name": "tmj_HON_intern", "lang": "en", "location": "Honolulu, HI", "create_at": date("2010-06-11"), "description": "Follow this account for geo-targeted Internships job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 261, "friends_count": 256, "statues_count": 4 }, "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-01-02T10:00:02.000Z"), "id": 683347030488543232, "text": "I finished oth last night.. What am i going to do with my 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": 565909539, "name": "alyssa", "screen_name": "AlyssaRadoicic", "lang": "en", "location": "Oak Lawn, IL", "create_at": date("2012-04-28"), "description": "SXU '19", "followers_count": 321, "friends_count": 523, "statues_count": 2102 }, "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-01-02T10:00:02.000Z"), "id": 683347030505316353, "text": "Nothing like knowing you are going to be let down right before it happens", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2235543823, "name": "ℓαℓαα✝", "screen_name": "lalaakohn", "lang": "en", "location": "somewhere w/ramses", "create_at": date("2013-12-07"), "description": "19- model for ATP •RME • don't slide in my DMs I'm not interested", "followers_count": 929, "friends_count": 921, "statues_count": 4358 }, "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-02T10:00:02.000Z"), "id": 683347030643720193, "text": "@VTav22 https://t.co/pCJ6M6JTNs", "in_reply_to_status": -1, "in_reply_to_user": 321028433, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 321028433 }}, "user": { "id": 323570029, "name": "Jake Strauss", "screen_name": "JakeStrauss", "lang": "en", "location": "Buffalo/State College", "create_at": date("2011-06-24"), "description": "Think Big Be Big. @THON 2016 Hospitality Captain", "followers_count": 559, "friends_count": 406, "statues_count": 16700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Harris Hill, NY", "id": "c08efac9b513fdb6", "name": "Harris Hill", "place_type": "city", "bounding_box": rectangle("-78.696945,42.956161 -78.617366,42.989592") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3632391, "cityName": "Harris Hill" } }
+{ "create_at": datetime("2016-01-02T10:00:02.000Z"), "id": 683347030660349952, "text": "Can you recommend anyone for this #job? Sales Manager 100290 - https://t.co/MdJdUS6toJ #Irwindale, CA #Sales #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.9353413,34.1069527"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Irwindale", "Sales", "Hiring", "CareerArc" }}, "user": { "id": 21298575, "name": "LA Sales Jobs", "screen_name": "tmj_lax_sales", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-02-19"), "description": "Follow this account for geo-targeted Sales job tweets in Los Angeles, CA. Need help? Tweet us at @CareerArc!", "followers_count": 672, "friends_count": 314, "statues_count": 787 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irwindale, CA", "id": "01162bc41f757d0f", "name": "Irwindale", "place_type": "city", "bounding_box": rectangle("-118.00695,34.076917 -117.925293,34.130208") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636826, "cityName": "Irwindale" } }
+{ "create_at": datetime("2016-01-02T10:00:02.000Z"), "id": 683347030895235072, "text": "47.2F (Feels: 47.2F) - Humidity: 57% - Wind: 8.3mph NE - Gust: 11.4mph - Pressure: 1040.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": 220710 }, "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-02T10:00:02.000Z"), "id": 683347031042187264, "text": "San Diego Intl to Detroit Metro Wayne Co on United for $283 #CheapFlights #SAN https://t.co/yKhLDkDrsv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.1894,32.733299"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CheapFlights", "SAN" }}, "user": { "id": 756129644, "name": "Flight Fishing", "screen_name": "FlightFishing", "lang": "en", "location": "Orlando, FL", "create_at": date("2012-08-13"), "description": "Collecting the internet's best airfares for flexible explorers. Sign up for free at http://www.flightfishing.com", "followers_count": 1127, "friends_count": 1505, "statues_count": 7946 }, "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-02T10:00:02.000Z"), "id": 683347031071432705, "text": "\"This ain't no punch lines ..\nNigga yo punches lying !\nCause the punch you Eva gave was Hawaiian\" �� - @ItsKiddKidd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 173659159 }}, "user": { "id": 77068504, "name": "Sir Flex-ALOT #Rx3", "screen_name": "papitodvnny", "lang": "en", "location": "New Warleans ", "create_at": date("2009-09-24"), "description": "R'eal | R'ecognize | R'eal Ent. #Rx3 #GotMyOwnWave #LoveNLoyaly #NeverChangedUp #YUMad #ARTIST. PapitoDvnny504@gmail.com", "followers_count": 1329, "friends_count": 908, "statues_count": 82898 }, "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-02T10:00:02.000Z"), "id": 683347031490990082, "text": "@Richkid_life �� no where close bro lol.", "in_reply_to_status": 683346149563052033, "in_reply_to_user": 248207523, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 248207523 }}, "user": { "id": 49496657, "name": "TommyTPhotography", "screen_name": "TommyT_", "lang": "en", "location": "Brooklyn-NY", "create_at": date("2009-06-21"), "description": "CEO & Founder of: TommyTPhotography LLC.", "followers_count": 808, "friends_count": 688, "statues_count": 8913 }, "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-02T10:00:02.000Z"), "id": 683347031549710336, "text": "Wind 1.6 mph NNW. Barometer 29.96 in, Falling. Temperature 34.0 °F. Rain today 0.00 in. Humidity 62%", "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": 156370 }, "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-02T10:00:02.000Z"), "id": 683347031553904641, "text": "QRT (@ Aulani, A Disney Resort & Spa - @disneyaulani in Kapolei, HI) https://t.co/IVbci0wEg5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-158.12378085,21.33923945"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 485036177 }}, "user": { "id": 13608552, "name": "Bryan", "screen_name": "KH6EF", "lang": "en", "location": "Hawaii, USA", "create_at": date("2008-02-17"), "description": "Radio and technology enthusiast. Technical Specialist for Walt Disney Parks and Resorts. Tweets are my own and do not reflect the opinion of The Walt Disney Co.", "followers_count": 302, "friends_count": 848, "statues_count": 1946 }, "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-01-02T10:00:02.000Z"), "id": 683347031579033601, "text": "@Wale \nShould I? \n#MoveToBoys2016", "in_reply_to_status": -1, "in_reply_to_user": 17929027, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MoveToBoys2016" }}, "user_mentions": {{ 17929027 }}, "user": { "id": 281202179, "name": "Jessica", "screen_name": "Fufu___Queen", "lang": "en", "location": "The Promised Land", "create_at": date("2011-04-12"), "description": "African➡ Nigerian ➡ Igbo MSU November 9♏♏ It all happens for a reason", "followers_count": 392, "friends_count": 513, "statues_count": 8350 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Essex, MD", "id": "8a8002ffef902f59", "name": "Essex", "place_type": "city", "bounding_box": rectangle("-76.489916,39.270065 -76.381559,39.330308") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2426600, "cityName": "Essex" } }
+{ "create_at": datetime("2016-01-02T10:00:02.000Z"), "id": 683347031599890432, "text": "@kristinspike5 will do", "in_reply_to_status": 683330927808876544, "in_reply_to_user": 370523938, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 370523938 }}, "user": { "id": 417593348, "name": "sairuh", "screen_name": "whitechoc_olate", "lang": "en", "location": "lll•XXll•XlV", "create_at": date("2011-11-20"), "description": "null", "followers_count": 542, "friends_count": 425, "statues_count": 11188 }, "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-01-02T10:00:02.000Z"), "id": 683347031600009216, "text": "I'm at Old Den in Huntington, IN https://t.co/0BtrSFpiUN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.50457111,40.88645155"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325382809, "name": "Xneotechnethia", "screen_name": "Xneotechnethia", "lang": "en", "location": "40.886749,-85.504161", "create_at": date("2011-06-27"), "description": "Just A Little Orange Fox.", "followers_count": 8, "friends_count": 34, "statues_count": 954 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington, IN", "id": "5a23da841b5691c7", "name": "Huntington", "place_type": "city", "bounding_box": rectangle("-85.544347,40.85283 -85.451869,40.916848") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18069, "countyName": "Huntington", "cityID": 1835302, "cityName": "Huntington" } }
+{ "create_at": datetime("2016-01-02T10:00:02.000Z"), "id": 683347031629402113, "text": "@kpinzvero I use a beauty blender for everything except the liquid illuminator! What powdered highlight do you use?", "in_reply_to_status": 683346833196859393, "in_reply_to_user": 497603672, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 497603672 }}, "user": { "id": 3332567233, "name": "Ms Threestacks", "screen_name": "fl0ralpantiess", "lang": "en", "location": "3000", "create_at": date("2015-08-24"), "description": "||the dream is free the hustles sold separately ||this is my personal Twitter for art and shit @l0gofff", "followers_count": 828, "friends_count": 539, "statues_count": 1226 }, "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-01-02T10:00:03.000Z"), "id": 683347031788654592, "text": "@mhonividente #preguntaleaMhoni Mhoni como me ira en el trabajo? 11/29/1977", "in_reply_to_status": -1, "in_reply_to_user": 59094592, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "preguntaleaMhoni" }}, "user_mentions": {{ 59094592 }}, "user": { "id": 50579054, "name": "NANCY SALAS", "screen_name": "NANCEE1729", "lang": "en", "location": "null", "create_at": date("2009-06-25"), "description": "null", "followers_count": 4, "friends_count": 133, "statues_count": 24 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-02T10:00:03.000Z"), "id": 683347031851597824, "text": "I have a good hour to do so", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 497512937, "name": "MOANica.", "screen_name": "xvoxv_", "lang": "en", "location": "null", "create_at": date("2012-02-19"), "description": "hannah & pizza, pizza & hannah.", "followers_count": 1774, "friends_count": 542, "statues_count": 245025 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-02T10:00:03.000Z"), "id": 683347031864115200, "text": "I'll only pull a trigger legally. I aint getting in trouble for no petty shit foh.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2198321977, "name": "✈", "screen_name": "JuanGotti_", "lang": "en", "location": "Get Rich or Die Trying", "create_at": date("2013-11-16"), "description": "null", "followers_count": 170, "friends_count": 134, "statues_count": 9470 }, "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-01-02T10:00:03.000Z"), "id": 683347032560562177, "text": "���������� https://t.co/ZpbmwFkfLq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 392335899, "name": "Geoffrey", "screen_name": "iam_geoff", "lang": "en", "location": "Dirty Jersey", "create_at": date("2011-10-16"), "description": "Add me on Snapchat: iam_geoff", "followers_count": 6835, "friends_count": 915, "statues_count": 16120 }, "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-02T10:00:03.000Z"), "id": 683347033084833795, "text": "Wind 5.8 mph W. Barometer 30.21 in, Falling. Temperature 45.7 °F. Rain today 0.00 in. Humidity 41%", "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": 305897 }, "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-02T10:00:03.000Z"), "id": 683347033504153600, "text": "U only got Yoself the rest don't give af 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": 3191362176, "name": "savage", "screen_name": "_babytank3", "lang": "en", "location": "SomeWhereShootingInnaGym!", "create_at": date("2015-05-10"), "description": "null", "followers_count": 566, "friends_count": 353, "statues_count": 9548 }, "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-02T10:00:03.000Z"), "id": 683347034431160324, "text": "Sooooo tired", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 912980780, "name": "Josh Robertson", "screen_name": "itsyaboyj_rob", "lang": "en", "location": "Elyria, OH", "create_at": date("2012-10-29"), "description": "United States Army", "followers_count": 723, "friends_count": 324, "statues_count": 35682 }, "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-01-02T10:00:03.000Z"), "id": 683347034460438528, "text": "Want to work at Levi Strauss & Co.? We're #hiring in #Honolulu, HI! Click for details: https://t.co/wiVhbxAIah #Retail #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8502967,21.2918645"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Honolulu", "Retail", "Job", "Jobs" }}, "user": { "id": 28625039, "name": "TMJ-HON Retail Jobs", "screen_name": "tmj_hon_retail", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 373, "friends_count": 311, "statues_count": 448 }, "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-02T10:00:03.000Z"), "id": 683347034514980864, "text": "MY BOY JACK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 455978424, "name": "Mike", "screen_name": "MichaelSanchize", "lang": "en", "location": "Miami, FL", "create_at": date("2012-01-05"), "description": "UF Alum. Law School TBA. Miami Hurricane. Miami Dolphin. #Heat #Marlins #FlaPanthers #Everton. I'm just 'bout that action,boss. I tweet a lot about Alex Morgan", "followers_count": 1007, "friends_count": 964, "statues_count": 22594 }, "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-02T10:00:03.000Z"), "id": 683347034523320320, "text": "Want to work in #Honolulu, HI? View our latest opening: https://t.co/HMFxeQwWkd #Healthcare #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8480364,21.3136151"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Honolulu", "Healthcare", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 23526678, "name": "TMJ- HON Health Jobs", "screen_name": "tmj_hon_health", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Healthcare job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 409, "friends_count": 304, "statues_count": 189 }, "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-02T10:00:03.000Z"), "id": 683347034619777024, "text": "Happy birthday @Myles51laven hope it's grand!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 381573527 }}, "user": { "id": 2150136318, "name": "Hansen Vermie", "screen_name": "Hansencub", "lang": "en", "location": "null", "create_at": date("2013-10-22"), "description": "My friends and I are just a bunch of goodass kids doing badass stuff. Owner of Tropical Sno in Bondurant. Brooklynn Hickle ♡", "followers_count": 400, "friends_count": 270, "statues_count": 4915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bondurant, IA", "id": "0000ae7240a1c214", "name": "Bondurant", "place_type": "city", "bounding_box": rectangle("-93.48443,41.662757 -93.443564,41.710201") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1907390, "cityName": "Bondurant" } }
+{ "create_at": datetime("2016-01-02T10:00:03.000Z"), "id": 683347034686935041, "text": "@Samoziv @gus_vandenberg #Suck his dick man!", "in_reply_to_status": 683326153172606980, "in_reply_to_user": 2364412586, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Suck" }}, "user_mentions": {{ 2364412586, 135240509 }}, "user": { "id": 46353503, "name": "jerry smith", "screen_name": "highland07", "lang": "en", "location": "null", "create_at": date("2009-06-11"), "description": "null", "followers_count": 726, "friends_count": 69, "statues_count": 3865 }, "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-02T10:00:03.000Z"), "id": 683347035030970368, "text": "Need to get my nails done sap ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 348153701, "name": "❤sam", "screen_name": "samwhickerr", "lang": "en", "location": "Hamilton, OH", "create_at": date("2011-08-03"), "description": "#Khamiahsworld #08032014 ❤ My daughter is better than yours ;)", "followers_count": 1944, "friends_count": 1227, "statues_count": 58160 }, "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-01-02T10:00:03.000Z"), "id": 683347035286732800, "text": "@BlessIsrael1948 \n\"Those who take delight in mocking\n& rejecting Him, He will condemn\nconfuse their thinking\n& cause them to believe fables\"", "in_reply_to_status": 683346758320033792, "in_reply_to_user": 1613460655, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1613460655 }}, "user": { "id": 1613460655, "name": "Bless Israel", "screen_name": "BlessIsrael1948", "lang": "en", "location": "null", "create_at": date("2013-07-22"), "description": "Gentile Christian who prays for the Jews and Israel", "followers_count": 4557, "friends_count": 4592, "statues_count": 32463 }, "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-02T10:00:03.000Z"), "id": 683347035706142721, "text": "Try the grey stuff, it's delicious! @ Be Our Guest Restaurant Inside the Beast's Castle https://t.co/4wbKmNlS5w", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.58071345,28.42100213"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15259908, "name": "Leoren Violan", "screen_name": "lggviolan", "lang": "en", "location": "Philippines", "create_at": date("2008-06-27"), "description": "Hey, look! I can officially say I'm a starving artist.\nLife. Art. Gibberish. Complaints. Shit like that.", "followers_count": 367, "friends_count": 165, "statues_count": 12682 }, "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-01-02T10:00:03.000Z"), "id": 683347035802578945, "text": "https://t.co/Bi0MKxTGBp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 599979316, "name": "Troy Geary", "screen_name": "TroyGeary", "lang": "en", "location": "Portland, OR", "create_at": date("2012-06-05"), "description": "Techno", "followers_count": 531, "friends_count": 522, "statues_count": 101272 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Hope, MN", "id": "76462ec13e8f0150", "name": "New Hope", "place_type": "city", "bounding_box": rectangle("-93.40099,45.007312 -93.370138,45.065743") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2745628, "cityName": "New Hope" } }
+{ "create_at": datetime("2016-01-02T10:00:04.000Z"), "id": 683347036112945152, "text": "Today's lesson was, \"Always be prepared (for #rain when #cycling in #Seattle),\" shown by positive example! #commute", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.33207,47.60621"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "rain", "cycling", "Seattle", "commute" }}, "user": { "id": 916596266, "name": "竹下E本", "screen_name": "takenji_ebooks", "lang": "en", "location": "The Past", "create_at": date("2012-10-31"), "description": "Eh, close enough.", "followers_count": 145, "friends_count": 1, "statues_count": 99217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "f42a863798156617", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.459696,47.491912 -122.224433,47.734145") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-01-02T10:00:04.000Z"), "id": 683347036112949248, "text": "Al-Qaeda affiliate uses Trump's anti-Muslim statements in recruitment video https://t.co/ohZVcz3PbU", "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": 17704, "friends_count": 17579, "statues_count": 66526 }, "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-02T10:00:04.000Z"), "id": 683347036456927232, "text": "Sunny this afternoon, high 44 (7 C). Low 23 (-5 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15953346, "name": "Amarillo Weather", "screen_name": "AmarilloWeather", "lang": "en", "location": "Amarillo, Texas", "create_at": date("2008-08-22"), "description": "Forecasts and advisories for Amarillo, Texas", "followers_count": 4041, "friends_count": 395, "statues_count": 8702 }, "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-01-02T10:00:04.000Z"), "id": 683347036666740736, "text": "What's the move man ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 539787273, "name": "realYASnigga❤️", "screen_name": "fornicateUrLife", "lang": "en", "location": " Cityy, NearestYou", "create_at": date("2012-03-28"), "description": "snapchat: yas.minee", "followers_count": 800, "friends_count": 706, "statues_count": 16064 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sylacauga, AL", "id": "0eafc6baaccf76d2", "name": "Sylacauga", "place_type": "city", "bounding_box": rectangle("-86.298149,33.14411 -86.201764,33.225738") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1121, "countyName": "Talladega", "cityID": 174352, "cityName": "Sylacauga" } }
+{ "create_at": datetime("2016-01-02T10:00:04.000Z"), "id": 683347036666761216, "text": "Chilly moe how we play it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 625912893, "name": "#️⃣OsoStrapped", "screen_name": "BN4L_dblack", "lang": "en", "location": "#️⃣RedZone", "create_at": date("2012-07-03"), "description": "Gotta win can't lose do it for my nigga Gabe #RipGabeD #FreeJames #FreeWayne #FreeDevi it ain't nothing but ah set back", "followers_count": 2052, "friends_count": 462, "statues_count": 50548 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beecher, MI", "id": "8d5dfd9dd04ba8a7", "name": "Beecher", "place_type": "city", "bounding_box": rectangle("-83.763336,43.075092 -83.674217,43.133995") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2606820, "cityName": "Beecher" } }
+{ "create_at": datetime("2016-01-02T10:00:04.000Z"), "id": 683347036947771392, "text": "I always come to work looking so bummy on Saturdays", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2571223301, "name": "KRISTIN", "screen_name": "_krisssyboo_", "lang": "en", "location": "null", "create_at": date("2014-05-29"), "description": "20 | IG:_krisssyboo_", "followers_count": 188, "friends_count": 162, "statues_count": 4297 }, "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-02T10:00:04.000Z"), "id": 683347037048274944, "text": "#Pozole is what you need.\nPhoto credit: https://t.co/clhUQnMWVV https://t.co/wFpGoeo91I", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Pozole" }}, "user": { "id": 1659913500, "name": "Aldacos at Dominion", "screen_name": "AldacosDominion", "lang": "en", "location": "22211 IH 10 West 78257 ", "create_at": date("2013-08-10"), "description": "Serving Zesty Food Since 1989\nMargarita Aficionado\nPioneer of the Tres Leches and Avocado Margarita \nBlanca Aldaco, The Hostess With The Mostess.", "followers_count": 429, "friends_count": 734, "statues_count": 2538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hill Country Village, TX", "id": "bd2885b447fe044f", "name": "Hill Country Village", "place_type": "city", "bounding_box": rectangle("-98.504515,29.566201 -98.474846,29.597081") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4833968, "cityName": "Hill Country Village" } }
+{ "create_at": datetime("2016-01-02T10:00:04.000Z"), "id": 683347037404925952, "text": "2nd Amendment not withstanding, some idiots should not be allowed to have guns.\n\nhttps://t.co/05BreYXqe8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14099014, "name": "Charles Stepp", "screen_name": "steppnav", "lang": "en", "location": "work", "create_at": date("2008-03-07"), "description": "Christian, 5 daughters, Unix Programming, Oracle, IPod Hater", "followers_count": 25, "friends_count": 30, "statues_count": 719 }, "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-02T10:00:04.000Z"), "id": 683347037476147200, "text": "We're #hiring! Read about our latest #job opening here: Dessert Finisher/Espresso Maker - https://t.co/4Maq3ZQEn7 #Honolulu, HI #Veterans", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8281055,21.2781434"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Honolulu", "Veterans" }}, "user": { "id": 23897767, "name": "TMJ-HON Hosp. Jobs", "screen_name": "tmj_hon_hrta", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-11"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 411, "friends_count": 292, "statues_count": 265 }, "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-02T10:00:04.000Z"), "id": 683347037509689347, "text": "CVS Health #Labor #Job: Truck Driver – CDL Class A or B (#KAPOLEI, HI) https://t.co/1DIrG17wfX #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-158.056892,21.3354034"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Labor", "Job", "KAPOLEI", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 118676555, "name": "TMJ-HI Labor Jobs", "screen_name": "tmj_HI_LABOR", "lang": "en", "location": "Hawaii", "create_at": date("2010-03-01"), "description": "Follow this account for geo-targeted General Labor job tweets in Hawaii Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 283, "friends_count": 279, "statues_count": 4 }, "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-02T10:00:04.000Z"), "id": 683347037576888321, "text": "Igs I'll go to the jcm vs south side game", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 605883242, "name": "$DM BabyBoy", "screen_name": "ThtGuyyBabyboy", "lang": "en", "location": "null", "create_at": date("2012-06-11"), "description": "Just a younging from a small city tryna make sum outta nothing I'm omw tho with the grace of God", "followers_count": 1505, "friends_count": 1344, "statues_count": 25720 }, "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-01-02T10:00:04.000Z"), "id": 683347038386388992, "text": "I'm at Hornell, NY in Hornell, NY https://t.co/rGJgqjA1hc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.65995026,42.32707774"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2373246818, "name": "Vince", "screen_name": "blamethebeer", "lang": "en", "location": "Hornell, NY", "create_at": date("2014-03-04"), "description": "Beer Connoisseur / FedEx Courier", "followers_count": 83, "friends_count": 243, "statues_count": 2249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hornell, NY", "id": "73464be9a3ee4bbf", "name": "Hornell", "place_type": "city", "bounding_box": rectangle("-77.685695,42.299074 -77.644926,42.350739") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36101, "countyName": "Steuben", "cityID": 3635672, "cityName": "Hornell" } }
+{ "create_at": datetime("2016-01-02T10:00:04.000Z"), "id": 683347038759555073, "text": "I'm single for life bruh https://t.co/l3EErL1HAg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 215872267, "name": "Tyrone.", "screen_name": "DetroitMade94", "lang": "en", "location": "Detroit, MI - Dallas, Tx", "create_at": date("2010-11-14"), "description": "Why accept failure when success is free #GodFirst John 3:16 RIP Grandad", "followers_count": 2151, "friends_count": 1216, "statues_count": 121143 }, "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-01-02T10:00:04.000Z"), "id": 683347038981853184, "text": "Partly cloudy this afternoon, high 41 (5 C). Low 22 (-6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712333, "name": "LubbockWeather", "screen_name": "LubbockWeather", "lang": "en", "location": "Lubbock, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Lubbock, Texas", "followers_count": 1187, "friends_count": 93, "statues_count": 8035 }, "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-02T10:00:04.000Z"), "id": 683347039028002816, "text": "temperature up 7°F -> 9°F\nhumidity up 74% -> 80%\npressure 30.44in rising\nvisibility 1mi -> 2mi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-116.69259,43.66744"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 240054225, "name": "Caldwell Weather", "screen_name": "CaldwellID", "lang": "en", "location": "Caldwell, ID", "create_at": date("2011-01-18"), "description": "Weather updates, forecast, warnings and information for Caldwell, ID. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 11, "friends_count": 3, "statues_count": 19891 }, "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-02T10:00:04.000Z"), "id": 683347039124508672, "text": "American National Insurance Agent (#Lexington, KY) https://t.co/ZD2folL4Jb #Sales #Job #Jobs #Hiring #CareerArc", "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": {{ "Lexington", "Sales", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 82637881, "name": "TMJ-KYL Sales Jobs", "screen_name": "tmj_kyl_sales", "lang": "en", "location": "Lexington, KY", "create_at": date("2009-10-15"), "description": "Follow this account for geo-targeted Sales job tweets in Lexington, KY. Need help? Tweet us at @CareerArc!", "followers_count": 329, "friends_count": 294, "statues_count": 93 }, "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-02T10:00:04.000Z"), "id": 683347039359352832, "text": "#Kahului, HI #Job: Sales Consultant Print Services at OfficeMax https://t.co/XdxKI6gj8Q #Veterans #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-156.454708,20.881571"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "Kahului", "Job", "Veterans", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 38082277, "name": "TMJ-MAU Jobs", "screen_name": "tmj_mau_jobs", "lang": "en", "location": "Maui, HI", "create_at": date("2009-05-05"), "description": "Follow this account for geo-targeted Other job tweets in Maui, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 547, "friends_count": 535, "statues_count": 16 }, "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-02T10:00:04.000Z"), "id": 683347039573413888, "text": "@BTS_twt https://t.co/Z7nce2KT8d", "in_reply_to_status": 683300258554642433, "in_reply_to_user": 335141638, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 335141638 }}, "user": { "id": 3384126261, "name": "GetWellSoonRM❤️", "screen_name": "jungcock1997", "lang": "en", "location": "Kris's Weave", "create_at": date("2015-07-19"), "description": "#Getwellsoonnamjoon! #HandsomeRM all you haters need to evaporate @ Namjoon JIMIN GOT YES JAMS #Getwellsoontaegi", "followers_count": 58, "friends_count": 118, "statues_count": 1791 }, "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-02T10:00:04.000Z"), "id": 683347039854313472, "text": "Never really made love but you feel love", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3146094195, "name": "Tieric Tommy G.", "screen_name": "itstbible", "lang": "en", "location": "null", "create_at": date("2015-04-07"), "description": "TheCircle™ | Artist | Snapchat http://t.bible | TexasState | Trainer in training, DM if interested in working out", "followers_count": 274, "friends_count": 155, "statues_count": 2340 }, "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.069786,31.639828") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4876000, "cityName": "Waco" } }
+{ "create_at": datetime("2016-01-02T10:00:04.000Z"), "id": 683347040009506816, "text": "just received 1st fragrance compliment ever from my son! #SOTD is #Hermes Eau des Merveilles #smellgood", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SOTD", "Hermes", "smellgood" }}, "user": { "id": 49597314, "name": "shannon thompson", "screen_name": "shadesofbleu", "lang": "en", "location": "wichita, kansas", "create_at": date("2009-06-22"), "description": "just a girl who loves Jayhawk sports, perfume, music, cruising, and my family", "followers_count": 59, "friends_count": 154, "statues_count": 685 }, "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-01-02T10:00:05.000Z"), "id": 683347040156434432, "text": "Martin Garrix // Bouncybob", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 979163916, "name": "Jordan", "screen_name": "Jordan_Warner30", "lang": "en", "location": "null", "create_at": date("2012-11-29"), "description": "Fort Osage State Champions #30", "followers_count": 676, "friends_count": 567, "statues_count": 17777 }, "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-01-02T10:00:05.000Z"), "id": 683347040315797504, "text": "Dabbin on yo Hoe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2227559154, "name": "Thaaa Kid Wank", "screen_name": "BitchesloveEli", "lang": "en", "location": "null", "create_at": date("2013-12-02"), "description": "tryna make 〽️'S | 17 years young| Wilson NC|", "followers_count": 370, "friends_count": 208, "statues_count": 3841 }, "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-01-02T10:00:05.000Z"), "id": 683347040454184960, "text": "also people need to understand hes an actor which means he could be completely different than the characters he has played.", "in_reply_to_status": 683346729945677825, "in_reply_to_user": 419810713, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 419810713, "name": "nancy", "screen_name": "ughnancy", "lang": "en", "location": "null", "create_at": date("2011-11-23"), "description": "null", "followers_count": 609, "friends_count": 415, "statues_count": 50508 }, "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-02T10:00:05.000Z"), "id": 683347040516964353, "text": "where's my sis that takes photos? and why you ain't got none of me ??@imanidwells", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4014697034 }}, "user": { "id": 2493687410, "name": "brownsuga", "screen_name": "astrydxo", "lang": "en", "location": "the gutter ", "create_at": date("2014-05-13"), "description": "cute lil bitch w/ a whole lotta heart", "followers_count": 349, "friends_count": 537, "statues_count": 15638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-02T10:00:05.000Z"), "id": 683347040701648900, "text": "@granthpaulsen Hey man at Mighty Khans checking it out.", "in_reply_to_status": -1, "in_reply_to_user": 24391041, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24391041 }}, "user": { "id": 447556696, "name": "Craig Mills", "screen_name": "1craigmills", "lang": "en", "location": "Virginia, USA", "create_at": date("2011-12-26"), "description": "A die hard UVA Football fan and Redskin fan Phil4:13, Psalm 27. Living off the beaten path, flying down a back road.", "followers_count": 106, "friends_count": 330, "statues_count": 2992 }, "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-01-02T10:00:05.000Z"), "id": 683347040752013314, "text": "Been on a roll with the beats lately. Raps always on point. New music coming soon ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 180056775, "name": "B", "screen_name": "95blaze_", "lang": "en", "location": "Basement ", "create_at": date("2010-08-18"), "description": "Music, and Mary (Occasionally) are my only drugs. Wanna talk business? Itsblaze95@gmail.com", "followers_count": 712, "friends_count": 737, "statues_count": 22168 }, "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.200639,42.517441") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2674900, "cityName": "Southfield" } }
+{ "create_at": datetime("2016-01-02T10:00:05.000Z"), "id": 683347040823201792, "text": "current weather in Madison: fair, 25°F\n66% humidity, wind 5mph, visibility 10mi, pressure 30.02in falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.38669,43.07295"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 141440102, "name": "Madison Weather", "screen_name": "_MadisonWI", "lang": "en", "location": "Madison, WI", "create_at": date("2010-05-07"), "description": "Weather updates, forecast, warnings and information for Madison, WI. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 145, "friends_count": 1, "statues_count": 23587 }, "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.491889 -86.24955,47.309715") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55025, "countyName": "Dane", "cityID": 5548000, "cityName": "Madison" } }
+{ "create_at": datetime("2016-01-02T10:00:05.000Z"), "id": 683347040982700032, "text": "@50shadesofchocc don't nobody make yall crazy��just like that nigga don't care you don't have to care", "in_reply_to_status": 683346658260811778, "in_reply_to_user": 568407371, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 568407371 }}, "user": { "id": 347797950, "name": "dariusB", "screen_name": "_saucydB", "lang": "en", "location": "Baltimore, MD", "create_at": date("2011-08-03"), "description": "dcked low bout my beezwax 25/8 #noLssss #2sc City16'...leachyB✨", "followers_count": 2160, "friends_count": 656, "statues_count": 29855 }, "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-02T10:00:05.000Z"), "id": 683347041116790784, "text": "Free bball clinic put on by @AndoverYMCA. Emmy had a lot of fun! https://t.co/Y3v2ofYt5O", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1511092393 }}, "user": { "id": 508891562, "name": "JenHinker", "screen_name": "JenniferHinker", "lang": "en", "location": "null", "create_at": date("2012-02-29"), "description": "null", "followers_count": 30, "friends_count": 30, "statues_count": 84 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Andover, MN", "id": "219ccc645d458eee", "name": "Andover", "place_type": "city", "bounding_box": rectangle("-93.40681,45.21178 -93.265395,45.304142") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27003, "countyName": "Anoka", "cityID": 2701486, "cityName": "Andover" } }
+{ "create_at": datetime("2016-01-02T10:00:05.000Z"), "id": 683347041129373696, "text": "���� https://t.co/TGNpud1z97", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 160440178, "name": "Smoov", "screen_name": "NewNameTBA", "lang": "en", "location": "null", "create_at": date("2010-06-27"), "description": "KosDos Smoov", "followers_count": 1402, "friends_count": 798, "statues_count": 126774 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-02T10:00:05.000Z"), "id": 683347041158836225, "text": "ALEX HAD THE VOICE I THINK I LIKED N OITNB. SAW THE FIRST SEASON N STOPPED WHEN ALEX GOT THE APT. 2ND SEASON?", "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. King, Of Tennessee. Married. 25. BLACKBILLY. I'm Silly. SloppyTweeter. Kid Komedian: IN PROGRESS. #BOSSING.", "followers_count": 36, "friends_count": 74, "statues_count": 32012 }, "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-02T10:00:05.000Z"), "id": 683347041498435584, "text": "Mostly sunny this afternoon, high 48 (9 C). Low 31 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712562, "name": "ElPasoWX", "screen_name": "ElPasoWX", "lang": "en", "location": "El Paso, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for El Paso, Texas", "followers_count": 526, "friends_count": 93, "statues_count": 8093 }, "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-02T10:00:05.000Z"), "id": 683347041762832385, "text": "@Deena_diamandis @kammllinn is he like done yet or", "in_reply_to_status": 683346902595825664, "in_reply_to_user": 2364989576, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2364989576, 418157605 }}, "user": { "id": 382272976, "name": "syd sek", "screen_name": "syddsek", "lang": "en", "location": "Canfield, OH", "create_at": date("2011-09-29"), "description": "srhs", "followers_count": 1552, "friends_count": 717, "statues_count": 33166 }, "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-02T10:00:05.000Z"), "id": 683347042161299456, "text": "I can't wait until he 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": 183412397, "name": "Larry Osburn", "screen_name": "StarringLarry", "lang": "en", "location": "Target", "create_at": date("2010-08-26"), "description": "19 years old!", "followers_count": 1185, "friends_count": 635, "statues_count": 44871 }, "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-02T10:00:05.000Z"), "id": 683347042190503936, "text": "Protection 1 #Utilities #Job: Residential Installer I (#Cypress, CA) https://t.co/nvwr2sDfSF #Jobs #Hiring https://t.co/hjeXqLT4Io", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.0315051,33.8079876"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Utilities", "Job", "Cypress", "Jobs", "Hiring" }}, "user": { "id": 2194610792, "name": "Protection 1 Careers", "screen_name": "Pro1Careers", "lang": "en", "location": "null", "create_at": date("2013-11-14"), "description": "All of our open positions will be tweeted here! Let's chat about how you can be part of the P1 Family. You can also follow us at @ProtectionOne", "followers_count": 144, "friends_count": 86, "statues_count": 314 }, "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-02T10:00:05.000Z"), "id": 683347042744287232, "text": "\"RT @_KJCRAY Could you date a bad texter?\"\n\n@MHoopz, you were saying? https://t.co/fEx4lPiBtz", "in_reply_to_status": 683339615646937089, "in_reply_to_user": 224383218, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 224383218, 28147241 }}, "user": { "id": 224354592, "name": "Rich Homo Quan", "screen_name": "XIXMCMLXXXV", "lang": "en", "location": "Angel Grove", "create_at": date("2010-12-08"), "description": "Welcome...AND RAQUEL TAKE THAT FUCKING JACKET OFF!!", "followers_count": 2144, "friends_count": 608, "statues_count": 63650 }, "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-02T10:00:05.000Z"), "id": 683347043356663814, "text": "@DaddyD___ fakin*", "in_reply_to_status": 674866848693805056, "in_reply_to_user": 3406581287, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3406581287 }}, "user": { "id": 421550601, "name": "Glorified Dumbass", "screen_name": "Son_OfA_Burch", "lang": "en", "location": "Maryland, USA", "create_at": date("2011-11-25"), "description": "you gotta be a dog", "followers_count": 493, "friends_count": 370, "statues_count": 15582 }, "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-02T10:00:05.000Z"), "id": 683347043444592640, "text": "Want to work at CVS Health? We're #hiring in #Honolulu, HI! Click for details: https://t.co/eYdTlftGOT #Marketing #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Honolulu", "Marketing", "Job", "Jobs", "CareerArc" }}, "user": { "id": 23493115, "name": "TMJ-HON Advert Jobs", "screen_name": "tmj_hon_adv", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Marketing/Ad/PR job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 290, "friends_count": 193, "statues_count": 9 }, "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-02T10:00:05.000Z"), "id": 683347043667062786, "text": "big brother really got that pressure", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2997551977, "name": "yeaa it's me :))", "screen_name": "theybeonher", "lang": "en", "location": "null", "create_at": date("2015-01-26"), "description": "one life , two chances", "followers_count": 1066, "friends_count": 709, "statues_count": 33765 }, "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-02T10:00:05.000Z"), "id": 683347043775942659, "text": "These hoes are for everyone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1640676416, "name": "Ben Kanobi Khaled", "screen_name": "JManoes", "lang": "en", "location": "null", "create_at": date("2013-08-02"), "description": "null", "followers_count": 508, "friends_count": 637, "statues_count": 20467 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Keansburg, NJ", "id": "70f8f5f2ad7e0722", "name": "Keansburg", "place_type": "city", "bounding_box": rectangle("-74.147592,40.435309 -74.11593,40.459534") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3436480, "cityName": "Keansburg" } }
+{ "create_at": datetime("2016-01-02T10:00:05.000Z"), "id": 683347044048568321, "text": "Positivity: Victoria is going to visit me! I'm earning money for NYC! Going out tonight! Spice Girls Pandora station!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14655475, "name": "Ewok Queen", "screen_name": "tspofnutmeg", "lang": "en", "location": "Seattle, WA", "create_at": date("2008-05-04"), "description": "Co-Founder of @HaveYouNerd, Co-Host at @FarFarAwayRadio, Rogue Leader for @RoguePodron. Proprietor of Force Apparel, Ewok Enthusiast.", "followers_count": 1100, "friends_count": 621, "statues_count": 18467 }, "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-02T10:00:05.000Z"), "id": 683347044228923392, "text": "Cloudy this afternoon, high 47 (8 C). Low 31 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712512, "name": "AbileneWeather", "screen_name": "AbileneWeather", "lang": "en", "location": "Abilene, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Abilene, Texas", "followers_count": 932, "friends_count": 93, "statues_count": 8133 }, "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-02T10:00:06.000Z"), "id": 683347044480741377, "text": "@TATJANASL Not one of Eisenstaedt's best. Must have used an older iPhone.", "in_reply_to_status": 683329146831958016, "in_reply_to_user": 1974775610, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1974775610 }}, "user": { "id": 18544606, "name": "Jay Schiavone", "screen_name": "jaytingle", "lang": "en", "location": "New Haven", "create_at": date("2009-01-01"), "description": "I am a union employee at a local phone company. I live with my wife and 3 dogs on a steep hill in a state forest. 'Love me, I'm a liberal.'", "followers_count": 315, "friends_count": 272, "statues_count": 17781 }, "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-01-02T10:00:06.000Z"), "id": 683347044480757761, "text": "Texas Tech vs Texas #GunsUp #HornsDown #BAM #wreckem (@ TTU - United Supermarkets Arena - @texastech in Lubbock, TX) https://t.co/htZP1NKeNY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-101.88683823,33.5813739"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GunsUp", "HornsDown", "BAM", "wreckem" }}, "user_mentions": {{ 14125882 }}, "user": { "id": 42051973, "name": "Joey Melton", "screen_name": "JoeyTTU01", "lang": "en", "location": "ÜT: 33.544554,-101.934278", "create_at": date("2009-05-23"), "description": "Texas Tech Red Raider Class of 2009", "followers_count": 186, "friends_count": 318, "statues_count": 7763 }, "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-02T10:00:06.000Z"), "id": 683347044610658305, "text": "@ColtenTyler3 this has to be a joke", "in_reply_to_status": 683334056763863040, "in_reply_to_user": 1869037224, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1869037224 }}, "user": { "id": 263944033, "name": "Pooh Bear", "screen_name": "Hyperphil25", "lang": "en", "location": "Italy✈️Corpus Christi, TX", "create_at": date("2011-03-10"), "description": "#TAMUCC| Ex Collegiate Hooper| EPTX| 337|", "followers_count": 1769, "friends_count": 1093, "statues_count": 38001 }, "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-02T10:00:06.000Z"), "id": 683347044715597824, "text": "@charles123_d lol..", "in_reply_to_status": 683345815994253312, "in_reply_to_user": 2937139623, "favorite_count": 0, "coordinate": point("-70.6869768,41.9535379"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2937139623 }}, "user": { "id": 110579152, "name": "Steve-O", "screen_name": "SnowballManny", "lang": "en", "location": "null", "create_at": date("2010-02-01"), "description": "null", "followers_count": 354, "friends_count": 909, "statues_count": 16665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plymouth, MA", "id": "01bbfc630e681dda", "name": "Plymouth", "place_type": "city", "bounding_box": rectangle("-70.758703,41.789998 -70.525503,41.972125") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25023, "countyName": "Plymouth", "cityID": 2554275, "cityName": "Plymouth" } }
+{ "create_at": datetime("2016-01-02T10:00:06.000Z"), "id": 683347044728094720, "text": "#JetsFollowJets", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "JetsFollowJets" }}, "user": { "id": 34311766, "name": "HARGZ™", "screen_name": "JohnHargz", "lang": "en", "location": "El Paso, TX ", "create_at": date("2009-04-22"), "description": "Staff Writer at @TurnOnTheJets #TOJ LI to UT to TX #Jets #Mets #Isles #Knicks @meghancuozzo 8-27-2013", "followers_count": 1342, "friends_count": 1176, "statues_count": 103296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lomas del Rey, El Paso", "id": "0ea548b3b422adc4", "name": "Lomas del Rey", "place_type": "neighborhood", "bounding_box": rectangle("-106.511932,31.805131 -106.499735,31.813507") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-01-02T10:00:06.000Z"), "id": 683347044895830018, "text": "This #SkilledTrade #job might be a great fit for you: Electrician - https://t.co/uJFDvFH9JJ #electrician #Gillette, WY #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.5022205,44.2910915"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SkilledTrade", "job", "electrician", "Gillette", "Hiring" }}, "user": { "id": 3011572896, "name": "Cloud Peak Jobs", "screen_name": "CloudPeakJobs", "lang": "en", "location": "null", "create_at": date("2015-02-06"), "description": "Cloud Peak Energy aims to be a leading energy producer operating in a safe, responsible and caring manner. Check here for current job openings.", "followers_count": 57, "friends_count": 270, "statues_count": 15 }, "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-01-02T10:00:06.000Z"), "id": 683347045000716288, "text": "When your whole family is in one car for a day of returns and shopping ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 545113498, "name": "Summer Templeton", "screen_name": "summmerjean", "lang": "en", "location": "Shawnee, OK", "create_at": date("2012-04-04"), "description": "Dallas // Oklahoma Baptist // kattmillionz // ΠΣΦ", "followers_count": 530, "friends_count": 518, "statues_count": 6558 }, "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-02T10:00:06.000Z"), "id": 683347045118308358, "text": "@Zach_Dorsey what colors will the hoodies be? I'm hoping for a black one ��", "in_reply_to_status": -1, "in_reply_to_user": 251796499, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 251796499 }}, "user": { "id": 799176576, "name": "Stephanie ♡", "screen_name": "AMsBigClark", "lang": "en", "location": "New York, NY", "create_at": date("2012-09-02"), "description": "| | 4/4 | | 12.09.14 | 08.11.15 | 08.16.15 | 11.11.15 | 11.12.15 | @AustinMahone everyday", "followers_count": 1889, "friends_count": 988, "statues_count": 20637 }, "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-02T10:00:06.000Z"), "id": 683347045243990017, "text": "#Nursing in #Soldotna, AK: RN - Preoperative Clinic at Central Peninsula Hospital https://t.co/OBXrTx7wLa #Alaskan https://t.co/t2ZPheXzmR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-151.0583332,60.4877778"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "hashtags": {{ "Nursing", "Soldotna", "Alaskan" }}, "user": { "id": 3278999924, "name": "CPH Jobs", "screen_name": "CPHJobs", "lang": "en", "location": "Soldotna, AK", "create_at": date("2015-07-13"), "description": "At CPH every employee is considered a caregiver, an integral part of our health care team. Follow us to learn more!", "followers_count": 24, "friends_count": 77, "statues_count": 49 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Soldotna, AK", "id": "350518f7c1ac93ff", "name": "Soldotna", "place_type": "city", "bounding_box": rectangle("-151.193024,60.458405 -151.048563,60.511008") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2122, "countyName": "Kenai Peninsula", "cityID": 271640, "cityName": "Soldotna" } }
+{ "create_at": datetime("2016-01-02T10:00:06.000Z"), "id": 683347045298663428, "text": "A chilly day at the park but still good to be outside. @ J.J. Biello Park https://t.co/KoVtHv7kYV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.5031586,34.117321"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14189080, "name": "Chris Saxon", "screen_name": "cmsaxon", "lang": "en", "location": "Canton, GA", "create_at": date("2008-03-20"), "description": "Husband to Lauren, dad to 2 wonderful boys, K12 technology support. And yes, I still work during the summer.", "followers_count": 277, "friends_count": 490, "statues_count": 1210 }, "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": 13057, "countyName": "Cherokee", "cityID": 1384176, "cityName": "Woodstock" } }
+{ "create_at": datetime("2016-01-02T10:00:06.000Z"), "id": 683347045474766848, "text": "@rachel_puckett https://t.co/A0T1Ly5uZW", "in_reply_to_status": -1, "in_reply_to_user": 328158126, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 328158126 }}, "user": { "id": 156138555, "name": "Bruce Wayne", "screen_name": "JackDeLucia", "lang": "en", "location": "Gotham ", "create_at": date("2010-06-15"), "description": "Whatever you are, be a good one", "followers_count": 557, "friends_count": 504, "statues_count": 29504 }, "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-01-02T10:00:06.000Z"), "id": 683347045579526144, "text": "my dad tryna bake me bruh he set the heater hella hot last night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2768535327, "name": "hannah", "screen_name": "hannideleon", "lang": "en", "location": "null", "create_at": date("2014-09-12"), "description": "vs 5/22", "followers_count": 176, "friends_count": 127, "statues_count": 2899 }, "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-02T10:00:06.000Z"), "id": 683347045873266689, "text": "@SaraLePew @weeformer @markerguru I say you can do it, Sara! (And you too, Alex)", "in_reply_to_status": 683346059662364672, "in_reply_to_user": 207745322, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 207745322, 2173113589, 98736528 }}, "user": { "id": 581138892, "name": "Transfan2", "screen_name": "Transfan2", "lang": "en", "location": "Hamilton, Ohio", "create_at": date("2012-05-15"), "description": "Decepticons Forever / Many #BFF like @Evochanger @callykarishokka, @RoboMRB @dkayevo / Also runs Transfan2's Shop N' Look", "followers_count": 550, "friends_count": 473, "statues_count": 63111 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, OH", "id": "010dd00546dbccf2", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-84.630868,39.301567 -74.25475,40.897853") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39017, "countyName": "Butler", "cityID": 3925970, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-01-02T10:00:06.000Z"), "id": 683347046041038849, "text": "@HeatherHabura Thank you for the follow! #HappyNewYear #Happy2016 Have a wonderful new year!", "in_reply_to_status": -1, "in_reply_to_user": 916560306, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HappyNewYear", "Happy2016" }}, "user_mentions": {{ 916560306 }}, "user": { "id": 31521032, "name": "Chris Carfizzi", "screen_name": "ChrisCarfizzi", "lang": "en", "location": "New York, NY", "create_at": date("2009-04-15"), "description": "Actor, Writer, Producer, Creator of #WhitePrivilegeFrankenstein http://bit.ly/1QOCEGp\n\nhttp://www.imdb.com/name/nm3218129/?ref_=fn_al_nm_1", "followers_count": 1308, "friends_count": 1309, "statues_count": 2944 }, "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-02T10:00:06.000Z"), "id": 683347046120710145, "text": "@resyesther23 @nalia23 Bonne annee Esther# Resy!!!", "in_reply_to_status": 683271271518310404, "in_reply_to_user": 2220190208, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 2220190208, 419156720 }}, "user": { "id": 2586937911, "name": "jouvenel Pierre", "screen_name": "jouvipi76", "lang": "en", "location": "null", "create_at": date("2014-06-06"), "description": "null", "followers_count": 136, "friends_count": 388, "statues_count": 296 }, "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-02T10:00:06.000Z"), "id": 683347046435172352, "text": "I'm so done �������� https://t.co/DEhKsXIexH", "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": "JiggyJes", "screen_name": "jcano98", "lang": "en", "location": "right thurr hawmie", "create_at": date("2015-12-02"), "description": "She's mine, and only mine.", "followers_count": 146, "friends_count": 128, "statues_count": 175 }, "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-02T10:00:06.000Z"), "id": 683347046456135681, "text": "This #Healthcare #job might be a great fit for you: PT / PHYSICAL THERAPIST - https://t.co/iQaJufdSon #PT #Racine, WI #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.7828523,42.7261309"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "PT", "Racine", "Hiring" }}, "user": { "id": 2365967174, "name": "Aureus Medical Jobs", "screen_name": "aureusmedjobs", "lang": "en", "location": "Nationwide", "create_at": date("2014-02-28"), "description": "Follow @aureusmedjobs for #travelnursing, #traveltherapy, imaging, med lab, physicians, NP, and PA #jobs nationwide. Follow our company @aureusmedical.", "followers_count": 371, "friends_count": 1, "statues_count": 10192 }, "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-02T10:00:06.000Z"), "id": 683347046607253504, "text": "Im an animal today https://t.co/PSV2FgoHnC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1041031362, "name": "Kendall Telep", "screen_name": "canddleee", "lang": "en", "location": "Tatooine ", "create_at": date("2012-12-27"), "description": "daughter of Luke Skywalker", "followers_count": 804, "friends_count": 979, "statues_count": 9204 }, "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-01-02T10:00:06.000Z"), "id": 683347046754025473, "text": "It's awkward when you're looking at a dog in the neighboring car and then the owner makes eye contact with you while you're mid-\"��\" face", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 245871411, "name": "J▲stun", "screen_name": "JSTNBLND", "lang": "en", "location": "Bealeton, VA", "create_at": date("2011-02-01"), "description": "Justin, 22, Northern Virginia, college grad, funny man/artist/producer, take my keyboards away from me", "followers_count": 240, "friends_count": 1258, "statues_count": 22462 }, "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-02T10:00:06.000Z"), "id": 683347047085420552, "text": "@RickertAmber thanks girl ����", "in_reply_to_status": 683343175059034112, "in_reply_to_user": 3015881802, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3015881802 }}, "user": { "id": 781266584, "name": "Rachel Koop", "screen_name": "koopa_troopa10", "lang": "en", "location": "null", "create_at": date("2012-08-25"), "description": "UNI Volleyball #14", "followers_count": 804, "friends_count": 609, "statues_count": 1609 }, "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-01-02T10:00:06.000Z"), "id": 683347047265730560, "text": "I'm at Broward County in Deerfield Beach, FL w/ @jasonbrinberg @domgiangiobbe https://t.co/1xjEF56jzO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.10526382,26.31449931"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19244987, 293219687 }}, "user": { "id": 29517640, "name": "Natalia Czoch", "screen_name": "NataliaCzoch", "lang": "en", "location": "NYC", "create_at": date("2009-04-07"), "description": "Digital Media, Travel, Puppy, Seal and Foodie Snacks Lover!!!!!", "followers_count": 555, "friends_count": 2025, "statues_count": 1331 }, "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-02T10:00:06.000Z"), "id": 683347047282532352, "text": "Agree ���� https://t.co/VduJEQyLBF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20641569, "name": "Amy Sims", "screen_name": "Queenmom1977", "lang": "en", "location": "Columbus, Ohio", "create_at": date("2009-02-11"), "description": "I have one beautiful biracial daughter...If u unfollow me I'll unfollow u..If u would like to know more abt me don't b afraid to ask I don't bite! :)", "followers_count": 906, "friends_count": 2059, "statues_count": 1259 }, "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-02T10:00:06.000Z"), "id": 683347047341252609, "text": "The Wait Is Over!!! Check Out This #NewVideo By @RobStashiz Called \"WHO'S BAD\" ft. @NAAMPFE On #YouTube #RightNow https://t.co/XwWzS90cdO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NewVideo", "YouTube", "RightNow" }}, "user_mentions": {{ 69172425, 284663798 }}, "user": { "id": 481606532, "name": "DJ COZ ♌", "screen_name": "REEMTHEDREAM215", "lang": "en", "location": "PHILLY", "create_at": date("2012-02-02"), "description": "Forward Is My Progress! Raw Deal Squad Ent., Mobile DJ For Bookings djcozrds@gmail.com or 215-501-9082", "followers_count": 1166, "friends_count": 1841, "statues_count": 52159 }, "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-02T10:00:06.000Z"), "id": 683347047424983040, "text": "Aluminum Welder - Aerotek: (#Honolulu, HI) https://t.co/l6rMf1Gw2Z #Manufacturing #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "nl", "is_retweet": false, "hashtags": {{ "Honolulu", "Manufacturing", "Job", "Jobs", "Hiring" }}, "user": { "id": 23527770, "name": "TMJ - HON Manuf Jobs", "screen_name": "tmj_hon_manuf", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Manufacturing job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 224, "friends_count": 202, "statues_count": 1 }, "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-02T10:00:06.000Z"), "id": 683347047655706624, "text": "Go crazy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2366018721, "name": "ant", "screen_name": "Antvargas300", "lang": "en", "location": "stamf", "create_at": date("2014-02-25"), "description": "null", "followers_count": 417, "friends_count": 385, "statues_count": 1570 }, "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-02T10:00:06.000Z"), "id": 683347047878115328, "text": "#HolaSebastian tweeteame una selfie tuya de ahora! @SEBTSB x25", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "HolaSebastian" }}, "user_mentions": {{ 1631128196 }}, "user": { "id": 2621545545, "name": "sarah", "screen_name": "confusemendes", "lang": "en", "location": "New York, USA", "create_at": date("2014-06-18"), "description": "* ✧ · . ☾ *| njh | jw | sm |* ✧ ·. ☾ *", "followers_count": 2704, "friends_count": 2079, "statues_count": 22265 }, "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-01-02T10:00:06.000Z"), "id": 683347047991369730, "text": "really sucks watching a loved one's health deteriorate and there's nothing you can do 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": 928893798, "name": "Taylor Smith", "screen_name": "xo_taylorsmith", "lang": "en", "location": "Cleveland, OH", "create_at": date("2012-11-05"), "description": "CSU", "followers_count": 452, "friends_count": 226, "statues_count": 15722 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Amherst, OH", "id": "911b22d874c731d0", "name": "Amherst", "place_type": "city", "bounding_box": rectangle("-82.265659,41.373736 -82.164579,41.426754") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39093, "countyName": "Lorain", "cityID": 3901798, "cityName": "Amherst" } }
+{ "create_at": datetime("2016-01-02T10:00:06.000Z"), "id": 683347048201097216, "text": "In serious preparation for @warpcon...apparently... - Drinking an Irish Stout at @raglanroadpub - https://t.co/kbnYgIzb5B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.5177,28.3711"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 179528762, 35746863 }}, "user": { "id": 14167540, "name": "John Kovalic", "screen_name": "muskrat_john", "lang": "en", "location": "Madison, WI", "create_at": date("2008-03-17"), "description": "I never could get the hang of Thursdays.", "followers_count": 16078, "friends_count": 788, "statues_count": 98796 }, "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": 1237625, "cityName": "Lake Buena Vista" } }
+{ "create_at": datetime("2016-01-02T10:00:06.000Z"), "id": 683347048356130816, "text": "IM ALWAYS HERE FOR THIS NICKI https://t.co/iieSarBLVD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 65460103, "name": "✨FEB2️⃣1️⃣✨", "screen_name": "tina_TURNHEROUT", "lang": "en", "location": "null", "create_at": date("2009-08-13"), "description": "@Drake is my everything, been about ya & im still about ya ✨", "followers_count": 1171, "friends_count": 726, "statues_count": 158533 }, "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-02T10:00:07.000Z"), "id": 683347048649912320, "text": "Things that make you go hmmmm! #deep #feelings #motivation #whatever @ Discovery at Kingwood https://t.co/xCPOAtrW7m", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.24658922,30.05893024"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "deep", "feelings", "motivation", "whatever" }}, "user": { "id": 78328839, "name": "Melissa Ibanez", "screen_name": "MelissaIbanez", "lang": "en", "location": "Houston, TX", "create_at": date("2009-09-29"), "description": "Work-aholic, MBA student. Fiesty, sarcastic, love my friends and family like crazy. Oh and how can I forget about my sailor-like potty mouth. :-)", "followers_count": 77, "friends_count": 87, "statues_count": 754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsville, TX", "id": "00b50e784d19b32f", "name": "Pittsville", "place_type": "city", "bounding_box": rectangle("-95.282602,30.032709 -95.222816,30.071386") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-02T10:00:07.000Z"), "id": 683347048687484928, "text": "Kci & Jojo ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "lt", "is_retweet": false, "user": { "id": 522604883, "name": "syd ✨", "screen_name": "lilsydnee", "lang": "en", "location": "flawda", "create_at": date("2012-03-12"), "description": "you know me", "followers_count": 460, "friends_count": 337, "statues_count": 54340 }, "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-01-02T10:00:07.000Z"), "id": 683347048775602176, "text": "@__paigelee best day ever", "in_reply_to_status": 683346786644287489, "in_reply_to_user": 2188259741, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2188259741 }}, "user": { "id": 366344440, "name": "Payton Clawson", "screen_name": "papaclaw", "lang": "en", "location": "null", "create_at": date("2011-09-01"), "description": "#DontBeAverage UC Football #56", "followers_count": 742, "friends_count": 624, "statues_count": 10279 }, "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-01-02T10:00:07.000Z"), "id": 683347048792461312, "text": "@keyizzle_ ��", "in_reply_to_status": 683001639758860289, "in_reply_to_user": 2748664565, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2748664565 }}, "user": { "id": 2748664565, "name": "key..", "screen_name": "keyizzle_", "lang": "en", "location": "null", "create_at": date("2014-08-20"), "description": "Joi☺️ Kiara❣", "followers_count": 993, "friends_count": 627, "statues_count": 21482 }, "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-02T10:00:07.000Z"), "id": 683347048846917632, "text": "The Payoff EP is now available on SoundCloud. Please, give it a listen and share it if you like it ��\n\nhttps://t.co/hvpQxCEVOh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 60706091, "name": "Kory LaQuess Pullam", "screen_name": "LaQuess", "lang": "en", "location": "Minneapolis", "create_at": date("2009-07-27"), "description": "Performer • Writer • Improviser • Producer • Leader", "followers_count": 536, "friends_count": 641, "statues_count": 12477 }, "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-02T10:00:07.000Z"), "id": 683347048989650944, "text": "What should I eat today?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 321031630, "name": "Chelt ✨", "screen_name": "chelsmarie0204", "lang": "en", "location": "#WCU18", "create_at": date("2011-06-20"), "description": "broke, hungry, and emotionally unstable 24/7 so yeah I'm in college", "followers_count": 324, "friends_count": 649, "statues_count": 2937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "King, NC", "id": "9f6a388f85a93288", "name": "King", "place_type": "city", "bounding_box": rectangle("-80.40585,36.238116 -80.306974,36.343131") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37169, "countyName": "Stokes", "cityID": 3735760, "cityName": "King" } }
+{ "create_at": datetime("2016-01-02T10:00:07.000Z"), "id": 683347049161428992, "text": "@ramonashelburne @espnWD @hburns33 @molly_knight wow excellent job miss ramona", "in_reply_to_status": 683178615425482752, "in_reply_to_user": 17507250, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17507250, 29507226, 336955874, 45882011 }}, "user": { "id": 4692877670, "name": "Carlos Samaniego", "screen_name": "CarlosS65488058", "lang": "es", "location": "null", "create_at": date("2016-01-01"), "description": "null", "followers_count": 25, "friends_count": 284, "statues_count": 34 }, "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-02T10:00:07.000Z"), "id": 683347049262137344, "text": "I'm crazy asf.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 543817175, "name": "big red", "screen_name": "itsaudreyyo", "lang": "en", "location": "gotta jet ", "create_at": date("2012-04-02"), "description": "junior | softball | ginger | d.w.", "followers_count": 398, "friends_count": 653, "statues_count": 7068 }, "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-02T10:00:07.000Z"), "id": 683347049270644736, "text": "Me and stinky was blasting Chris Brown all day yesterday 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": 228265333, "name": "Jada ♥", "screen_name": "iamjadamichelle", "lang": "en", "location": "null", "create_at": date("2010-12-18"), "description": "VII•XXII•MCMXCVII♋ \n\n \nSnapchat• jada_greeny", "followers_count": 441, "friends_count": 326, "statues_count": 17958 }, "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-02T10:00:07.000Z"), "id": 683347049274818560, "text": "Have you tried our new peach & matcha green tea smoothie?!? It's delish! #nomnomnom @ Stakz… https://t.co/5XQqlEmk0e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.43172054,36.96163623"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "nomnomnom" }}, "user": { "id": 282682485, "name": "Stakz Frozen Yogurt", "screen_name": "StakzBG", "lang": "en", "location": "Bowling Green, KY", "create_at": date("2011-04-15"), "description": "Located behind Rafferty's in the Thoroughbred Shopping Center. Come get your STAKZ on!", "followers_count": 1588, "friends_count": 307, "statues_count": 958 }, "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-02T10:00:07.000Z"), "id": 683347049568288769, "text": "Morning clouds/afternoon sun this afternoon, high 45 (7 C). Low 29 (-2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712530, "name": "MidlandWeather", "screen_name": "MidlandWeather", "lang": "en", "location": "Midland/Odessa, TX", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Midland/Odessa, TX", "followers_count": 880, "friends_count": 93, "statues_count": 8117 }, "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-01-02T10:00:07.000Z"), "id": 683347050562387968, "text": "I'm trying 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": 2872358017, "name": "Jada Smith", "screen_name": "iamjadasmith", "lang": "en", "location": "HOUSTON", "create_at": date("2014-10-22"), "description": "18| HairStylist | http://Styleseat.com/StylesByJada | iamjadasmith@icloud.com", "followers_count": 936, "friends_count": 277, "statues_count": 9996 }, "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-01-02T10:00:07.000Z"), "id": 683347051023749120, "text": "https://t.co/CzDeXlwKjS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 824331973, "name": "Thomas Menke", "screen_name": "menke_thomas", "lang": "en", "location": "null", "create_at": date("2012-09-14"), "description": "▪️Wake Forest ⚽️ commit '20▪️If God is for us, who can be against us? w/ soph", "followers_count": 550, "friends_count": 235, "statues_count": 3777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fredericksburg, TX", "id": "d0fae5a6f195a1f8", "name": "Fredericksburg", "place_type": "city", "bounding_box": rectangle("-98.905602,30.243818 -98.83495,30.308171") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48171, "countyName": "Gillespie", "cityID": 4827348, "cityName": "Fredericksburg" } }
+{ "create_at": datetime("2016-01-02T10:00:07.000Z"), "id": 683347051233447937, "text": "1/2/2016 - 12:00\nTemp: 48.4F \nHum: 52%\nWind: 2.0 mph\nBaro: 30.374in. & 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": 106, "friends_count": 52, "statues_count": 48634 }, "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-02T10:00:07.000Z"), "id": 683347051539750912, "text": "Best one yet!! #elephant #FavorieAnimal @ Carnival Magic! https://t.co/12RlXvPX5w", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.79660757,29.30878021"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "elephant", "FavorieAnimal" }}, "user": { "id": 37236676, "name": "Michael M", "screen_name": "ThinAndNotSunny", "lang": "en", "location": "null", "create_at": date("2009-05-02"), "description": "Ut18 McCombs \\m/ HOOK EM", "followers_count": 243, "friends_count": 239, "statues_count": 1474 }, "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-01-02T10:00:07.000Z"), "id": 683347051594256385, "text": "\"The mind of man is capable of anything--because everything is in it, all the past as well as all the future.\" - Joseph Conrad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1099847616, "name": "Miguel Jackson", "screen_name": "GodsWordIsFree", "lang": "en", "location": "NYC now...ATL summer 2016", "create_at": date("2013-01-17"), "description": "Founder of the World Bible Project. We spread the Word by offering free Bibles. Soon will assist, inspire, and empower people to home ownership in ATL area.", "followers_count": 24639, "friends_count": 24387, "statues_count": 53098 }, "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-02T10:00:07.000Z"), "id": 683347051615158272, "text": "My mom loved my #Zootopia stuff so much, she's going to get flatsies for my nephews and niece. Everybody gets a Judy Hopps!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Zootopia" }}, "user": { "id": 616675970, "name": "Meghan", "screen_name": "MinnesotaMeghan", "lang": "en", "location": "Minnesota", "create_at": date("2012-06-23"), "description": "Zootopia is where I belong. Ginnifer Goodwin is my spirit animal. Adele speaks to my soul. Pure Minnesotan.", "followers_count": 59, "friends_count": 162, "statues_count": 5099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Savage, MN", "id": "aab09ade8f67410a", "name": "Savage", "place_type": "city", "bounding_box": rectangle("-93.39978,44.694941 -93.328217,44.790981") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27139, "countyName": "Scott", "cityID": 2758738, "cityName": "Savage" } }
+{ "create_at": datetime("2016-01-02T10:00:07.000Z"), "id": 683347051623673856, "text": "I wanna tyedye some shirts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 73400674, "name": "Stan", "screen_name": "yelnats_", "lang": "en", "location": "United States", "create_at": date("2009-09-11"), "description": "null", "followers_count": 666, "friends_count": 574, "statues_count": 59296 }, "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-02T10:00:07.000Z"), "id": 683347051741097984, "text": "@gurtbunn ☺️ https://t.co/9oHb6UjG3k", "in_reply_to_status": -1, "in_reply_to_user": 2458568077, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2458568077 }}, "user": { "id": 729339374, "name": "Sarah Elston", "screen_name": "selston5", "lang": "en", "location": "null", "create_at": date("2012-07-31"), "description": "null", "followers_count": 425, "friends_count": 548, "statues_count": 2229 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westmont, PA", "id": "d8c70f2bca39489e", "name": "Westmont", "place_type": "city", "bounding_box": rectangle("-78.985595,40.306212 -78.921944,40.335526") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42021, "countyName": "Cambria", "cityID": 4283584, "cityName": "Westmont" } }
+{ "create_at": datetime("2016-01-02T10:00:07.000Z"), "id": 683347051761905664, "text": "Happy 9th adoption birthday to this old crab!! ❤️ @ Poplar Grove Golf Course https://t.co/6JmwlgFWuj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.04736266,37.55033126"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21707516, "name": "Kat VigilanteVeverka", "screen_name": "Kat_V_Veverka", "lang": "en", "location": "Amherst, Va", "create_at": date("2009-02-23"), "description": "Been married for 8 years and feels more like 18. We have 2 beautiful children. Our son is 10 and our daughter is 7 and one more on the way!", "followers_count": 387, "friends_count": 1195, "statues_count": 16760 }, "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": 51009, "countyName": "Amherst" } }
+{ "create_at": datetime("2016-01-02T10:00:07.000Z"), "id": 683347052047159300, "text": "Rain this afternoon, high 46 (8 C). Low 41 (5 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712593, "name": "CorpusChristiWX", "screen_name": "CorpusChristiWX", "lang": "en", "location": "Corpus Christi, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Corpus Christi, Texas", "followers_count": 1014, "friends_count": 93, "statues_count": 8108 }, "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-02T10:00:07.000Z"), "id": 683347052504326144, "text": "Multiple orgasms, that's my MO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1597637138, "name": "rob santos", "screen_name": "recluserob", "lang": "en", "location": "Tempe, AZ", "create_at": date("2013-07-15"), "description": "Kobe20", "followers_count": 521, "friends_count": 398, "statues_count": 20165 }, "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-02T10:00:07.000Z"), "id": 683347052537876480, "text": "What's a Longhorn grad like myself to wear to this year's @valeroalamobowl? I don't want to be \"that guy\" in burnt orange!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19612436 }}, "user": { "id": 37266742, "name": "Ellie Holmes", "screen_name": "ellierosetx", "lang": "en", "location": "San Antonio, TX", "create_at": date("2009-05-02"), "description": "4 PM producer @KENS5 - sometimes I'm on TV | @UTAustin Alum | Texan by the grace of God | Dog mom | #raceforseis | News tips: eholmes@kens5.com", "followers_count": 1630, "friends_count": 1575, "statues_count": 49528 }, "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-02T10:00:08.000Z"), "id": 683347052986777601, "text": "I'm at Pier 1 Imports Corporate Office in Fort Worth, TX https://t.co/MGUPP2CfPl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.34384537,32.75263255"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 383901955, "name": "Josh Vickerson", "screen_name": "joshvickerson", "lang": "en", "location": "Rochester, New York", "create_at": date("2011-10-02"), "description": "From the Internet. Proud @RITtigers student. Aspiring #minimalist. Love/hate relationship with #javascript.\nCo-founder @RITLocalhost | Developer @myspiderwebapp", "followers_count": 362, "friends_count": 248, "statues_count": 17223 }, "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-02T10:00:08.000Z"), "id": 683347053385093120, "text": "Doesn't even feel real that I'll be turning 20 in 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": 801136436, "name": "lex", "screen_name": "vexilanity", "lang": "en", "location": "null", "create_at": date("2012-09-03"), "description": "|Tetrahydrocannabinol | 19 | Capricorn|", "followers_count": 419, "friends_count": 87, "statues_count": 8646 }, "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-01-02T10:00:08.000Z"), "id": 683347053414584321, "text": "@Rebel2Amerikkka you still not talking to me", "in_reply_to_status": -1, "in_reply_to_user": 211418311, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 211418311 }}, "user": { "id": 241562484, "name": "Yves St.Lauren", "screen_name": "Kolour_mePINK", "lang": "en", "location": "KING", "create_at": date("2011-01-22"), "description": "I just write beautiful love stories I don't think i'll ever live one", "followers_count": 1134, "friends_count": 988, "statues_count": 77441 }, "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-02T10:00:08.000Z"), "id": 683347053653528577, "text": "My bday in 23 days the turn up gone be real asf ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 51360387, "name": "Madison Monae", "screen_name": "Boss_avee", "lang": "en", "location": "null", "create_at": date("2009-06-26"), "description": "4:14. Trσυвle Lιғe", "followers_count": 268, "friends_count": 451, "statues_count": 3583 }, "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-01-02T10:00:08.000Z"), "id": 683347053913726980, "text": "Kevin Gates and I https://t.co/Wwh8F1VWCb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 564883792, "name": "Morgan Morrow", "screen_name": "morgie_inparis", "lang": "en", "location": "San Marcos, Tx", "create_at": date("2012-04-27"), "description": "#TXST19 | Sometimes it's the journey that teaches you a lot about your destination.", "followers_count": 153, "friends_count": 157, "statues_count": 161 }, "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-02T10:00:08.000Z"), "id": 683347054001807360, "text": "Ty pretty!�� RT @tatertatss: @fernyyy12 @jennysmallls happy birthday girls! ����", "in_reply_to_status": 683289978646958080, "in_reply_to_user": 32476424, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 32476424, 232365647, 159879657 }}, "user": { "id": 159879657, "name": "Jennifer", "screen_name": "jennysmallls", "lang": "en", "location": "Wade County", "create_at": date("2010-06-26"), "description": "null", "followers_count": 1016, "friends_count": 605, "statues_count": 36127 }, "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("2016-01-02T10:00:08.000Z"), "id": 683347054014365696, "text": "13:00:02 |Temp: 47.7ºF | Wind Chill 47.7ºF |Dew Point 38.0ºF | Rain today: 0.00 inches | Wind: 0.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": 96, "friends_count": 21, "statues_count": 87901 }, "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-02T10:00:08.000Z"), "id": 683347054144389120, "text": "Privacy scares those who always want 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": 124340941, "name": "Kyle Zackary", "screen_name": "Heil_Kyle_", "lang": "en", "location": "null", "create_at": date("2010-03-18"), "description": "tremendous cream, fuck a dollar and a dream", "followers_count": 383, "friends_count": 351, "statues_count": 50977 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Knightdale, NC", "id": "015772750fc654c1", "name": "Knightdale", "place_type": "city", "bounding_box": rectangle("-78.540011,35.753563 -78.458292,35.823284") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3736080, "cityName": "Knightdale" } }
+{ "create_at": datetime("2016-01-02T10:00:08.000Z"), "id": 683347054152663040, "text": "Aminoácidos y café negro como tu conciencia para pre Workout.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 716342946, "name": "ram", "screen_name": "Rambocriollo", "lang": "es", "location": "En tu corazón", "create_at": date("2012-07-25"), "description": "Software Developer, Personal Trainer, gordo de corazón y suculento como pierna de chancho.", "followers_count": 12335, "friends_count": 3085, "statues_count": 140978 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-02T10:00:08.000Z"), "id": 683347054182010880, "text": "Some of these people on horse tourneys lol. No matter how many entries you buy you still can't get a top 10 finish. #saveyourmoney", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "saveyourmoney" }}, "user": { "id": 445230490, "name": "Nick DAgostino", "screen_name": "Chef_vegas", "lang": "en", "location": "Newport Beach, CA ", "create_at": date("2011-12-23"), "description": "Executive Chef/ Father of 2/New York born/Horse Racing Handicapper/Sports Gambling Sharp/Sicilian/ Expert in Restaurant construction and design, NY GIANTS hater", "followers_count": 79, "friends_count": 284, "statues_count": 1543 }, "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-02T10:00:08.000Z"), "id": 683347054438051840, "text": "Ayeeeeee!!! Tumi.teenager Na naman ako!hihihi #SPSHello2016 https://t.co/7IJfyvxNai", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "SPSHello2016" }}, "user": { "id": 3627191477, "name": "EJMP", "screen_name": "emyak_emyat", "lang": "en", "location": "Waukegan, IL", "create_at": date("2015-09-11"), "description": "I am ME.", "followers_count": 82, "friends_count": 120, "statues_count": 4729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elk Grove Village, IL", "id": "227d358f6ed8d29e", "name": "Elk Grove Village", "place_type": "city", "bounding_box": rectangle("-88.0605,41.983375 -87.935116,42.040488") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1723256, "cityName": "Elk Grove Village" } }
+{ "create_at": datetime("2016-01-02T10:00:08.000Z"), "id": 683347054463156224, "text": "@mattmclendonyo Bc you hate me", "in_reply_to_status": 683345812936617984, "in_reply_to_user": 1798427671, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1798427671 }}, "user": { "id": 3102255874, "name": "Katsandra", "screen_name": "KATsandrajayne", "lang": "en", "location": "Zanesville, OH", "create_at": date("2015-03-21"), "description": "snapchat & instagram @kassandrajayne", "followers_count": 568, "friends_count": 548, "statues_count": 6615 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Zanesville, OH", "id": "f4a27b9e3d616757", "name": "Zanesville", "place_type": "city", "bounding_box": rectangle("-82.121534,39.908312 -81.976901,39.999527") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39119, "countyName": "Muskingum", "cityID": 3988084, "cityName": "Zanesville" } }
+{ "create_at": datetime("2016-01-02T10:00:08.000Z"), "id": 683347054534365184, "text": "@taylormathenia lifesaver", "in_reply_to_status": 683345989797822465, "in_reply_to_user": 2467015128, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2467015128 }}, "user": { "id": 1028541002, "name": "zak kieser", "screen_name": "ZKieser", "lang": "en", "location": "null", "create_at": date("2012-12-22"), "description": "Belleville West", "followers_count": 404, "friends_count": 317, "statues_count": 1122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Millstadt, IL", "id": "00a56a8d6eacde1f", "name": "Millstadt", "place_type": "city", "bounding_box": rectangle("-90.108729,38.425523 -90.073085,38.4691") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17163, "countyName": "St. Clair", "cityID": 1749386, "cityName": "Millstadt" } }
+{ "create_at": datetime("2016-01-02T10:00:08.000Z"), "id": 683347054752612352, "text": "Uhh... I can explain https://t.co/uXQRJQ3DpW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 100529379, "name": "Amethyst Thurmond", "screen_name": "amethysthurmond", "lang": "en", "location": "small town, west virginia", "create_at": date("2009-12-30"), "description": "•WVU '19•", "followers_count": 602, "friends_count": 332, "statues_count": 12093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Barboursville, WV", "id": "bedb151fa68edb66", "name": "Barboursville", "place_type": "city", "bounding_box": rectangle("-82.322131,38.383756 -82.249985,38.427939") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54011, "countyName": "Cabell", "cityID": 5404276, "cityName": "Barboursville" } }
+{ "create_at": datetime("2016-01-02T10:00:08.000Z"), "id": 683347054983159808, "text": "Sunny this afternoon, high 50 (10 C). Low 27 (-3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712706, "name": "TulsaWX", "screen_name": "TulsaWX", "lang": "en", "location": "Tulsa, Oklahoma", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Tulsa, Oklahoma", "followers_count": 518, "friends_count": 93, "statues_count": 8170 }, "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-02T10:00:08.000Z"), "id": 683347055004233728, "text": "Don't even count ���� https://t.co/QaR4Y80lCz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 637261538, "name": "GQ", "screen_name": "_CATDOG__", "lang": "en", "location": "null", "create_at": date("2012-07-16"), "description": "SEPTEMBER 23", "followers_count": 298, "friends_count": 271, "statues_count": 10291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oberlin, OH", "id": "3cc5ef44326641fb", "name": "Oberlin", "place_type": "city", "bounding_box": rectangle("-82.241921,41.279512 -82.186074,41.305916") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39093, "countyName": "Lorain", "cityID": 3957834, "cityName": "Oberlin" } }
+{ "create_at": datetime("2016-01-02T10:00:08.000Z"), "id": 683347055054577664, "text": "@HackworthBobby sand keys gonna be live on Monday? I'm down to go dude", "in_reply_to_status": 683346869192372225, "in_reply_to_user": 2491053393, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2491053393 }}, "user": { "id": 1480914757, "name": "Tall boy", "screen_name": "RaymondGillum", "lang": "en", "location": "null", "create_at": date("2013-06-03"), "description": "IG: itssrayyy #Respectheforce, You'll Find me with my #Buschbrothers", "followers_count": 881, "friends_count": 385, "statues_count": 10080 }, "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-01-02T10:00:08.000Z"), "id": 683347055708774400, "text": "@pkedrosky @exjon Most cable moneey comes from subscription fees, which means the demo does not matter\nMedian age of a newspaper reader?\n65", "in_reply_to_status": 683303142184779776, "in_reply_to_user": 1717291, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1717291, 16930489 }}, "user": { "id": 18034072, "name": "Don Surber", "screen_name": "donsurber", "lang": "en", "location": "Poca WV", "create_at": date("2008-12-10"), "description": "Retired. Author of EXCEPTIONAL AMERICANS 1 http://t.co/VVsXoqxSM9 EXCEPTIONAL AMERICANS 2: THE CAPITALISTS http://t.co/IQYwFlhQnO", "followers_count": 2805, "friends_count": 82, "statues_count": 30197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Poca, WV", "id": "001a9f0813d7a82f", "name": "Poca", "place_type": "city", "bounding_box": rectangle("-81.825749,38.43513 -81.799353,38.490068") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54079, "countyName": "Putnam", "cityID": 5464516, "cityName": "Poca" } }
+{ "create_at": datetime("2016-01-02T10:00:08.000Z"), "id": 683347056451137537, "text": "The guy in this movie is 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": 499266299, "name": "lemon_lime_margarita", "screen_name": "MargaritaLettuc", "lang": "en", "location": "probably by the food ", "create_at": date("2012-02-21"), "description": "people see what they wish to see, in most cases they are told what to see.", "followers_count": 692, "friends_count": 1260, "statues_count": 7390 }, "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-02T10:00:08.000Z"), "id": 683347056455385088, "text": "yes lawd https://t.co/YuQV0Q8MVo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 338832424, "name": "Petty Wap ⚜", "screen_name": "ScottieDrippin", "lang": "en", "location": "Next To The Money", "create_at": date("2011-07-19"), "description": "got her own shit, she a hu$tler, she a grinder.", "followers_count": 1880, "friends_count": 1124, "statues_count": 157238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-02T10:00:08.000Z"), "id": 683347056564514816, "text": "It has felt like a Saturday for the past week.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 565585048, "name": "Lyndsay", "screen_name": "LyndsayHampton2", "lang": "en", "location": "probs watching netflix", "create_at": date("2012-04-28"), "description": "I haven't been everywhere, but it's on my list", "followers_count": 370, "friends_count": 166, "statues_count": 4550 }, "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-02T10:00:08.000Z"), "id": 683347056665083904, "text": "@nicekicks @Silverlight54 ?", "in_reply_to_status": 683316817620668417, "in_reply_to_user": 15097566, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 15097566, 369601278 }}, "user": { "id": 1363450370, "name": "McGregor", "screen_name": "Mac_McGregor23", "lang": "en", "location": "null", "create_at": date("2013-04-18"), "description": "Hi Jacey", "followers_count": 278, "friends_count": 192, "statues_count": 3408 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Purcell, OK", "id": "1f9f17a2260b4ff8", "name": "Purcell", "place_type": "city", "bounding_box": rectangle("-97.395705,34.971543 -97.347925,35.058867") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40087, "countyName": "McClain", "cityID": 4061150, "cityName": "Purcell" } }
+{ "create_at": datetime("2016-01-02T10:00:08.000Z"), "id": 683347056782635008, "text": "@maryswazi I had plans what you saying ��", "in_reply_to_status": 683346260703576068, "in_reply_to_user": 328871985, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 328871985 }}, "user": { "id": 517118447, "name": "Blanca.", "screen_name": "underRose_", "lang": "en", "location": "sin city", "create_at": date("2012-03-06"), "description": "null", "followers_count": 850, "friends_count": 604, "statues_count": 36265 }, "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-01-02T10:00:08.000Z"), "id": 683347056853831680, "text": "2nd https://t.co/PpxUdj7Vcs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2951384420, "name": "Kristèn .", "screen_name": "Whitegalkris", "lang": "en", "location": "null", "create_at": date("2014-12-29"), "description": "Follow Me On Instagram @Whitegalkris__✈️ AMOSC Whitegalkris⌛️", "followers_count": 1750, "friends_count": 986, "statues_count": 15909 }, "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-02T10:00:08.000Z"), "id": 683347056866504704, "text": "@n3wp3rsp3ctiv3 when you're best friend is the best ��❤️", "in_reply_to_status": 683346446104526848, "in_reply_to_user": 2845140374, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2845140374 }}, "user": { "id": 1620587166, "name": "Sophs", "screen_name": "sophia__davis", "lang": "en", "location": "g", "create_at": date("2013-07-25"), "description": "WSHS", "followers_count": 141, "friends_count": 143, "statues_count": 1361 }, "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-02T10:00:08.000Z"), "id": 683347056908451842, "text": "What yaw doin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 171961162, "name": "Black Bulma.", "screen_name": "Lee_Phenomenal", "lang": "en", "location": "Columbus, OH", "create_at": date("2010-07-28"), "description": "Firm believer of refilling the first swig of juice.", "followers_count": 1381, "friends_count": 809, "statues_count": 110130 }, "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-02T10:00:09.000Z"), "id": 683347056950403072, "text": "@MoSayed2 @WrightorWrongAL @pambesteder @AllenWest The people of UK have allowed this to happen. Its to late for England! STOP IT HERE NOW!", "in_reply_to_status": 683319905370820612, "in_reply_to_user": 911500014, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 911500014, 2678528420, 58510019, 237403203 }}, "user": { "id": 3407362587, "name": "BrevardFlorida4Trump", "screen_name": "brant_roger", "lang": "en", "location": "null", "create_at": date("2015-08-07"), "description": "null", "followers_count": 532, "friends_count": 39, "statues_count": 5371 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merritt Island, FL", "id": "13fa221b2b90ae80", "name": "Merritt Island", "place_type": "city", "bounding_box": rectangle("-80.731463,28.142537 -80.60386,28.408742") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12009, "countyName": "Brevard", "cityID": 1244275, "cityName": "Merritt Island" } }
+{ "create_at": datetime("2016-01-02T10:00:09.000Z"), "id": 683347057080283136, "text": "Wind 0.3 mph NNW. Barometer 30.522 in, Falling slowly. Temperature 41.1 °F. Rain today 0.12in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.11555556,29.6725"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25210383, "name": "New Braunfels WX", "screen_name": "jeffreycentexwx", "lang": "en", "location": "New Braunfels, TX", "create_at": date("2009-03-18"), "description": "Weather Station located on County Line Road in New Braunfels, TX.\r\n\r\nRealtime Weather @ http://www.nbtx-wx.com\r\nFollow me @ http://twitter.com/jeffreycentex", "followers_count": 379, "friends_count": 24, "statues_count": 159807 }, "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-01-02T10:00:09.000Z"), "id": 683347057223012352, "text": "Difference right now is 3point shooting. #UNC is 1 of 7 as a team. Adam Smith is 4 of 5 by himself.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "UNC" }}, "user": { "id": 19192748, "name": "Star-News ACC", "screen_name": "starnewsacc", "lang": "en", "location": "null", "create_at": date("2009-01-19"), "description": "null", "followers_count": 2959, "friends_count": 637, "statues_count": 56955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chapel Hill, NC", "id": "23ebe811ea93fd41", "name": "Chapel Hill", "place_type": "city", "bounding_box": rectangle("-79.083153,35.869376 -78.968887,35.969507") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37135, "countyName": "Orange", "cityID": 3711800, "cityName": "Chapel Hill" } }
+{ "create_at": datetime("2016-01-02T10:00:09.000Z"), "id": 683347057315188736, "text": "#Homewood, AL #Sales #Job: Part Time Customer Service Associate II - Weekday Flex Team at Lowe's https://t.co/usHtJgZL9L #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.8008228,33.4717732"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Homewood", "Sales", "Job", "Jobs", "Hiring" }}, "user": { "id": 21315667, "name": "TMJ - BHM Sales Jobs", "screen_name": "tmj_bhm_sales", "lang": "en", "location": "Birmingham, AL", "create_at": date("2009-02-19"), "description": "Follow this account for geo-targeted Sales job tweets in Birmingham, AL. Need help? Tweet us at @CareerArc!", "followers_count": 386, "friends_count": 308, "statues_count": 95 }, "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-02T10:00:09.000Z"), "id": 683347057449414656, "text": "Partly cloudy this afternoon, high 45 (7 C). Low 27 (-3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712679, "name": "OKCWX", "screen_name": "OKCWX", "lang": "en", "location": "Oklahoma City, Oklahoma", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Oklahoma City, Oklahoma", "followers_count": 185, "friends_count": 93, "statues_count": 8197 }, "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-02T10:00:09.000Z"), "id": 683347057613000704, "text": "Every time I hear the word \"pageantry\" while watching that game, I throw up a little it in the back of my mouth... https://t.co/OW1ygA78TL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 112057733, "name": "Kelly Williams", "screen_name": "williamsjrkelly", "lang": "en", "location": "Clay Center, KS", "create_at": date("2010-02-06"), "description": "Husband, Father, Teacher and Coach...", "followers_count": 425, "friends_count": 370, "statues_count": 9498 }, "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-02T10:00:09.000Z"), "id": 683347057667518465, "text": "I feel like having some golden wok or ding how rn ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 379918542, "name": "Abree", "screen_name": "abreanna13", "lang": "en", "location": "null", "create_at": date("2011-09-25"), "description": "SHS Varsity cheer", "followers_count": 246, "friends_count": 212, "statues_count": 970 }, "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-02T10:00:09.000Z"), "id": 683347057759813632, "text": "Refuge. Love the artwork Suzanne does. . #Buddhism #Seattle #Kadampa https://t.co/tgyCl0Q1yB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Buddhism", "Seattle", "Kadampa" }}, "user": { "id": 2211532982, "name": "Kelsang Khedrub", "screen_name": "KelsangKhedrub", "lang": "en", "location": "Seattle, WA", "create_at": date("2013-11-23"), "description": "Buddhist monk living in Seattle.", "followers_count": 875, "friends_count": 156, "statues_count": 1894 }, "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-02T10:00:09.000Z"), "id": 683347058095357954, "text": "Last day home ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1912899078, "name": "n̶o̶a̶h̶ sam macrane", "screen_name": "ThomasMacrina", "lang": "en", "location": "Space", "create_at": date("2013-09-27"), "description": "my first names crossed out cause it's not important anymore", "followers_count": 175, "friends_count": 366, "statues_count": 4400 }, "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-02T10:00:09.000Z"), "id": 683347058116444161, "text": "@JCC_xo19 iron roost in ballston spa!!!", "in_reply_to_status": 683346912506941440, "in_reply_to_user": 3416963068, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3416963068 }}, "user": { "id": 506760475, "name": "Annese Kim❁☪", "screen_name": "annese_kimberly", "lang": "en", "location": "ig//annese_kimberly ", "create_at": date("2012-02-27"), "description": "|FJL| • College of St. Rose '17. Albany, NY.", "followers_count": 1039, "friends_count": 671, "statues_count": 30889 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Spa, NY", "id": "6b7ea0f26db87c8b", "name": "Ballston Spa", "place_type": "city", "bounding_box": rectangle("-73.873008,42.967278 -73.835225,43.025133") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36091, "countyName": "Saratoga", "cityID": 3604253, "cityName": "Ballston Spa" } }
+{ "create_at": datetime("2016-01-02T10:00:09.000Z"), "id": 683347058238078976, "text": "How exciting.....Baker family in a #Disney video!! https://t.co/yjohk4MGaT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Disney" }}, "user": { "id": 751308998, "name": "Angela Baker", "screen_name": "angelabaker321", "lang": "en", "location": "null", "create_at": date("2012-08-11"), "description": "null", "followers_count": 111, "friends_count": 125, "statues_count": 1226 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Royalton, OH", "id": "4506ca1d65ca5b4d", "name": "North Royalton", "place_type": "city", "bounding_box": rectangle("-81.785322,41.264708 -81.70169,41.350954") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3957008, "cityName": "North Royalton" } }
+{ "create_at": datetime("2016-01-02T10:00:09.000Z"), "id": 683347058284228608, "text": "@__D_Locc_ What's up?", "in_reply_to_status": 683345526884941824, "in_reply_to_user": 2317277494, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2317277494 }}, "user": { "id": 353314695, "name": "Angela Simmons", "screen_name": "maaganda_x", "lang": "en", "location": "null", "create_at": date("2011-08-11"), "description": "yeah I'm good...", "followers_count": 308, "friends_count": 285, "statues_count": 2296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Williamsburg, FL", "id": "0ca5b17a9815da67", "name": "Williamsburg", "place_type": "city", "bounding_box": rectangle("-81.465117,28.378495 -81.429657,28.421281") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1277735, "cityName": "Williamsburg" } }
+{ "create_at": datetime("2016-01-02T10:00:09.000Z"), "id": 683347058611204096, "text": "GET INTO THE MUSES! https://t.co/SvYBstDlPt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315241300, "name": "J-Rawww", "screen_name": "LordJames23", "lang": "en", "location": "null", "create_at": date("2013-03-29"), "description": "what white nonsense was that?", "followers_count": 709, "friends_count": 470, "statues_count": 55844 }, "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.17314,33.884965") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1377652, "cityName": "Tucker" } }
+{ "create_at": datetime("2016-01-02T10:00:09.000Z"), "id": 683347058678435840, "text": "... Haven't felt that in a long time. Can't remember how it feel honestly.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 408955551, "name": "BAEthoven.", "screen_name": "x_OvO_jAzZ", "lang": "en", "location": "Ohio Against The World.", "create_at": date("2011-11-09"), "description": "introvert.|internal medicine.", "followers_count": 570, "friends_count": 329, "statues_count": 29490 }, "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-02T10:00:09.000Z"), "id": 683347058724630528, "text": "CHECK OUT \"Alufina\" By Wil Milton. Avail on Path Life Music CLICK BELOW to LISTEN\nhttps://t.co/PA2WR8RChH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30327335, "name": "Wil Milton", "screen_name": "WilMIlton", "lang": "en", "location": "New York", "create_at": date("2009-04-10"), "description": "Music Producer, Songwriter, DJ", "followers_count": 2720, "friends_count": 2982, "statues_count": 4859 }, "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-02T10:00:09.000Z"), "id": 683347059320045568, "text": "I would love to show you my #listing at 32728 N 43rd Street #CaveCreek #AZ https://t.co/9AP9Bh5y9v #realestate https://t.co/NsrytbK8Jh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.988306,33.782761"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "CaveCreek", "AZ", "realestate" }}, "user": { "id": 17801950, "name": "James Wehner", "screen_name": "JamesWehnerAZ", "lang": "en", "location": "Scottsdale, AZ", "create_at": date("2008-12-01"), "description": "Associate Broker, REALTOR® with West USA Realty ~ Buying | Selling | Leasing | Property Mgmt ~ Located in the Phoenix, AZ metro area. 1.800.937.8872", "followers_count": 250, "friends_count": 257, "statues_count": 530 }, "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-02T10:00:09.000Z"), "id": 683347059693375488, "text": "Most likely Ima go home & just sleep 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": 314316422, "name": "➖Alexyss", "screen_name": "_fabulex", "lang": "en", "location": "houstontx!", "create_at": date("2011-06-09"), "description": "Remain humble!", "followers_count": 1639, "friends_count": 691, "statues_count": 77571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-02T10:00:09.000Z"), "id": 683347059827580929, "text": "#Accounting #Job in #Quincy, Quincy: Compliance Auditor at Quincy Medical Group https://t.co/M6XCcW1F7N #medical #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.790254,41.0402645"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Accounting", "Job", "Quincy", "medical", "Jobs", "Hiring" }}, "user": { "id": 2166774096, "name": "Quincy MedGroup", "screen_name": "QuincyMedGroup", "lang": "en", "location": "Quincy, IL", "create_at": date("2013-10-31"), "description": "null", "followers_count": 108, "friends_count": 62, "statues_count": 46 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Iowa, USA", "id": "3cd4c18d3615bbc9", "name": "Iowa", "place_type": "admin", "bounding_box": rectangle("-96.639706,40.375437 -90.140061,43.50102") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19003, "countyName": "Adams" } }
+{ "create_at": datetime("2016-01-02T10:00:09.000Z"), "id": 683347059865288704, "text": "Rain this afternoon, high 45 (7 C). Low 39 (4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712488, "name": "SanAntonioWX", "screen_name": "SanAntonioWX", "lang": "en", "location": "San Antonio, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for San Antonio, Texas", "followers_count": 251, "friends_count": 93, "statues_count": 8187 }, "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-02T10:00:09.000Z"), "id": 683347059919945728, "text": "#TakeYourShoot2k16", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "TakeYourShoot2k16" }}, "user": { "id": 301780154, "name": "Boogie✨", "screen_name": "LustMakeYaBust", "lang": "en", "location": "Norfolk, VA", "create_at": date("2011-05-19"), "description": "21 / ⚠️ 90's baby a millennium kid \\ Ambassador of forbidden knowledge", "followers_count": 1944, "friends_count": 917, "statues_count": 42798 }, "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-02T10:00:09.000Z"), "id": 683347060666400768, "text": "@Drake , it's time https://t.co/XH4W1rmZfD", "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": 439623172, "name": "Jazmine", "screen_name": "Ayee_PeepThis", "lang": "en", "location": "null", "create_at": date("2011-12-17"), "description": "htx | UTSA", "followers_count": 717, "friends_count": 601, "statues_count": 18013 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-02T10:00:09.000Z"), "id": 683347060964376576, "text": "@WhatsTheStatus LOL", "in_reply_to_status": 683346985953398785, "in_reply_to_user": 110571555, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 110571555 }}, "user": { "id": 84648756, "name": "Robert Fuse", "screen_name": "ThatRobFuseMan", "lang": "en", "location": "Baltimore, Maryland", "create_at": date("2009-10-23"), "description": "27. Random, sporadic tweeter. I guess I'm a singer-songwriter even though I suck. Resident weirdo. Oh and I photoshop. #RavensNation", "followers_count": 1724, "friends_count": 1982, "statues_count": 11276 }, "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-02T10:00:10.000Z"), "id": 683347061786320896, "text": "@hart_annabelle what the hell hahah", "in_reply_to_status": 679940969559539712, "in_reply_to_user": 635747547, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 635747547 }}, "user": { "id": 736211952, "name": "alisa", "screen_name": "Alisa_M_Ramirez", "lang": "en", "location": "null", "create_at": date("2012-08-03"), "description": "gettalooooowwng lil dowgie", "followers_count": 410, "friends_count": 277, "statues_count": 9144 }, "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-02T10:00:10.000Z"), "id": 683347061920653312, "text": "Happy birthday @Fantone can't wait for the show to come back live", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21314008 }}, "user": { "id": 240578898, "name": "the dustinator™", "screen_name": "dustinv916", "lang": "en", "location": "Cleveland, OH", "create_at": date("2011-01-19"), "description": "country boy at heart living in the big city #CLE, Metalhead, Football, 2nd Amendment\n#20millstrong", "followers_count": 159, "friends_count": 420, "statues_count": 5453 }, "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-01-02T10:00:10.000Z"), "id": 683347061987610625, "text": "@moni_boo2014 happy birthday ����", "in_reply_to_status": -1, "in_reply_to_user": 1244827058, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1244827058 }}, "user": { "id": 35073721, "name": "Jimmy of Dem Boy$", "screen_name": "1Jimmy__", "lang": "en", "location": "Dallas, TX/Commerce, TX", "create_at": date("2009-04-24"), "description": "I really don't givadam bout nothin... Rest Easy Shel D #LongLiveClyde Dem Boy$ Live Forever #TAMUC", "followers_count": 3861, "friends_count": 984, "statues_count": 279222 }, "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-02T10:00:10.000Z"), "id": 683347062071558145, "text": "#TakeAKneeAndThankTheMan", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "TakeAKneeAndThankTheMan" }}, "user": { "id": 149021946, "name": "Torrance Williams", "screen_name": "TWill_10", "lang": "en", "location": "Waco, Texas ", "create_at": date("2010-05-27"), "description": "God 1st • Family Forever • #SHSUAlum • All-American [2 Positions] • IG: TWill_10 • Highlight Video Below!", "followers_count": 1962, "friends_count": 380, "statues_count": 52721 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-02T10:00:10.000Z"), "id": 683347062126194688, "text": "@Demon_Henry22 I look good, chill", "in_reply_to_status": 683343911172071426, "in_reply_to_user": 405991574, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 405991574 }}, "user": { "id": 275243768, "name": "Kendall Stephens", "screen_name": "Kengaroos", "lang": "en", "location": "somewhere stalking katy perry ", "create_at": date("2011-03-31"), "description": "I have cankles and im proud", "followers_count": 1079, "friends_count": 993, "statues_count": 62894 }, "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-02T10:00:10.000Z"), "id": 683347062142926848, "text": "I need a job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 233328252, "name": "Samborghini", "screen_name": "BLXCKSAM", "lang": "en", "location": "null", "create_at": date("2011-01-02"), "description": "null", "followers_count": 1324, "friends_count": 499, "statues_count": 66310 }, "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-02T10:00:10.000Z"), "id": 683347062222553088, "text": "“Life is really simple, but we insist on making it complicated” ~ Confucius https://t.co/3DHAIozbVs @mybalance365", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2416480616 }}, "user": { "id": 354862425, "name": "MiaQDaYogi", "screen_name": "MiaQC", "lang": "en", "location": "Chicago", "create_at": date("2011-08-14"), "description": "null", "followers_count": 19, "friends_count": 36, "statues_count": 57 }, "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-02T10:00:10.000Z"), "id": 683347062549659648, "text": "I would love to show you my #listing at 32728 N 43rd Street #CaveCreek #AZ https://t.co/6pDQNHQXDi #realestate https://t.co/OrJl1jUZkk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.988306,33.782761"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "CaveCreek", "AZ", "realestate" }}, "user": { "id": 17801950, "name": "James Wehner", "screen_name": "JamesWehnerAZ", "lang": "en", "location": "Scottsdale, AZ", "create_at": date("2008-12-01"), "description": "Associate Broker, REALTOR® with West USA Realty ~ Buying | Selling | Leasing | Property Mgmt ~ Located in the Phoenix, AZ metro area. 1.800.937.8872", "followers_count": 250, "friends_count": 257, "statues_count": 531 }, "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-02T10:00:10.000Z"), "id": 683347062608375808, "text": "Metro is the freshest producer of 2k15", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 780205652, "name": "Edoardo", "screen_name": "edoardo_xo", "lang": "en", "location": "6", "create_at": date("2012-08-25"), "description": "Congratulations, you played yourself. Bless up", "followers_count": 436, "friends_count": 322, "statues_count": 7899 }, "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-01-02T10:00:10.000Z"), "id": 683347062671282176, "text": "Cloudy this afternoon, high 51 (11 C). Low 36 (2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712452, "name": "FortWorthWX", "screen_name": "FortWorthWX", "lang": "en", "location": "Fort Worth, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Fort Worth, Texas", "followers_count": 218, "friends_count": 93, "statues_count": 8174 }, "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-02T10:00:10.000Z"), "id": 683347062709219328, "text": "I'm at Vicki's Diner in Westfield, NJ https://t.co/bXOOZMIyXZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.34903,40.65085599"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 271288784, "name": "Sh0rtstack.", "screen_name": "ajavs", "lang": "en", "location": "null", "create_at": date("2011-03-23"), "description": "#GabbyNotAna", "followers_count": 89, "friends_count": 82, "statues_count": 10199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westfield, NJ", "id": "777dd5b3b30a625e", "name": "Westfield", "place_type": "city", "bounding_box": rectangle("-74.379383,40.624437 -74.313149,40.682332") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3479040, "cityName": "Westfield" } }
+{ "create_at": datetime("2016-01-02T10:00:10.000Z"), "id": 683347062864408577, "text": "@shehulkgrr @emoondvik AD ��", "in_reply_to_status": 652099338253004800, "in_reply_to_user": 1461176444, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1461176444, 189630902 }}, "user": { "id": 152307675, "name": "lex", "screen_name": "_AKforty7", "lang": "en", "location": "null", "create_at": date("2010-06-05"), "description": "#dandy", "followers_count": 536, "friends_count": 428, "statues_count": 40139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nyack, NY", "id": "41161b11ee419444", "name": "Nyack", "place_type": "city", "bounding_box": rectangle("-73.93834,41.086552 -73.913934,41.10109") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36087, "countyName": "Rockland", "cityID": 3654100, "cityName": "Nyack" } }
+{ "create_at": datetime("2016-01-02T10:00:10.000Z"), "id": 683347062906212352, "text": "This #Transportation #job might be a great fit for you: Parts Delivery - https://t.co/CZ0jey4QNm #PORTAGE, MI #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.5894106,42.2101135"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Transportation", "job", "PORTAGE", "Hiring", "CareerArc" }}, "user": { "id": 149735286, "name": "TMJ-KAL Transport.", "screen_name": "tmj_KAL_transp", "lang": "en", "location": "Kalamazoo, MI", "create_at": date("2010-05-29"), "description": "Follow this account for geo-targeted Transportation job tweets in Kalamazoo, MI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 278, "friends_count": 274, "statues_count": 23 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portage, MI", "id": "3af1aa005c106143", "name": "Portage", "place_type": "city", "bounding_box": rectangle("-87.222662,41.536049 -85.491151,42.245344") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26077, "countyName": "Kalamazoo", "cityID": 2665560, "cityName": "Portage" } }
+{ "create_at": datetime("2016-01-02T10:00:10.000Z"), "id": 683347062910402560, "text": "LMAOOO https://t.co/ywmOm4U5uO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 142474906, "name": "v", "screen_name": "_notvanessa", "lang": "en", "location": "California, USA", "create_at": date("2010-05-10"), "description": "Kanye 2020", "followers_count": 866, "friends_count": 849, "statues_count": 75361 }, "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-02T10:00:10.000Z"), "id": 683347062931492865, "text": "bruh this song is so freaking good", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1532118410, "name": "JonJon", "screen_name": "GC_Jon", "lang": "en", "location": "#JesusTalks", "create_at": date("2013-06-19"), "description": "there's a line and I've crossed it", "followers_count": 380, "friends_count": 343, "statues_count": 11841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Port, FL", "id": "81b55f54466d8ae4", "name": "North Port", "place_type": "city", "bounding_box": rectangle("-82.272379,27.026227 -82.067993,27.111839") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12115, "countyName": "Sarasota", "cityID": 1249675, "cityName": "North Port" } }
+{ "create_at": datetime("2016-01-02T10:00:10.000Z"), "id": 683347063191408641, "text": "Cool��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2955770096, "name": "Hood ®ich⭐", "screen_name": "YOUNG__N", "lang": "en", "location": "F.XXGLIZZY WORLD❗❗", "create_at": date("2015-01-01"), "description": "yasmine❤||Be somebody❤||", "followers_count": 20, "friends_count": 16, "statues_count": 1435 }, "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-02T10:00:10.000Z"), "id": 683347063204003840, "text": "Bring back the golfing weather.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2657479343, "name": "Zack.", "screen_name": "ffZackree", "lang": "en", "location": "null", "create_at": date("2014-06-30"), "description": "Stay lost and adventure on.", "followers_count": 496, "friends_count": 468, "statues_count": 1614 }, "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-01-02T10:00:10.000Z"), "id": 683347063497601024, "text": "N4TRQ-9 Near Norfolk International Terminal (NIT), Norfolk, VA https://t.co/9RUhbv74Kq #APRS 13b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.32133,36.91983"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "APRS" }}, "user": { "id": 56001919, "name": "Arthur", "screen_name": "n4trqjedi", "lang": "en", "location": "APRS N of Chesapeake, VA", "create_at": date("2009-07-11"), "description": "Electronics Technician and Amateur Radio Operator N4TRQ\r\nhttp://n4trq.wordpress.com", "followers_count": 428, "friends_count": 691, "statues_count": 47216 }, "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-02T10:00:10.000Z"), "id": 683347063699075072, "text": "‼️‼️‼️‼️‼️ https://t.co/43QCfOBM3P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 438632750, "name": "♠️ OG PABLO ♣️", "screen_name": "OMFG_CHRIS", "lang": "en", "location": "null", "create_at": date("2011-12-16"), "description": "Maricka♥️", "followers_count": 672, "friends_count": 560, "statues_count": 24632 }, "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-02T10:00:10.000Z"), "id": 683347064038674433, "text": "@lourryinmyheart Thank you for the follow.", "in_reply_to_status": -1, "in_reply_to_user": 507940553, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 507940553 }}, "user": { "id": 2174567323, "name": "Ann Wilmer-Lasky", "screen_name": "awlasky", "lang": "en", "location": "Roswell, NM", "create_at": date("2013-11-04"), "description": "Strange old lady who lives in Roswell, New Mexico and writes science fiction, fantasy and HORROR. Did I mention poetry? Die-Hard Jimmy Fallon #FalPal", "followers_count": 8770, "friends_count": 9613, "statues_count": 63973 }, "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-01-02T10:00:10.000Z"), "id": 683347064248496128, "text": "my head is underwater but I'm breathing fine.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 42149740, "name": "MARI", "screen_name": "maari29_", "lang": "es", "location": "RepDominicana/ Puerto Rico", "create_at": date("2009-05-23"), "description": "Yo soy Cristina Yang y ella es yo.", "followers_count": 538, "friends_count": 290, "statues_count": 65303 }, "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-01-02T10:00:10.000Z"), "id": 683347064353239040, "text": "District starts today and it's going to hurt not being able to compete out there with them but I know my boys ready", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1146845594, "name": "Jared Jenkins", "screen_name": "13Rashad", "lang": "en", "location": "Irving, TX", "create_at": date("2013-02-03"), "description": "#SWIE", "followers_count": 1090, "friends_count": 570, "statues_count": 18869 }, "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-02T10:00:10.000Z"), "id": 683347064374181888, "text": "@TreDigital @fscottmoody @terrycardiff pretty regularly - but a lot of what I do is promote our #tech #arts #startup ecosystem :)", "in_reply_to_status": 683342868094689284, "in_reply_to_user": 529794801, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "tech", "arts", "startup" }}, "user_mentions": {{ 529794801, 30852442, 2340300414 }}, "user": { "id": 14121456, "name": "ewhitmore", "screen_name": "ewhitmore", "lang": "en", "location": "ÜT: 35.100683,-106.661229", "create_at": date("2008-03-11"), "description": "Into #community-driven economic development & creative tech. Making #ABQ the friendliest place on the planet for #entrepreneurs! @1millioncupsABQ @sw_newmexico", "followers_count": 4129, "friends_count": 3849, "statues_count": 21139 }, "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-02T10:00:10.000Z"), "id": 683347064445624320, "text": "It's been a good year...\n\nDay 2, 2016, Easy Way, Lower Empire, Bobcat, Porcupine Pass & Bobcat… https://t.co/6AZmkSkJTT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.86165577,44.35361445"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21317149, "name": "Whiteface Mountain", "screen_name": "SkiWhiteface", "lang": "en", "location": "Wilmington, NY", "create_at": date("2009-02-19"), "description": "Welcome to Whiteface Lake Placid! Whiteface boasts the greatest vertical drop east of the Rockies and has been ranked #1 for offhill activities in ski Magazine", "followers_count": 8329, "friends_count": 826, "statues_count": 3001 }, "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": 36031, "countyName": "Essex", "cityID": 3682304, "cityName": "Wilmington" } }
+{ "create_at": datetime("2016-01-02T10:00:10.000Z"), "id": 683347064508399616, "text": "A real relationship 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": 2429627095, "name": "Han", "screen_name": "hannaah_lynn", "lang": "en", "location": "null", "create_at": date("2014-04-05"), "description": "null", "followers_count": 347, "friends_count": 252, "statues_count": 2749 }, "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-02T10:00:10.000Z"), "id": 683347064562958336, "text": "Nope. The content being streamed is 90% from others. Look at the top streamed shows out now. https://t.co/PmuMsLWyP7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2210038416, "name": "Tha Remaker", "screen_name": "CedricRemaker", "lang": "en", "location": "Houston, Texas", "create_at": date("2013-11-22"), "description": "A voracious reader, amateur poet, an enthusiastic lover of all genres of music, and a perpetual student of life.", "followers_count": 295, "friends_count": 255, "statues_count": 12057 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-02T10:00:10.000Z"), "id": 683347064915386369, "text": "@Chanel_MAC12 https://t.co/DbXKnd4MVn", "in_reply_to_status": -1, "in_reply_to_user": 231003357, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 231003357 }}, "user": { "id": 239260859, "name": "Sabrina", "screen_name": "PoseidonsMaiden", "lang": "en", "location": "Hufflepuff House", "create_at": date("2011-01-16"), "description": "Hammered Hannah . #NatsNation ⚾️❤️", "followers_count": 539, "friends_count": 356, "statues_count": 83639 }, "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-02T10:00:10.000Z"), "id": 683347064969822208, "text": "10/10 would recommend https://t.co/0dXNSnOuj0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 309351582, "name": "Cody Carroll", "screen_name": "cody_carroll", "lang": "en", "location": "Texas Tech University | BYX", "create_at": date("2011-06-01"), "description": "fight the good fight", "followers_count": 594, "friends_count": 279, "statues_count": 12685 }, "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-02T10:00:10.000Z"), "id": 683347065007554560, "text": "#Retail #Job alert: Retail Store Manager – KS – Overland Park (Antioch) | CVS Health | #OverlandPark, KS https://t.co/irv5WFwLYT #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.6707917,38.9822282"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "Job", "OverlandPark", "Jobs" }}, "user": { "id": 100240048, "name": "TMJ-KSK Retail Jobs", "screen_name": "tmj_ksk_retail", "lang": "en", "location": "Kansas City, KS", "create_at": date("2009-12-29"), "description": "Follow this account for geo-targeted Retail job tweets in Kansas City, KS from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 313, "friends_count": 293, "statues_count": 160 }, "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-02T10:00:10.000Z"), "id": 683347065242398720, "text": "Cloudy this afternoon, high 50 (10 C). Low 36 (2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712424, "name": "DallasWX", "screen_name": "DallasWX", "lang": "en", "location": "Dallas, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Dallas, Texas", "followers_count": 195, "friends_count": 93, "statues_count": 8151 }, "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-02T10:00:11.000Z"), "id": 683347065447972864, "text": "We're #hiring! Read about our latest #job opening here: Full-time Retail Associate (Outlet of New Port Richey) - https://t.co/0Fdrt9xNpb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.7192671,28.2441768"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job" }}, "user": { "id": 48348692, "name": "HSNCareers", "screen_name": "HSNCareers", "lang": "en", "location": "St. Petersburg, Florida", "create_at": date("2009-06-18"), "description": "Check out the GREAT CAREERS at HSN", "followers_count": 574, "friends_count": 77, "statues_count": 842 }, "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-02T10:00:11.000Z"), "id": 683347065552769025, "text": "Today’s daily concentrate special at 2nd Step Dispensary in Salem, Oregon is Dab Society! All Dab Society BHO Shatters and Waxes are just", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2750476154, "name": "Nature Quest LLC", "screen_name": "Nature_Questllc", "lang": "en", "location": "Salem, Oregon", "create_at": date("2014-08-20"), "description": "Nature Quest is a medical marijuana clinic and dispensary located in Salem, Oregon.", "followers_count": 219, "friends_count": 58, "statues_count": 2652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Four Corners, OR", "id": "8271f56611245451", "name": "Four Corners", "place_type": "city", "bounding_box": rectangle("-122.990493,44.909874 -122.955303,44.962335") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41047, "countyName": "Marion", "cityID": 4126750, "cityName": "Four Corners" } }
+{ "create_at": datetime("2016-01-02T10:00:11.000Z"), "id": 683347065615814656, "text": "@DjTheBasedKing boy", "in_reply_to_status": 683346927828602880, "in_reply_to_user": 322167500, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 322167500 }}, "user": { "id": 181866650, "name": "Quis Sumlin .", "screen_name": "TBQuis", "lang": "en", "location": "DeSoto/Bryan , TX", "create_at": date("2010-08-23"), "description": "Mega Part-Time Photographer . #TAMUC16 . RIP @Clyde_FadeAll .", "followers_count": 5990, "friends_count": 984, "statues_count": 396192 }, "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-02T10:00:11.000Z"), "id": 683347065850601474, "text": "Minnesota was fun had fun with 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": 2581528621, "name": "Alyssa jensen97", "screen_name": "alyssa_jensen17", "lang": "en", "location": "null", "create_at": date("2014-06-21"), "description": "whs senior❤️", "followers_count": 234, "friends_count": 1077, "statues_count": 1321 }, "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-01-02T10:00:11.000Z"), "id": 683347066005749762, "text": "https://t.co/p9JBcbazOH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1529390766, "name": "BPGOD✊", "screen_name": "handses", "lang": "en", "location": "null", "create_at": date("2013-06-18"), "description": "20 yrs old body count : 15.5 // you a mom? *dms you*", "followers_count": 1071, "friends_count": 898, "statues_count": 68482 }, "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-02T10:00:11.000Z"), "id": 683347066089783296, "text": "6-236 @ Monte Carlo Resort & Casino- Las Vegas https://t.co/OnKRam5dCN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.17907413,36.10487504"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 130684490, "name": "sewon ahn", "screen_name": "sewonist", "lang": "en", "location": "Teh internets", "create_at": date("2010-04-07"), "description": "null", "followers_count": 123, "friends_count": 104, "statues_count": 811 }, "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-02T10:00:11.000Z"), "id": 683347066127433728, "text": "@sneezywashere btw I like yo bio ��", "in_reply_to_status": 683346736157335552, "in_reply_to_user": 340421857, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 340421857 }}, "user": { "id": 719331485, "name": "Julian Alegria", "screen_name": "JulianAlegria7", "lang": "en", "location": "null", "create_at": date("2013-10-17"), "description": "17 years old | My life isn't perfect, it NEVER will be, I just know that my life is NOTHING without Jesus", "followers_count": 663, "friends_count": 454, "statues_count": 13060 }, "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-02T10:00:11.000Z"), "id": 683347066282708992, "text": "“For some laundry line believers we're just kids in many ways\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.8804097,37.4331738"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1201228344, "name": "Jesse Belt", "screen_name": "MrJesseBelt", "lang": "en", "location": "Lawrence, Kansas", "create_at": date("2013-02-20"), "description": "Cheerios, Space, and NPR.", "followers_count": 252, "friends_count": 409, "statues_count": 912 }, "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("2016-01-02T10:00:11.000Z"), "id": 683347066408579073, "text": "when you third wheel on your own Insta post https://t.co/jUXsCNJBIf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 577514014, "name": "liv", "screen_name": "livnelson17", "lang": "en", "location": "Spring Lake, MI", "create_at": date("2012-05-11"), "description": "michigangster", "followers_count": 264, "friends_count": 468, "statues_count": 2373 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spring Lake, MI", "id": "0168643f487321e1", "name": "Spring Lake", "place_type": "city", "bounding_box": rectangle("-86.213904,43.043093 -86.115054,43.117833") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26139, "countyName": "Ottawa", "cityID": 2675820, "cityName": "Spring Lake" } }
+{ "create_at": datetime("2016-01-02T10:00:11.000Z"), "id": 683347066450382849, "text": "We're #hiring! Read about our latest #job opening here: Sales Associate - https://t.co/LqWndYQrYT #LittleRock, AR #Retail", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.3415677,34.7501503"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "LittleRock", "Retail" }}, "user": { "id": 3248848340, "name": "Jobs at Jared", "screen_name": "JobsAtJared", "lang": "en", "location": "null", "create_at": date("2015-06-18"), "description": "Your Future Starts Here.", "followers_count": 14, "friends_count": 1, "statues_count": 504 }, "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-02T10:00:11.000Z"), "id": 683347066748334080, "text": "Damn man smh https://t.co/XrVkUkjhph", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 33315561, "name": "awesome", "screen_name": "__BJ_", "lang": "en", "location": "Frontatown Calliope NewOrleans", "create_at": date("2009-04-19"), "description": "just one cool nigga stick around long enough you will find out alot\r\n\n#teampissyopants \r\n#49ers #irish #braves #pacers", "followers_count": 2323, "friends_count": 2122, "statues_count": 331393 }, "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-02T10:00:11.000Z"), "id": 683347067159330816, "text": "Say it again for the jacks in the back. https://t.co/CjiT2EKP6M", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 357572344, "name": "e.", "screen_name": "ericaeverett26", "lang": "en", "location": "DET✈️[D]MV", "create_at": date("2011-08-18"), "description": "18. regina alum '15. Howard U '19.", "followers_count": 1586, "friends_count": 1388, "statues_count": 41436 }, "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-02T10:00:11.000Z"), "id": 683347067259977729, "text": "@Adriana_xo15 can I know who ur talking about", "in_reply_to_status": -1, "in_reply_to_user": 2862735943, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2862735943 }}, "user": { "id": 339392417, "name": "nic", "screen_name": "nicolemessinaa", "lang": "en", "location": "null", "create_at": date("2011-07-20"), "description": "ʙᴏᴏᴛʏ|nyc", "followers_count": 1453, "friends_count": 997, "statues_count": 10292 }, "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-02T10:00:11.000Z"), "id": 683347067348086785, "text": "The top most priority of #India in 2016 shd be solving #farmersSuicide problem. @_YogendraYadav to play the key role.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "India", "farmersSuicide" }}, "user_mentions": {{ 1317076194 }}, "user": { "id": 247373138, "name": "Addhyan", "screen_name": "addhyan_pandey", "lang": "en", "location": "Chicago, IL", "create_at": date("2011-02-04"), "description": "Football Data-Science Music R Start-up IndianPolitics theRealistNUFCfan", "followers_count": 230, "friends_count": 98, "statues_count": 1422 }, "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-02T10:00:11.000Z"), "id": 683347067419377666, "text": "#ibetyamouthbiggathanyapockets", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "ibetyamouthbiggathanyapockets" }}, "user": { "id": 383337327, "name": "rip mom", "screen_name": "selenabina_", "lang": "en", "location": "null", "create_at": date("2011-10-01"), "description": "null", "followers_count": 2421, "friends_count": 2224, "statues_count": 17428 }, "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-02T10:00:11.000Z"), "id": 683347067813654528, "text": "@BTS_twt Cute cute cute", "in_reply_to_status": 683300258554642433, "in_reply_to_user": 335141638, "favorite_count": 0, "retweet_count": 0, "lang": "ro", "is_retweet": false, "user_mentions": {{ 335141638 }}, "user": { "id": 3384126261, "name": "GetWellSoonRM❤️", "screen_name": "jungcock1997", "lang": "en", "location": "Kris's Weave", "create_at": date("2015-07-19"), "description": "#Getwellsoonnamjoon! #HandsomeRM all you haters need to evaporate @ Namjoon JIMIN GOT YES JAMS #Getwellsoontaegi", "followers_count": 58, "friends_count": 118, "statues_count": 1792 }, "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-02T10:00:11.000Z"), "id": 683347067922595840, "text": "@wclittle \nI've spent a lot of time reading about nutrition and fat storage/loss etc over the past few years, so this all makes sense to me!", "in_reply_to_status": 683329181179056129, "in_reply_to_user": 23640452, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23640452 }}, "user": { "id": 172650478, "name": "Isabella O.", "screen_name": "IsabellaOrgan", "lang": "en", "location": "Seattle", "create_at": date("2010-07-29"), "description": "Software Developer", "followers_count": 534, "friends_count": 403, "statues_count": 3707 }, "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-02T10:00:11.000Z"), "id": 683347068438462464, "text": "Production Worker - Kelly Services: (#SanMarcos, CA) https://t.co/useTY07uaR #Manufacturing #KellyJobs #KellyServices #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.1617685,33.1630271"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SanMarcos", "Manufacturing", "KellyJobs", "KellyServices", "Job", "Jobs", "Hiring" }}, "user": { "id": 843975835, "name": "CA Manufacturing", "screen_name": "tmj_ca_manuf", "lang": "en", "location": "California Non-Metro", "create_at": date("2012-09-24"), "description": "Follow this account for geo-targeted Manufacturing job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 331, "friends_count": 209, "statues_count": 339 }, "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-02T10:00:11.000Z"), "id": 683347068740603905, "text": "Nobs https://t.co/WRy7JVGohh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 612062382, "name": "tica ❣", "screen_name": "xxnauticax", "lang": "en", "location": "null", "create_at": date("2012-06-18"), "description": "❄❤", "followers_count": 1805, "friends_count": 959, "statues_count": 67448 }, "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-02T10:00:11.000Z"), "id": 683347069109678080, "text": "@tommycoyote_ i got work from 5-930��", "in_reply_to_status": 683346240633999360, "in_reply_to_user": 787880802, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 787880802 }}, "user": { "id": 290823494, "name": "♡ tina ♡", "screen_name": "CMellino", "lang": "en", "location": "free the nips", "create_at": date("2011-04-30"), "description": "g☯☯d woman \\\\ bad girl #buymepizza it's all about dat intersectional feminism ♡ #ECU18", "followers_count": 455, "friends_count": 296, "statues_count": 25262 }, "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.794651,35.767843") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3701520, "cityName": "Apex" } }
+{ "create_at": datetime("2016-01-02T10:00:11.000Z"), "id": 683347069126324224, "text": "Save it x Tory Lanez >", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 442945874, "name": "lil mona'", "screen_name": "imanimona_", "lang": "en", "location": "Arkansas, USA", "create_at": date("2011-12-21"), "description": "Glo'n up. Growin up. Goin up.\nRIPG23", "followers_count": 1888, "friends_count": 1634, "statues_count": 34931 }, "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-02T10:00:11.000Z"), "id": 683347069231235072, "text": "Rain this afternoon, high 46 (8 C). Low 40 (4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712472, "name": "AustinWX", "screen_name": "AustinWX", "lang": "en", "location": "Austin, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Austin, Texas", "followers_count": 1011, "friends_count": 1366, "statues_count": 8012 }, "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-02T10:00:11.000Z"), "id": 683347069357146112, "text": "@justinbieber hi", "in_reply_to_status": -1, "in_reply_to_user": 27260086, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 27260086 }}, "user": { "id": 3004526483, "name": "Mashel", "screen_name": "mashaalkhan259", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2015-01-29"), "description": "null", "followers_count": 4, "friends_count": 21, "statues_count": 51 }, "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-02T10:00:11.000Z"), "id": 683347069377974277, "text": "Want to work at Smokey Bones Bar and Fire Grill? We're #hiring in #Lithonia, GA! Click for details: https://t.co/FO6yXcx7Gm #Hospitality", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.0897182,33.6992043"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Lithonia", "Hospitality" }}, "user": { "id": 88113750, "name": "TMJ-GA HRTA Jobs", "screen_name": "tmj_ga_hrta", "lang": "en", "location": "Georgia", "create_at": date("2009-11-06"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Georgia Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 400, "friends_count": 288, "statues_count": 1386 }, "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": 13247, "countyName": "Rockdale" } }
+{ "create_at": datetime("2016-01-02T10:00:12.000Z"), "id": 683347069508059136, "text": "Can you recommend anyone for this #Hospitality #job? https://t.co/BnlbfdNQiC #Orlando, FL #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4905646,28.4514278"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Orlando", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 22536061, "name": "TMJ-ORL HRTA Jobs", "screen_name": "tmj_orl_hrta", "lang": "en", "location": "Orlando, FL", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Orlando, FL. Need help? Tweet us at @CareerArc!", "followers_count": 416, "friends_count": 291, "statues_count": 617 }, "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-02T10:00:12.000Z"), "id": 683347069566717952, "text": "@WATCHMANBILL no brother. .I get off in 1hr..hopefully. .IJN..#GodsLoveChats", "in_reply_to_status": 683305953220005888, "in_reply_to_user": 51746404, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GodsLoveChats" }}, "user_mentions": {{ 51746404 }}, "user": { "id": 555582336, "name": "joel torres jr", "screen_name": "tx_21_7", "lang": "en", "location": "null", "create_at": date("2012-04-16"), "description": "Soldier for GOD..Enemy of Satan and this Wicked World. .MMA STUDENT", "followers_count": 342, "friends_count": 992, "statues_count": 3427 }, "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-02T10:00:12.000Z"), "id": 683347069768196096, "text": "���� https://t.co/34i9qyRqU7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3390362781, "name": "kelly dellarocco", "screen_name": "kelldell22", "lang": "en", "location": "Connecticut, USA", "create_at": date("2015-07-23"), "description": "wrhs | CT", "followers_count": 122, "friends_count": 128, "statues_count": 151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Torrington, CT", "id": "01d25a2412697203", "name": "Torrington", "place_type": "city", "bounding_box": rectangle("-73.168971,41.76345 -73.058981,41.885906") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9005, "countyName": "Litchfield", "cityID": 976500, "cityName": "Torrington" } }
+{ "create_at": datetime("2016-01-02T10:00:12.000Z"), "id": 683347069809999872, "text": "This #Nursing #job might be a great fit for you: State Tested Nursing Assistant - $500 Sign-On Bonus - https://t.co/N1EZWlu20e #STNA #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4899136,41.5011199"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "STNA", "Hiring" }}, "user": { "id": 214122801, "name": "MenorahParkJobs", "screen_name": "MenorahParkJobs", "lang": "en", "location": "Beachwood, Ohio", "create_at": date("2010-11-10"), "description": "Menorah Park Center for Senior Living is one of the largest senior living communities in the state of Ohio.", "followers_count": 133, "friends_count": 82, "statues_count": 1380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beachwood, OH", "id": "fa5bd676191716ee", "name": "Beachwood", "place_type": "city", "bounding_box": rectangle("-81.532635,41.449669 -81.48037,41.501448") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3904500, "cityName": "Beachwood" } }
+{ "create_at": datetime("2016-01-02T10:00:12.000Z"), "id": 683347069822603265, "text": "Want to work at Love's Travel Stops & Country St...? We're #hiring in #Whiteland, IN! Click for details: https://t.co/hL5loI6rSi #Retail", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.0797079,39.5500485"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Whiteland", "Retail" }}, "user": { "id": 28542508, "name": "TMJ-IND Retail Jobs", "screen_name": "tmj_ind_retail", "lang": "en", "location": "Indianapolis, IN", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Indianapolis, IN. Need help? Tweet us at @CareerArc!", "followers_count": 384, "friends_count": 310, "statues_count": 847 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whiteland, IN", "id": "0fca31f1aec6d752", "name": "Whiteland", "place_type": "city", "bounding_box": rectangle("-86.104661,39.533945 -86.072599,39.564835") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18081, "countyName": "Johnson", "cityID": 1883816, "cityName": "Whiteland" } }
+{ "create_at": datetime("2016-01-02T10:00:12.000Z"), "id": 683347070376390656, "text": "I forgot about that settlement money. I wonder if any women lied on Bill Cosby for some money. Im sure they got broke off", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27275057, "name": "TheAbstract", "screen_name": "Abstract_cool", "lang": "en", "location": "Washington, DC", "create_at": date("2009-03-28"), "description": "Capricorn x DC Sports enthusiast x Fitness enthusiast x Lover of natural dark skin women x Capitalist slave x Also an X-men member", "followers_count": 2835, "friends_count": 1712, "statues_count": 269504 }, "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-01-02T10:00:12.000Z"), "id": 683347070422392836, "text": "Yes please ���� https://t.co/sqytNbBBb1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 416119344, "name": "Fashionista ♔", "screen_name": "TheOneYouNeed_x", "lang": "en", "location": "Kansas, USA", "create_at": date("2011-11-18"), "description": "l'm naturally funny because my life is a joke. Dream as big as you can dream...⭐️ •FASHION •MUSIC •ART", "followers_count": 1060, "friends_count": 772, "statues_count": 13410 }, "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-02T10:00:12.000Z"), "id": 683347070477033472, "text": "Barberton Oh Temp:29.6°F Wind:3 mph Dir:SW Baro:Falling Rain2day:0.00in Hum:73% UV:0.0 @ 13:00 01/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": 237, "friends_count": 228, "statues_count": 113621 }, "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-01-02T10:00:12.000Z"), "id": 683347070611095552, "text": "Daniel https://t.co/BHhiGfpCle", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user": { "id": 2359048100, "name": "Katline✨", "screen_name": "KaitlynLofton", "lang": "en", "location": "null", "create_at": date("2014-02-23"), "description": "@dcruz_82 over you ❤️", "followers_count": 507, "friends_count": 427, "statues_count": 5131 }, "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-02T10:00:12.000Z"), "id": 683347070816772098, "text": "Second time watching @BiggK757 vs @MATHHOFFA now!!����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 238043667, 23317405 }}, "user": { "id": 168940210, "name": "IG: @BugsChicago", "screen_name": "BugsChicago", "lang": "en", "location": "AND YOU SAY CHI-CITY", "create_at": date("2010-07-20"), "description": "BOXING ENTHUSIAST • CUB FAN BUD MAN • HIP HOP YA DON'T STOP • THE FORCE IS STRONG IN MY FAMILY• Periscope/Vine: BugsChicago", "followers_count": 405, "friends_count": 1959, "statues_count": 9569 }, "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-02T10:00:12.000Z"), "id": 683347070820851712, "text": "It's glittery on 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": 2464944565, "name": "BIRTHDAY GIRL", "screen_name": "Sams_Neverland", "lang": "en", "location": "Neverland✨", "create_at": date("2014-04-26"), "description": "~To die would be an awfully big adventure!✨ |Juliana||tjhs||c/o 2017||art||band||airforce|", "followers_count": 192, "friends_count": 481, "statues_count": 4121 }, "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-02T10:00:12.000Z"), "id": 683347071429029888, "text": "Throwback. https://t.co/mflFbhF1Qu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30230591, "name": "Tricia Seeley", "screen_name": "tricianicolee", "lang": "en", "location": "North Myrtle Beach, SC", "create_at": date("2009-04-10"), "description": "hopeless romantic. girl almighty.", "followers_count": 339, "friends_count": 391, "statues_count": 20178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Myrtle Beach, SC", "id": "e5797011cad97adf", "name": "North Myrtle Beach", "place_type": "city", "bounding_box": rectangle("-78.781654,33.787057 -78.596,33.861971") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4551280, "cityName": "North Myrtle Beach" } }
+{ "create_at": datetime("2016-01-02T10:00:12.000Z"), "id": 683347071450009601, "text": "I Apologize x @DomKennedyOPM ��Word For Word��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 146354237 }}, "user": { "id": 387113549, "name": "JayRoyality", "screen_name": "Unknown_ForNow", "lang": "en", "location": "null", "create_at": date("2011-10-08"), "description": "#Tamuc18 Commerce #Cliff SC:thepolishrifle Insta: Jay_Royalty_ ImFamousLikeTomBrady", "followers_count": 521, "friends_count": 647, "statues_count": 4188 }, "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-02T10:00:12.000Z"), "id": 683347071534022656, "text": "Some good ole #FriedChicken on this Saturday afternoon #SaturdayKitchen #Wings #Food https://t.co/NwhRkRFVti", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FriedChicken", "SaturdayKitchen", "Wings", "Food" }}, "user": { "id": 22370380, "name": "PMC aka PrimeMC", "screen_name": "PMCakaPrimeMC", "lang": "en", "location": "Raleigh, NC", "create_at": date("2009-03-01"), "description": "Hip Hop Artist / CEO of Tarhillz Entertainment! Knicks Fan till I die!!! Contact me @ pmcakaprimemc@yahoo.com & Send beats 2 beats4prime@gmail.com", "followers_count": 1472, "friends_count": 1301, "statues_count": 10522 }, "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-02T10:00:12.000Z"), "id": 683347071559012352, "text": ".@Webpass: 5 min install, 1ms ping, 750 down, 500 up ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 35861740 }}, "user": { "id": 1680541, "name": "Ben Drucker", "screen_name": "bendrucker", "lang": "en", "location": "San Francisco, CA", "create_at": date("2007-03-20"), "description": "Code, cameras, and coffee • JavaScript is my second language • Engineer at @Eaze_Team", "followers_count": 1172, "friends_count": 93, "statues_count": 5814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South of Market, San Francisco", "id": "1d019624e6b4dcff", "name": "South of Market", "place_type": "neighborhood", "bounding_box": rectangle("-122.418714,37.764094 -122.379692,37.789283") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-02T10:00:12.000Z"), "id": 683347071731122176, "text": "Oh I see..... Some people tweet to be assholes and hide behind their areas of expertise https://t.co/TD9wgTC11U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30870576, "name": "Diatonic Dissonance", "screen_name": "urbangaygriot", "lang": "en", "location": "In The Groove", "create_at": date("2009-04-13"), "description": "#NSFW But Lover of Music and People... Candid, Raunchy, and Intellectual. Professional Musician and Educator. LGBT and Social Service Advocate. #iCook", "followers_count": 6908, "friends_count": 1951, "statues_count": 204963 }, "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-02T10:00:12.000Z"), "id": 683347071768727552, "text": "Showers this afternoon, high 51 (11 C). Low 43 (6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712388, "name": "HoustonWX", "screen_name": "HoustonWX", "lang": "en", "location": "Houston, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Houston, Texas", "followers_count": 1817, "friends_count": 77, "statues_count": 8021 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-02T10:00:12.000Z"), "id": 683347071978479616, "text": "[Sat Jan 2nd, 12:00pm] Temperature: 68.36 F, 20.2 C; Humidity: 38.3%; Light: 23.54%. https://t.co/UI5qWnQ21d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 269430133, "name": "Shoemaker Sandbox", "screen_name": "shoe_sandbox", "lang": "en", "location": "Earth (for now). ", "create_at": date("2011-03-20"), "description": "I am a bot. Developer account for @shoemaker.", "followers_count": 7, "friends_count": 1, "statues_count": 24431 }, "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-02T10:00:12.000Z"), "id": 683347072083337220, "text": "Can you recommend anyone for this #IT #job? https://t.co/9hS2P1qYj8 #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": 99868456, "name": "TMJ-KYL IT Adm. Jobs", "screen_name": "tmj_KYL_adm", "lang": "en", "location": "Lexington, KY", "create_at": date("2009-12-27"), "description": "Follow this account for geo-targeted IT-Support/Administration job tweets in Lexington, KY from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 271, "friends_count": 265, "statues_count": 13 }, "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-02T10:00:12.000Z"), "id": 683347072091828225, "text": "First brunch of the year is with Intandem. (@ Corner Cafe & Bakery - @cornercafemad in New York, NY) https://t.co/Yxu2FznUdS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.95710904,40.78298807"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 262368898 }}, "user": { "id": 22041637, "name": "Walei", "screen_name": "CorruptedSight", "lang": "en", "location": "Astoria, New York", "create_at": date("2009-02-26"), "description": "Public servant, disability awareness and accessibility consultant, advocate, student, half-assed musician and video gamer", "followers_count": 157, "friends_count": 265, "statues_count": 1097 }, "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-02T10:00:12.000Z"), "id": 683347072511307782, "text": "Yo me saqué una tripleta�� https://t.co/4aDKhK0p5o", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2202054037, "name": "Luis Angel", "screen_name": "LuiggyD_", "lang": "en", "location": "Bronx NYC", "create_at": date("2013-11-18"), "description": "IG & SC kingluiggy", "followers_count": 316, "friends_count": 287, "statues_count": 37543 }, "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-02T10:00:12.000Z"), "id": 683347072574185472, "text": "I PRAISE YOU FOR SAYING THIS BC I SAID THE SAME THING https://t.co/mzHmcXPFqJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 304672954, "name": "als", "screen_name": "alazaymaldonado", "lang": "en", "location": "null", "create_at": date("2011-05-24"), "description": "we better live our lives up to the fullest", "followers_count": 872, "friends_count": 1984, "statues_count": 44478 }, "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-02T10:00:12.000Z"), "id": 683347073182384128, "text": "Why my beard look look like pubes yo ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 163268049, "name": "Champagne Baba", "screen_name": "_FerasSalehh", "lang": "en", "location": "Rest Easy Uncle", "create_at": date("2010-07-05"), "description": "Philadelphia // Boca Raton sc: fsaleh23", "followers_count": 628, "friends_count": 474, "statues_count": 18406 }, "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-02T10:00:12.000Z"), "id": 683347073509494784, "text": "@JLo I figure I hog tie you...slap the Teddy Bear on you...and well I'm driving Jen's old hot rod.", "in_reply_to_status": 683346876666646528, "in_reply_to_user": 2458671007, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 85603854 }}, "user": { "id": 2458671007, "name": "Son of David", "screen_name": "KingofKingONE", "lang": "en", "location": "null", "create_at": date("2014-04-22"), "description": "And Jesus answered him, The first of all the commandments is, Hear, O Israel; The Lord our God is one Lord:", "followers_count": 2269, "friends_count": 2451, "statues_count": 159258 }, "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-02T10:00:12.000Z"), "id": 683347073647939584, "text": "Wind 10.0 mph WNW. Barometer 29.814 in, Falling Rapidly. Temperature 33.3 °F. Rain today 0.00 in. Humidity 58%", "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": 3321 }, "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-02T10:00:13.000Z"), "id": 683347073706647552, "text": "#Repost @capecodinsta\n・・・\nWho wants to live in an area that provides a sunrise like this?! ☀️… https://t.co/k3z9pKaRri", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.06576374,42.3614161"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Repost" }}, "user_mentions": {{ 3350609765 }}, "user": { "id": 304032525, "name": "Cheryl Dwyer", "screen_name": "Real_Estate_MA", "lang": "en", "location": "South Shore, MA ", "create_at": date("2011-05-23"), "description": "My expertise is selling your home quickly & finding your dream home! \n\nAlso assist those who are underwater & need a short sale expert to get it done!", "followers_count": 320, "friends_count": 451, "statues_count": 991 }, "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-02T10:00:13.000Z"), "id": 683347073933160448, "text": "@presh_le @", "in_reply_to_status": 683346608138764290, "in_reply_to_user": 2793066109, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2793066109 }}, "user": { "id": 485219034, "name": "White Thugga ✊", "screen_name": "JayJayKing_", "lang": "en", "location": "19 • #RIPScottMoos", "create_at": date("2012-02-06"), "description": "welcome to Gangster 101", "followers_count": 1067, "friends_count": 520, "statues_count": 123159 }, "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-01-02T10:00:13.000Z"), "id": 683347073954000896, "text": "We're #hiring! Click to apply: Registered Nurse (RN)-CVICU - https://t.co/psqTodhVra #Nursing #cardiaccare #nurse #Lubbock, TX #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-101.8919288,33.5881366"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Nursing", "cardiaccare", "nurse", "Lubbock", "Job", "Jobs" }}, "user": { "id": 2940007148, "name": "UMC Nursing Jobs", "screen_name": "UMCNursingJobs", "lang": "en", "location": "null", "create_at": date("2014-12-22"), "description": "UMC Health System is the leader in comprehensive healthcare delivery in West Texas and Eastern New Mexico. Follow us to learn about our Nursing Jobs!", "followers_count": 91, "friends_count": 278, "statues_count": 116 }, "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-02T10:00:13.000Z"), "id": 683347074037972993, "text": "Balcony views. ☺️ https://t.co/7s4uyYAQIy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 85721521, "name": "Queen Whit", "screen_name": "German_Chocl8", "lang": "en", "location": "On a runway. ", "create_at": date("2009-10-27"), "description": "✨CEO of Embrace The Fame ✨\nGerman made.\nRunway Model | Aspiring Actress", "followers_count": 3708, "friends_count": 460, "statues_count": 123319 }, "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-02T10:00:13.000Z"), "id": 683347074348257280, "text": "Cloudy this afternoon, high 47 (8 C). Low 31 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575081, "name": "San Angelo Weather", "screen_name": "SanAngeloWX", "lang": "en", "location": "San Angelo, TX", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for San Angelo, TX", "followers_count": 1943, "friends_count": 92, "statues_count": 7911 }, "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-02T10:00:13.000Z"), "id": 683347074365177856, "text": "I want news from a journalist not a celebrity. https://t.co/AiZvyrYYqo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18126801, "name": "Jeff Jowdy", "screen_name": "jeffjowdy", "lang": "en", "location": "Athens, GA + Franklin, TN", "create_at": date("2008-12-14"), "description": "Nonprofit strategist, author, speaker, board member and fundraising professional. Passionate about nonprofit effectiveness and maximizing philanthropic support.", "followers_count": 5036, "friends_count": 5453, "statues_count": 25865 }, "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-02T10:00:13.000Z"), "id": 683347074490880004, "text": "OMG HAHAHAHHAH\n\n https://t.co/nVBc0lxGNL\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.53103016,32.95310833"), "retweet_count": 0, "lang": "nl", "is_retweet": false, "user": { "id": 2731929152, "name": "SmexyStyles", "screen_name": "juliad_15", "lang": "en", "location": "null", "create_at": date("2014-08-14"), "description": "❤'He gave me a Galaxy when I only expected a star'❤ Harry please follow me?❤", "followers_count": 155, "friends_count": 298, "statues_count": 2431 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Liberty Grove, TX", "id": "009916e42b074cec", "name": "Liberty Grove", "place_type": "city", "bounding_box": rectangle("-96.548727,32.938772 -96.520093,32.962834") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-02T10:00:13.000Z"), "id": 683347074620915712, "text": "�� https://t.co/h5B2hbjDUT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2174029523, "name": "Lebronze Allen", "screen_name": "purplekusha_", "lang": "en", "location": "null", "create_at": date("2013-11-08"), "description": "Free Ken @flluensy - insta", "followers_count": 302, "friends_count": 290, "statues_count": 2489 }, "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-02T10:00:13.000Z"), "id": 683347075136917505, "text": "To please everybody is fake. I couldnt say it any better", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 125435166, "name": "Jamal Gasol", "screen_name": "WhoIsJamalGasol", "lang": "en", "location": "716", "create_at": date("2010-03-22"), "description": "Artist / Songwriter Inquires:WhoIsJamalGasol@gmail.com #PIFF", "followers_count": 3099, "friends_count": 3486, "statues_count": 5148 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lockport, NY", "id": "45009687ba062971", "name": "Lockport", "place_type": "city", "bounding_box": rectangle("-78.744599,43.138056 -78.618744,43.205149") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36063, "countyName": "Niagara", "cityID": 3643082, "cityName": "Lockport" } }
+{ "create_at": datetime("2016-01-02T10:00:13.000Z"), "id": 683347075384258560, "text": "Quality Assurance Tester - NASCAR: (#Charlotte, NC) https://t.co/LhzmDdzFja #Marketing #QA #Agile #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.8431267,35.2270869"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Charlotte", "Marketing", "QA", "Agile", "Job", "Jobs", "Hiring" }}, "user": { "id": 2935068602, "name": "NASCAR Jobs", "screen_name": "NASCARJobs", "lang": "en", "location": "null", "create_at": date("2014-12-18"), "description": "The National Association for Stock Car Auto Racing, Inc. (NASCAR) is the sanctioning body for one of North America's premier sports.", "followers_count": 1073, "friends_count": 14, "statues_count": 159 }, "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-02T10:00:13.000Z"), "id": 683347075405185024, "text": "People let's stop letting stress take over. Like let's just chill, kick our feet up & bask in this glorious world we live in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2722672978, "name": "Brandon Smith Jr", "screen_name": "_Validated", "lang": "en", "location": "null", "create_at": date("2014-07-23"), "description": "Perspective matters.", "followers_count": 241, "friends_count": 272, "statues_count": 1771 }, "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-02T10:00:13.000Z"), "id": 683347075723997186, "text": "@lawrenjonae lmfao idkkk", "in_reply_to_status": 683346958602207233, "in_reply_to_user": 2795206942, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2795206942 }}, "user": { "id": 330045563, "name": "Lo", "screen_name": "xxxLovePink", "lang": "en", "location": "null", "create_at": date("2011-07-05"), "description": "DALLV$|SFA", "followers_count": 1154, "friends_count": 740, "statues_count": 32276 }, "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-02T10:00:13.000Z"), "id": 683347076017688576, "text": "I'm dying �� https://t.co/rGoggJ1PvO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 162229173, "name": "Ms. Turgeon ✨", "screen_name": "kayslt", "lang": "en", "location": "919", "create_at": date("2010-07-02"), "description": "Demand what you deserve. | #UNC18", "followers_count": 1716, "friends_count": 2037, "statues_count": 37065 }, "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-02T10:00:13.000Z"), "id": 683347076025958400, "text": "Football at 2 chisholm @ReginaldMacon6 @Traw_Norman9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2214188678, 2379643885 }}, "user": { "id": 1693756796, "name": "Fish", "screen_name": "Jerrelll10", "lang": "en", "location": "patt patt", "create_at": date("2013-08-23"), "description": "God, family,football #FreeRoy #RestEasyShanG #Saints", "followers_count": 976, "friends_count": 938, "statues_count": 31648 }, "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-02T10:00:13.000Z"), "id": 683347076885811200, "text": "@CechRepublic https://t.co/zwmJnl3rTJ", "in_reply_to_status": -1, "in_reply_to_user": 351303878, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 351303878 }}, "user": { "id": 779808722, "name": "burt macklin", "screen_name": "haleynoodle", "lang": "en", "location": "Oregon, USA", "create_at": date("2012-08-25"), "description": "you may not ride in my little red wagon", "followers_count": 347, "friends_count": 349, "statues_count": 3613 }, "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-02T10:00:13.000Z"), "id": 683347076890144768, "text": "#HolaSebastian tweeteame una selfie tuya de ahora! @SEBTSB x26", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "HolaSebastian" }}, "user_mentions": {{ 1631128196 }}, "user": { "id": 2621545545, "name": "sarah", "screen_name": "confusemendes", "lang": "en", "location": "New York, USA", "create_at": date("2014-06-18"), "description": "* ✧ · . ☾ *| njh | jw | sm |* ✧ ·. ☾ *", "followers_count": 2704, "friends_count": 2079, "statues_count": 22266 }, "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-01-02T10:00:13.000Z"), "id": 683347077078892545, "text": "GO TIGERS!!! #TigerWellFans https://t.co/mtCmahxydy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TigerWellFans" }}, "user": { "id": 3005420136, "name": "Smurfy", "screen_name": "smurfgrl8", "lang": "en", "location": "null", "create_at": date("2015-01-31"), "description": "I am and always will be The Optimist. The Hoper of far flung hopes. The Dreamer of Improbable dreams. GERONIMO!!!!", "followers_count": 8, "friends_count": 51, "statues_count": 52 }, "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-02T10:00:13.000Z"), "id": 683347077271662592, "text": "Partly cloudy this afternoon, high 53 (12 C). Low 43 (6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575147, "name": "Baton Rouge Weather", "screen_name": "BatonRougeWX", "lang": "en", "location": "Baton Rouge, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Baton Rouge, LA", "followers_count": 2872, "friends_count": 92, "statues_count": 7864 }, "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-02T10:00:13.000Z"), "id": 683347077632401408, "text": "on the road back to hell", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2367252776, "name": "hoepie", "screen_name": "hopeegutierrez", "lang": "en", "location": "null", "create_at": date("2014-03-01"), "description": "null", "followers_count": 1140, "friends_count": 2087, "statues_count": 4303 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lampasas, TX", "id": "4c0915681da6011d", "name": "Lampasas", "place_type": "city", "bounding_box": rectangle("-98.211885,31.042738 -98.160497,31.090413") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48281, "countyName": "Lampasas", "cityID": 4841188, "cityName": "Lampasas" } }
+{ "create_at": datetime("2016-01-02T10:00:14.000Z"), "id": 683347078114717696, "text": "I want a friend that's just down to go anywhere at any moment, no planning no hesitation. Someone who's goofy & and free of judgement", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2171297576, "name": "Belen Lopez", "screen_name": "bellennxoxo", "lang": "en", "location": "null", "create_at": date("2013-11-02"), "description": "your face, I like that shit", "followers_count": 62, "friends_count": 63, "statues_count": 284 }, "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-01-02T10:00:14.000Z"), "id": 683347078135726080, "text": "Shift Supervisor - Ryder: (#ATLANTA, GA) https://t.co/BrNeYP7glL #BusinessMgmt #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.39,33.76"), "retweet_count": 0, "lang": "da", "is_retweet": false, "hashtags": {{ "ATLANTA", "BusinessMgmt", "Job", "Jobs", "Hiring" }}, "user": { "id": 126380751, "name": "Ryder Jobs", "screen_name": "Ryderjobs", "lang": "en", "location": "null", "create_at": date("2010-03-25"), "description": "Ryder System, Inc. is a FORTUNE 500 provider of transportation and supply chain management solutions. View our job openings and Get All You Need to Succeed.", "followers_count": 753, "friends_count": 8, "statues_count": 5597 }, "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-02T10:00:14.000Z"), "id": 683347078177767425, "text": "@ktinusa wish we were neighbors", "in_reply_to_status": 683327176280641540, "in_reply_to_user": 2236172949, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2236172949 }}, "user": { "id": 609278045, "name": "scott north", "screen_name": "scottn332", "lang": "en", "location": "Midwest US", "create_at": date("2012-06-15"), "description": "I’m real and I hope some of my followers are too.", "followers_count": 191, "friends_count": 1277, "statues_count": 12 }, "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-01-02T10:00:14.000Z"), "id": 683347078991462401, "text": "@Lainemarvin don't be taking my word money", "in_reply_to_status": 683333789150502912, "in_reply_to_user": 1679591930, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1679591930 }}, "user": { "id": 337907515, "name": "Damon Salvatore", "screen_name": "keese_wheat1", "lang": "en", "location": "thugga !!!!! ", "create_at": date("2011-07-18"), "description": "kent state student marketing major⚡️⚡️ R.I.P #BrockStrong", "followers_count": 382, "friends_count": 83, "statues_count": 13059 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Canal Winchester, OH", "id": "c2ffda47cfe4cba5", "name": "Canal Winchester", "place_type": "city", "bounding_box": rectangle("-82.838543,39.8113 -82.777538,39.865041") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39045, "countyName": "Fairfield", "cityID": 3911332, "cityName": "Canal Winchester" } }
+{ "create_at": datetime("2016-01-02T10:00:14.000Z"), "id": 683347079075237888, "text": "When I can look Life in the eyes,Grown calm & very coldly wise,Life will HV given me the Truth,And taken in exchange - my Uth. #appleAday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.521638,41.818926"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "appleAday" }}, "user": { "id": 534998779, "name": "Denim Ben", "screen_name": "Me_DailyThought", "lang": "en", "location": "New York,USA", "create_at": date("2012-03-23"), "description": "#quote #Follow me to get selected famous quotes,Tweets which will inspire you in day to day life. Live, love, be happy, be Positive and enjoy life! with #noAds", "followers_count": 828, "friends_count": 1827, "statues_count": 5684 }, "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": 36105, "countyName": "Sullivan" } }
+{ "create_at": datetime("2016-01-02T10:00:14.000Z"), "id": 683347079339442176, "text": "This is exactly why @Scrillalcf is my bestfriend https://t.co/HWf07ZqRt8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 628618732 }}, "user": { "id": 720572724, "name": "Jan10th", "screen_name": "tianna_theBOSS", "lang": "en", "location": "null", "create_at": date("2012-07-27"), "description": "null", "followers_count": 1013, "friends_count": 1792, "statues_count": 14452 }, "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-01-02T10:00:14.000Z"), "id": 683347079343702017, "text": "Want to work at Ghirardelli? We're #hiring in #SanFrancisco, CA! Click for details: https://t.co/7GLVPxq3YO #BusinessMgmt #management #Job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.4194155,37.7749295"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "SanFrancisco", "BusinessMgmt", "management", "Job" }}, "user": { "id": 2228943546, "name": "Ghirardelli Careers", "screen_name": "ghirardellijobs", "lang": "en", "location": "null", "create_at": date("2013-12-03"), "description": "Ghirardelli invites you to explore our open #jobs and discover a place where your ideas come to life, as part of our delicious legacy.", "followers_count": 51, "friends_count": 1, "statues_count": 84 }, "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-02T10:00:14.000Z"), "id": 683347079687749633, "text": "Yes I'll be with you James Franco https://t.co/1AZmlNJQpP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2864503199, "name": "Ana✨", "screen_name": "_DeniseAna_", "lang": "en", "location": "ig: Deniseannac ⬅️", "create_at": date("2014-11-06"), "description": "you do what you want when you poppin'", "followers_count": 562, "friends_count": 520, "statues_count": 6113 }, "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-02T10:00:14.000Z"), "id": 683347079725486085, "text": "<Air Jordan IV Alternate 89> all of my preorders came through. If you missed out, I have 2 extra… https://t.co/fEFMIZsiPt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.62,33.4378"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34170216, "name": "David Wahab", "screen_name": "davidwahab", "lang": "en", "location": "New York, NY", "create_at": date("2009-04-21"), "description": "null", "followers_count": 53, "friends_count": 571, "statues_count": 225 }, "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-01-02T10:00:14.000Z"), "id": 683347079830212609, "text": "Cloudy this afternoon, high 54 (12 C). Low 47 (8 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575204, "name": "New Orleans Weather", "screen_name": "NewOrleansWX", "lang": "en", "location": "New Orleans, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for New Orleans, LA", "followers_count": 2899, "friends_count": 92, "statues_count": 7882 }, "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-02T10:00:14.000Z"), "id": 683347079951941632, "text": "All the bad bitches still go", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 712043723, "name": "Bankroll Fresh", "screen_name": "AlmightyRico_", "lang": "en", "location": "Yo Bitch Wet Dreams ", "create_at": date("2013-10-12"), "description": "Kill People , Burn Shit , Fuck School I'm A Fucking Unicorn , you see that nigga that looked swagged out ? THAT'S ME BITCH !", "followers_count": 3920, "friends_count": 3872, "statues_count": 22118 }, "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-02T10:00:14.000Z"), "id": 683347079968731136, "text": "Don't you, forget about me #lifetimecycle @ Life Time Fitness https://t.co/ExPFKFKRDc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.91842529,40.55179106"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "lifetimecycle" }}, "user": { "id": 18647401, "name": "Andy Lovell", "screen_name": "andylovell07", "lang": "en", "location": "Eagle Mountain, UT", "create_at": date("2009-01-05"), "description": "null", "followers_count": 188, "friends_count": 446, "statues_count": 4125 }, "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-02T10:00:14.000Z"), "id": 683347080383823872, "text": "The forecast for tonight is clear with a low of 20°F. #PetTravelAndSafetyDay #NationalPersonalTrainerAwarenessDay #NationalBuffetDay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.5333,41.5167"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PetTravelAndSafetyDay", "NationalPersonalTrainerAwarenessDay", "NationalBuffetDay" }}, "user": { "id": 42666285, "name": "Quad Cities Weather", "screen_name": "qcweather", "lang": "en", "location": "Quad Cities", "create_at": date("2009-05-26"), "description": "Is it going to be warm, wet or cold in the Quad Cities today? You can also find us on Facebook.", "followers_count": 740, "friends_count": 0, "statues_count": 11417 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rock Island Arsenal, IL", "id": "8f966c3da836a196", "name": "Rock Island Arsenal", "place_type": "city", "bounding_box": rectangle("-90.570223,41.509992 -90.511303,41.523635") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17161, "countyName": "Rock Island", "cityID": 1765104, "cityName": "Rock Island Arsenal" } }
+{ "create_at": datetime("2016-01-02T10:00:14.000Z"), "id": 683347080597749760, "text": "forgot about this https://t.co/nd7ta6TZE2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 349074529, "name": "montessr", "screen_name": "Supreme_Tay", "lang": "en", "location": "null", "create_at": date("2011-08-05"), "description": "19. coolin but will never freeze up", "followers_count": 7539, "friends_count": 2438, "statues_count": 204690 }, "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-02T10:00:14.000Z"), "id": 683347080874713088, "text": "Lol you wild, Wyd though?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2849102728, "name": "marie.", "screen_name": "xdmst", "lang": "en", "location": "CT", "create_at": date("2014-10-28"), "description": "IG: dmst.xo", "followers_count": 1550, "friends_count": 1316, "statues_count": 27173 }, "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-01-02T10:00:14.000Z"), "id": 683347081164140545, "text": "#PitBulls100 my boy rosco https://t.co/Nw49T9owHw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PitBulls100" }}, "user": { "id": 1941291038, "name": "Kerri Johnson", "screen_name": "Kerristj", "lang": "en", "location": "null", "create_at": date("2013-10-06"), "description": "#1Wife, Mother of 2.Univ. of MD Univ. College. Bachelors of Science in Criminal Justice with a Minor in Forensics. Associates Degree in Arts and Sciences", "followers_count": 101, "friends_count": 451, "statues_count": 794 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, MD", "id": "01ebc4b817a3c584", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-76.50289,38.416485 -76.454347,38.483135") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24009, "countyName": "Calvert", "cityID": 2448025, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-02T10:00:14.000Z"), "id": 683347081189306368, "text": "https://t.co/5NaFrv0wU9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 766272535, "name": "AjD", "screen_name": "_imalexd", "lang": "en", "location": "Tennessee, USA", "create_at": date("2012-08-18"), "description": "cape verdean NYG NYK R.I.P Dery L.$", "followers_count": 418, "friends_count": 248, "statues_count": 23908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pawtucket, RI", "id": "47623c4cf6ca7957", "name": "Pawtucket", "place_type": "city", "bounding_box": rectangle("-71.422117,41.856092 -71.333899,41.8984") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4454640, "cityName": "Pawtucket" } }
+{ "create_at": datetime("2016-01-02T10:00:14.000Z"), "id": 683347081516445696, "text": "She callin' me Fablo⁰She think that I'm Pablo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 263354911, "name": "Andrew", "screen_name": "ereyes6553", "lang": "en", "location": "864", "create_at": date("2011-03-09"), "description": "1-800-WHERE-YO-ASS-WAS-AT", "followers_count": 342, "friends_count": 422, "statues_count": 8239 }, "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-02T10:00:14.000Z"), "id": 683347081516478464, "text": "@CES Scheduling madness ..Do #work!", "in_reply_to_status": -1, "in_reply_to_user": 10668202, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "work" }}, "user_mentions": {{ 10668202 }}, "user": { "id": 345026255, "name": "Andrea Fagan", "screen_name": "akfagan", "lang": "en", "location": "New York", "create_at": date("2011-07-29"), "description": "lil bit NJ, LA, SD, NYC, Music Lover, Culinary Master, Fitness Freak, O+ and Living Authentically! blogger: http://kitchentakeaway.com", "followers_count": 175, "friends_count": 191, "statues_count": 2419 }, "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-02T10:00:14.000Z"), "id": 683347081868783617, "text": "Gracias miss u RT @Marrrhz: @fernyyy12 @jennysmallls happy birthday girls, enjoy ������", "in_reply_to_status": 683303263152689153, "in_reply_to_user": 69436773, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 69436773, 232365647, 159879657 }}, "user": { "id": 159879657, "name": "Jennifer", "screen_name": "jennysmallls", "lang": "en", "location": "Wade County", "create_at": date("2010-06-26"), "description": "null", "followers_count": 1016, "friends_count": 605, "statues_count": 36128 }, "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("2016-01-02T10:00:14.000Z"), "id": 683347082007064576, "text": "After losing all that weight my shoe size went from a 7/7.5 to a 6 how the hell was that possible ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 399792028, "name": "inaya", "screen_name": "InayaaAsmaa_", "lang": "en", "location": "Newark, CA", "create_at": date("2011-10-27"), "description": "sfsu", "followers_count": 179, "friends_count": 355, "statues_count": 18180 }, "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-02T10:00:15.000Z"), "id": 683347082107838464, "text": "Soooo are you supposed to ask for head or is the girl just supposed to already 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": 2724830492, "name": "ひCaptainSaveAHoe☄✨", "screen_name": "HoesLoveJoeDirt", "lang": "en", "location": "SAV912✈️Memphis901✈️SAV912", "create_at": date("2014-08-11"), "description": "#SouthernNotState #UglyGangCEO✨ SC: joedirty_22 RestEasyFatQuail", "followers_count": 1973, "friends_count": 2080, "statues_count": 116202 }, "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-02T10:00:15.000Z"), "id": 683347082187530240, "text": "when u actually do ur hair for once https://t.co/5863F9g7zb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 513931681, "name": "maddy", "screen_name": "maddyylake", "lang": "en", "location": "Tallahassee, FL", "create_at": date("2012-03-03"), "description": "supermodel/humanitarian/business mogul", "followers_count": 292, "friends_count": 281, "statues_count": 8279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pinellas Park, FL", "id": "d598cb5ad6f7a09f", "name": "Pinellas Park", "place_type": "city", "bounding_box": rectangle("-82.752428,27.824633 -82.663128,27.897396") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1256975, "cityName": "Pinellas Park" } }
+{ "create_at": datetime("2016-01-02T10:00:15.000Z"), "id": 683347082258853888, "text": "Every now and then I wish @Starbucks delivered ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 30973 }}, "user": { "id": 3246663641, "name": "Victoria S. Humphrey", "screen_name": "vshumphreyy", "lang": "en", "location": "null", "create_at": date("2015-05-11"), "description": "Miss Winter Park 2016 | Future MD | Founder of #Apples4Education | @univmiami alumna | @CollegeTourist Blogger | For we walk by faith, not by sight", "followers_count": 49, "friends_count": 96, "statues_count": 295 }, "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-02T10:00:15.000Z"), "id": 683347082321608709, "text": "The Refragmentation (awesome) https://t.co/txLZr2GSCJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 22518809, "name": "Matthew J. Cantieri", "screen_name": "mcantieri", "lang": "en", "location": "San Francisco Bay Area", "create_at": date("2009-03-02"), "description": "Son/brother/husband/dad in chronological order, aspiring polymath, BD @adallom (now @microsoft)", "followers_count": 356, "friends_count": 191, "statues_count": 8893 }, "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-02T10:00:15.000Z"), "id": 683347082460004352, "text": "Cloudy this afternoon, high 51 (11 C). Low 40 (4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575255, "name": "Shreveport Weather", "screen_name": "ShreveportWX", "lang": "en", "location": "Shreveport, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Shreveport, LA", "followers_count": 697, "friends_count": 92, "statues_count": 7940 }, "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-02T10:00:15.000Z"), "id": 683347082506272768, "text": "@sicherthanmost WHAT LOL", "in_reply_to_status": 683340591527292928, "in_reply_to_user": 2303801500, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2303801500 }}, "user": { "id": 186075397, "name": "❀ ℕoεℓℓε ℂαrdεn ❀", "screen_name": "yungcardi95", "lang": "en", "location": "Pennsylvania", "create_at": date("2010-09-02"), "description": "✞ Noelle Christine. 20. ♐ ✌ Floridian born and raised from the 727. ☀❤ Don't let anyone talk you out of your dreams. IG: noellechristineeee", "followers_count": 418, "friends_count": 444, "statues_count": 8860 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ancient Oaks, PA", "id": "df7eff07b0870f32", "name": "Ancient Oaks", "place_type": "city", "bounding_box": rectangle("-75.602004,40.521037 -75.568517,40.550309") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42077, "countyName": "Lehigh", "cityID": 4202416, "cityName": "Ancient Oaks" } }
+{ "create_at": datetime("2016-01-02T10:00:15.000Z"), "id": 683347082703454208, "text": "Not OU Twitter. Not even reading allat. But he admitted to giving Quaaludes. All I got to say https://t.co/DWPFTUodPT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34618846, "name": "no hoes 2016", "screen_name": "SaigeSmoove", "lang": "en", "location": "Michigan for now", "create_at": date("2009-04-23"), "description": "Let me escort you to my DMs #Futurehive #Yeezyhive [SC: t00faded]", "followers_count": 594, "friends_count": 527, "statues_count": 18273 }, "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-02T10:00:15.000Z"), "id": 683347082900541441, "text": "@CharissaRules happy new year... more blessings!!", "in_reply_to_status": -1, "in_reply_to_user": 3806929103, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3806929103 }}, "user": { "id": 950424894, "name": "Ugbede", "screen_name": "Iam_ugbede", "lang": "en", "location": "istanbul turkey", "create_at": date("2012-11-15"), "description": "God first... Iam_ugbede youu all know is flex... Just do youu in every thing youu do... #MUFC... http://www.instagram.com thisizugbede", "followers_count": 1205, "friends_count": 866, "statues_count": 5437 }, "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-02T10:00:15.000Z"), "id": 683347082946711553, "text": "more like bronze https://t.co/vNLgzfKNZt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1292468468, "name": "shortcake", "screen_name": "samsam_mcgee", "lang": "en", "location": "null", "create_at": date("2013-03-23"), "description": "love yourself || 10/03/15 cam weezy❤️", "followers_count": 961, "friends_count": 367, "statues_count": 12053 }, "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-02T10:00:15.000Z"), "id": 683347082963488768, "text": "@TRestaino this is hilarious https://t.co/9YiJ1rnMSH", "in_reply_to_status": -1, "in_reply_to_user": 141337832, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 141337832 }}, "user": { "id": 337953096, "name": "anthony j restaino", "screen_name": "tonytalkssports", "lang": "en", "location": "Western New York ", "create_at": date("2011-07-18"), "description": "Avid fan who loves talking about what's going on in the world of sports, especially Yanks, Browns & Leafs.", "followers_count": 361, "friends_count": 379, "statues_count": 44205 }, "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-01-02T10:00:15.000Z"), "id": 683347083571625984, "text": "@Verdett We're reuniting tomorrow for our best of 2015 episode. Should be fun.", "in_reply_to_status": 683344142064291840, "in_reply_to_user": 67441120, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 67441120 }}, "user": { "id": 2955865089, "name": "Wrong Reel", "screen_name": "WrongReel", "lang": "en", "location": "New York City", "create_at": date("2015-01-01"), "description": "Wrong Reel is a podcast for hardcore cinephiles, film fanatics and movie buffs of all kinds. Hosted by @pdixon_ @thekaradimov @colebrax.", "followers_count": 8690, "friends_count": 7147, "statues_count": 15744 }, "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-02T10:00:15.000Z"), "id": 683347083844268032, "text": "Fab! (@ West Side Steakhouse in New York, NY) https://t.co/xku7iHYnOo https://t.co/TAsRrOyRM6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9949199,40.76045985"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14819061, "name": "Jeff Wallace", "screen_name": "rocketman528", "lang": "en", "location": "Washington, D.C.", "create_at": date("2008-05-18"), "description": "CTO, athlete, author, chef: Saving world 1 project @ a time:-) Causes Célèbres - ESA, NASA, Smithsonian and White House. Extra in @KatyPerry's Hot n Cold!", "followers_count": 8550, "friends_count": 9363, "statues_count": 89044 }, "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-02T10:00:15.000Z"), "id": 683347083932209152, "text": "See our latest #job and click to apply: Barista (US) - https://t.co/H9Kcw0lcV4 #Hospitality #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.1890972,33.7728399"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Hospitality", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 22454941, "name": "TMJ-LAX HRTA Jobs", "screen_name": "tmj_lax_hrta", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-03-01"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Los Angeles, CA. Need help? Tweet us at @CareerArc!", "followers_count": 450, "friends_count": 295, "statues_count": 1390 }, "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-02T10:00:15.000Z"), "id": 683347084049821697, "text": "Dead ass. My dawg flow brazy https://t.co/A7huPXJzDE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 231959765, "name": "Jay.", "screen_name": "ItsJusLangston", "lang": "en", "location": "south florida☀️ JWU 19'", "create_at": date("2010-12-29"), "description": "I'm a product of the 90s", "followers_count": 1469, "friends_count": 1840, "statues_count": 41880 }, "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-02T10:00:15.000Z"), "id": 683347084104216577, "text": "@longlivekaybee i aint know we had it but yea i should", "in_reply_to_status": 683346941363634176, "in_reply_to_user": 3005267922, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3005267922 }}, "user": { "id": 1226494213, "name": "#LongLiveLaRose", "screen_name": "mocitytwest", "lang": "en", "location": "null", "create_at": date("2013-02-27"), "description": "MoCity,Tx 5|14 KayBeeGang 5|3 DexGang 7|7 BurlGang 7|20 LaRoseGang #FreeThaGuyz #FreeBook #FreeDerrick #FreeTyler #FreeBoe 11|18 BabyMcKenzie4Eva", "followers_count": 728, "friends_count": 1185, "statues_count": 20149 }, "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-01-02T10:00:15.000Z"), "id": 683347084397789184, "text": "Wind 2.0 mph SSE. Barometer 30.144 in, Steady. Temperature 41.3 °F. Rain today 0.00 in. Humidity 41%", "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": 191, "friends_count": 58, "statues_count": 239417 }, "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-02T10:00:15.000Z"), "id": 683347084410355712, "text": "Can you recommend anyone for this #Sales #job? https://t.co/ePoFUi5rs0 #Wyomissing, PA #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.9652117,40.329537"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "job", "Wyomissing", "Hiring" }}, "user": { "id": 2565844555, "name": "ExpressCareers", "screen_name": "expresscareers", "lang": "en", "location": "Columbus, OH", "create_at": date("2014-06-13"), "description": "Love everything fashion and the latest trends and styles? Why not mix business with pleasure - work at Express, we're hiring!", "followers_count": 262, "friends_count": 60, "statues_count": 856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wyomissing, PA", "id": "1e735f8fb045152d", "name": "Wyomissing", "place_type": "city", "bounding_box": rectangle("-75.988788,40.30576 -75.947615,40.35633") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42011, "countyName": "Berks", "cityID": 4286880, "cityName": "Wyomissing" } }
+{ "create_at": datetime("2016-01-02T10:00:15.000Z"), "id": 683347084435689472, "text": "Fr �� https://t.co/8HG46RiH19", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 554667950, "name": "Lauraaa❤️✨☄", "screen_name": "Ohhmylauraa", "lang": "en", "location": "305 to my City .", "create_at": date("2012-04-15"), "description": "null", "followers_count": 366, "friends_count": 234, "statues_count": 4935 }, "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-01-02T10:00:15.000Z"), "id": 683347084469088256, "text": "Kendrick Lamar https://t.co/pkLs642Yiq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 493739354, "name": "OluwaSmoke", "screen_name": "TopFlightSmokey", "lang": "en", "location": "null", "create_at": date("2012-02-15"), "description": "OG of BTN | Top Flight original | Co-Founder of FYN | Proud member of Go Live Ent. | #TAMUC18 | #futurehive | #CowboysNation | #ThrowUpTheX", "followers_count": 1563, "friends_count": 867, "statues_count": 51025 }, "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-02T10:00:15.000Z"), "id": 683347084607668224, "text": "Rt �� https://t.co/UEQJocC0YI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3989820939, "name": "Angelica", "screen_name": "angeliicaa_08", "lang": "en", "location": "Waterford, CT", "create_at": date("2015-10-18"), "description": "Brazilian ♠️", "followers_count": 530, "friends_count": 326, "statues_count": 3097 }, "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-01-02T10:00:15.000Z"), "id": 683347084645380096, "text": "@Brettiferrr LOL nope I'm a youngin!!", "in_reply_to_status": 683346927661002752, "in_reply_to_user": 2274056797, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2274056797 }}, "user": { "id": 269280725, "name": "amanda", "screen_name": "cxssiopeia", "lang": "en", "location": "charlotte", "create_at": date("2011-03-20"), "description": "constantly fighting the urge to dance in public", "followers_count": 379, "friends_count": 242, "statues_count": 40687 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laurinburg, NC", "id": "5746369a6d2df74c", "name": "Laurinburg", "place_type": "city", "bounding_box": rectangle("-79.54214,34.703163 -79.427342,34.805532") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37165, "countyName": "Scotland", "cityID": 3737220, "cityName": "Laurinburg" } }
+{ "create_at": datetime("2016-01-02T10:00:15.000Z"), "id": 683347085161181184, "text": "@Kamerynnkipilii happy birthday!! ❤️ I hope you have a good day ��", "in_reply_to_status": -1, "in_reply_to_user": 2872278884, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2872278884 }}, "user": { "id": 319468124, "name": "Nikkalena Caforio", "screen_name": "nikkicafz", "lang": "en", "location": "null", "create_at": date("2011-06-17"), "description": "I really like hotdogs", "followers_count": 1187, "friends_count": 754, "statues_count": 14277 }, "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-02T10:00:15.000Z"), "id": 683347085228249088, "text": "Stay hungry and surround yourself with people as hungry as you!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 597185344, "name": "JSal The Spectacular", "screen_name": "j_salazar95", "lang": "en", "location": "null", "create_at": date("2012-06-02"), "description": "FSMG artist | JSal | Omega Delta Pi Ralphy | We on the rise! Tribe/Squad | ΩΔΠR | http://soundcloud.com/jsal-the-spectacular", "followers_count": 628, "friends_count": 600, "statues_count": 16871 }, "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-02T10:00:15.000Z"), "id": 683347085635108866, "text": "Partly cloudy this afternoon, high 52 (11 C). Low 43 (6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575312, "name": "Lake Charles Weather", "screen_name": "LakeCharlesWX", "lang": "en", "location": "Lake Charles, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Lake Charles, LA", "followers_count": 540, "friends_count": 92, "statues_count": 7924 }, "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-02T10:00:15.000Z"), "id": 683347085769363456, "text": "Ready for the rodeo! �������� @RODEOHOUSTON", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 74486139 }}, "user": { "id": 357008051, "name": "Liz Ruiz⚡", "screen_name": "Lizz_Ruiz", "lang": "en", "location": "null", "create_at": date("2011-08-17"), "description": "HTX.", "followers_count": 235, "friends_count": 226, "statues_count": 4427 }, "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-01-02T10:00:15.000Z"), "id": 683347086130085888, "text": "@hoesay_what @BleacherReport @NickSwagyPYoung @adidasoriginals gah damn", "in_reply_to_status": 683335038935629825, "in_reply_to_user": 1947314761, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 1947314761, 890891, 2242915207, 20348431 }}, "user": { "id": 382831571, "name": "Tyler", "screen_name": "Tylerr_Hamilton", "lang": "en", "location": "null", "create_at": date("2011-09-30"), "description": "On the first day, Man created God.", "followers_count": 297, "friends_count": 286, "statues_count": 9722 }, "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-02T10:00:16.000Z"), "id": 683347086461374464, "text": "https://t.co/f5aZc3eRnp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 239541911, "name": "A", "screen_name": "mandileexx3", "lang": "en", "location": "null", "create_at": date("2011-01-17"), "description": "null", "followers_count": 529, "friends_count": 334, "statues_count": 37620 }, "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-02T10:00:16.000Z"), "id": 683347086969057280, "text": "@NMyZ0N3 going on a date today https://t.co/5Ip8mOkVLD", "in_reply_to_status": -1, "in_reply_to_user": 308770994, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 308770994 }}, "user": { "id": 44347742, "name": "Keese B", "screen_name": "lifesAlyric", "lang": "en", "location": "I'm somewhere in a song ...ATL", "create_at": date("2009-06-03"), "description": "I'm crazy, funny, Agnostic, fun, loving, caring, and not too hard on the eyes .", "followers_count": 2022, "friends_count": 941, "statues_count": 118129 }, "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-02T10:00:16.000Z"), "id": 683347087350628352, "text": "@deladante lose my number ��", "in_reply_to_status": 683346154369617921, "in_reply_to_user": 3085859854, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3085859854 }}, "user": { "id": 343746796, "name": "Chloe Hendricks", "screen_name": "ChloeHendricks", "lang": "en", "location": "850/251 ", "create_at": date("2011-07-27"), "description": "stuck in a nomadic point of madness", "followers_count": 2789, "friends_count": 292, "statues_count": 30098 }, "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-02T10:00:16.000Z"), "id": 683347087539318784, "text": "Game today against Shaler at home. Varsity-6 JV-7:30 ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2936270998, "name": "BHS Girls Basketball", "screen_name": "BHS_BBALL_", "lang": "en", "location": "null", "create_at": date("2014-12-21"), "description": "Baldwin Girls Basketball Team", "followers_count": 141, "friends_count": 139, "statues_count": 197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baldwin, PA", "id": "22412db95b3dc8b6", "name": "Baldwin", "place_type": "city", "bounding_box": rectangle("-80.010434,40.330924 -79.935608,40.411537") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4203928, "cityName": "Baldwin" } }
+{ "create_at": datetime("2016-01-02T10:00:16.000Z"), "id": 683347087854014464, "text": "so happy I started one tree hill", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1432713553, "name": "Hannah Sullivan", "screen_name": "hannnnerin", "lang": "en", "location": "845", "create_at": date("2013-05-16"), "description": "null", "followers_count": 367, "friends_count": 228, "statues_count": 2466 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fishkill, NY", "id": "00b4168b67ea75f8", "name": "Fishkill", "place_type": "city", "bounding_box": rectangle("-73.984359,41.474922 -73.870812,41.554628") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36027, "countyName": "Dutchess", "cityID": 3625967, "cityName": "Fishkill" } }
+{ "create_at": datetime("2016-01-02T10:00:16.000Z"), "id": 683347087971323904, "text": "@brokenhalo H8 felt like QT taking all of his movies & saying \"I am so monolithic now even I am derivative of myself\". Love his films.", "in_reply_to_status": 683342373875679232, "in_reply_to_user": 15770608, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15770608 }}, "user": { "id": 23566785, "name": "Michael Herman", "screen_name": "MichaelRHerman", "lang": "en", "location": "Minneapolis, MN", "create_at": date("2009-03-10"), "description": "null", "followers_count": 291, "friends_count": 130, "statues_count": 14383 }, "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-02T10:00:16.000Z"), "id": 683347088160116736, "text": "Partly cloudy this afternoon, high 52 (11 C). Low 44 (7 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575370, "name": "Lafayette Weather", "screen_name": "LafayetteWX", "lang": "en", "location": "Lafayette, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Lafayette, LA", "followers_count": 471, "friends_count": 92, "statues_count": 7963 }, "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-02T10:00:16.000Z"), "id": 683347088172650496, "text": "@Catale666 LMAOOOOO my bad why are u in there.", "in_reply_to_status": 683346916311080960, "in_reply_to_user": 1350410401, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1350410401 }}, "user": { "id": 1264545739, "name": "Steven Serious", "screen_name": "SupahhStroke93", "lang": "en", "location": "West Valley City, UT", "create_at": date("2013-03-13"), "description": "Hey is for horses. #AspiringBarber #1SeriousCut", "followers_count": 446, "friends_count": 386, "statues_count": 15345 }, "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-02T10:00:16.000Z"), "id": 683347088671928320, "text": "@Rare2793 I was so big there!", "in_reply_to_status": 683333441010675712, "in_reply_to_user": 1282182174, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1282182174 }}, "user": { "id": 280383600, "name": "Luis Ramirez ⚾️⚡️", "screen_name": "BigLu_", "lang": "en", "location": "Hollywood, FL", "create_at": date("2011-04-10"), "description": "Humble in the outside, Confident on the inside. @ATPBJJ #RMNU #ATPBJJ #BJJ", "followers_count": 297, "friends_count": 700, "statues_count": 14286 }, "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-02T10:00:16.000Z"), "id": 683347089045098497, "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": 98229521, "name": "mads", "screen_name": "MaddiMays", "lang": "en", "location": "sandy eggo CA", "create_at": date("2009-12-20"), "description": "stripper booty and a rack like wow", "followers_count": 403, "friends_count": 242, "statues_count": 33186 }, "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-02T10:00:16.000Z"), "id": 683347089158320128, "text": "We're #hiring! Click to apply: Assistant Manager - https://t.co/dHqlrSfKP1 #Hospitality #SONIC #GreenwellSprings, LA #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-91.026734,30.514003"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Hospitality", "SONIC", "GreenwellSprings", "Job", "Jobs", "CareerArc" }}, "user": { "id": 88181812, "name": "TMJ-LA HRTA Jobs", "screen_name": "tmj_la_hrta", "lang": "en", "location": "Louisiana", "create_at": date("2009-11-07"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Louisiana Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 372, "friends_count": 282, "statues_count": 1519 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Central, LA", "id": "01df2ed232dfaaf3", "name": "Central", "place_type": "city", "bounding_box": rectangle("-91.101943,30.486062 -90.967394,30.598818") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2213960, "cityName": "Central" } }
+{ "create_at": datetime("2016-01-02T10:00:16.000Z"), "id": 683347089556807681, "text": "@TSBible nabbed from @WelBeast", "in_reply_to_status": 683346477356322817, "in_reply_to_user": 435225922, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 435225922, 221377111 }}, "user": { "id": 389881156, "name": "Kyle", "screen_name": "ElzarifKyle", "lang": "en", "location": "null", "create_at": date("2011-10-12"), "description": "sure", "followers_count": 311, "friends_count": 492, "statues_count": 4264 }, "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-02T10:00:16.000Z"), "id": 683347089644994560, "text": "\"I love my campaign but I am rooting for Sen. @BarbaraBoxer!\" @CarlyFiorina circa 2010. #CarlyCurse https://t.co/OShThbUDJQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CarlyCurse" }}, "user_mentions": {{ 15939889, 65691824 }}, "user": { "id": 97273575, "name": "Scott Wooledge", "screen_name": "Clarknt67", "lang": "en", "location": "Brooklyn, Baby!", "create_at": date("2009-12-16"), "description": "The mind behind @memeographs. Topics: politics, TV, LGBT & human rights, often sarcastic. #Corvie's human assistant. Blocked by @RyanTAnd & @BryanJFischer!", "followers_count": 7961, "friends_count": 3787, "statues_count": 114670 }, "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-02T10:00:16.000Z"), "id": 683347090131451906, "text": "Want to work in #SantaCruz, CA? View our latest opening: https://t.co/0bQqD2UgiE #HR #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.9871217,37.0353977"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SantaCruz", "HR", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 23130194, "name": "San Jose, CA HR Jobs", "screen_name": "tmj_sjc_hr", "lang": "en", "location": "San Jose, CA", "create_at": date("2009-03-06"), "description": "Follow this account for geo-targeted Human Resources job tweets in San Jose, CA. Need help? Tweet us at @CareerArc!", "followers_count": 647, "friends_count": 296, "statues_count": 39 }, "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": 6087, "countyName": "Santa Cruz", "cityID": 670588, "cityName": "Scotts Valley" } }
+{ "create_at": datetime("2016-01-02T10:00:17.000Z"), "id": 683347090596970496, "text": "Happy Caturday, KoKo!\n\n#caturday #cat #catsofinstagram #catsofcatsonlyvethosp #philly #boarding… https://t.co/rsoss2E7mx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.33153281,40.1248999"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "caturday", "cat", "catsofinstagram", "catsofcatsonlyvethosp", "philly", "boarding" }}, "user": { "id": 2709293174, "name": "Cats Only Vet Hosp", "screen_name": "CatsOnlyVetHosp", "lang": "en", "location": "Norristown, PA", "create_at": date("2014-08-05"), "description": "Cat Hospital", "followers_count": 28, "friends_count": 11, "statues_count": 151 }, "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-02T10:00:17.000Z"), "id": 683347090722828288, "text": "Mostly sunny this afternoon, high 49 (9 C). Low 29 (-2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575409, "name": "Fort Smith Weather", "screen_name": "FortSmithWX", "lang": "en", "location": "Fort Smith, AR", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Fort Smith, AR", "followers_count": 442, "friends_count": 92, "statues_count": 8012 }, "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-02T10:00:17.000Z"), "id": 683347090768924672, "text": "This #Healthcare #job might be a great fit for you: Sleep Lab Tech Lic-Sleep Disorder... - https://t.co/17tH0w5a4d https://t.co/ncH1fGNYY8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.9131935,36.5787155"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job" }}, "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": 54, "friends_count": 27, "statues_count": 50 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monterey, CA", "id": "49af5b43d4963f4c", "name": "Monterey", "place_type": "city", "bounding_box": rectangle("-121.927024,36.573741 -121.830974,36.618826") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 648872, "cityName": "Monterey" } }
+{ "create_at": datetime("2016-01-02T10:00:17.000Z"), "id": 683347090777362434, "text": "h8 h8 h8 when Oomf takes forever to text back ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1180435002, "name": "Տíҽ", "screen_name": "SierraaCurtis", "lang": "en", "location": "null", "create_at": date("2013-02-14"), "description": "•I don't need a King to be a Queen. Karl is my bffl", "followers_count": 862, "friends_count": 706, "statues_count": 26035 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Michigan Center, MI", "id": "008d66995ceff3c1", "name": "Michigan Center", "place_type": "city", "bounding_box": rectangle("-84.365262,42.21073 -84.283863,42.27948") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26075, "countyName": "Jackson", "cityID": 2653580, "cityName": "Michigan Center" } }
+{ "create_at": datetime("2016-01-02T10:00:17.000Z"), "id": 683347091003813888, "text": "@VaTxn @Nupe117 @cspanwj https://t.co/q14a5gQj0J", "in_reply_to_status": 683346414689214464, "in_reply_to_user": 453265065, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 453265065, 275190051, 15923226 }}, "user": { "id": 36326139, "name": "Kevin", "screen_name": "Inked1BNA", "lang": "en", "location": "Brentwood Tn", "create_at": date("2009-04-29"), "description": "Tn Titans,UnitedBlue Philly Boy stuck in the deep south NFL, NHL, MLB, Music Edm, DeepHouse, Trance, House, Design, global Travel", "followers_count": 1273, "friends_count": 1958, "statues_count": 31895 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, TN", "id": "42835dec78de1327", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-86.869446,35.939893 -86.686525,36.05065") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47187, "countyName": "Williamson", "cityID": 4708280, "cityName": "Brentwood" } }
+{ "create_at": datetime("2016-01-02T10:00:17.000Z"), "id": 683347091058475009, "text": "@BTS_twt rest well hobi", "in_reply_to_status": 683300258554642433, "in_reply_to_user": 335141638, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 335141638 }}, "user": { "id": 3384126261, "name": "GetWellSoonRM❤️", "screen_name": "jungcock1997", "lang": "en", "location": "Kris's Weave", "create_at": date("2015-07-19"), "description": "#Getwellsoonnamjoon! #HandsomeRM all you haters need to evaporate @ Namjoon JIMIN GOT YES JAMS #Getwellsoontaegi", "followers_count": 58, "friends_count": 118, "statues_count": 1793 }, "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-02T10:00:17.000Z"), "id": 683347091184160770, "text": "But you're love's too good", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 243048524, "name": "George ✨", "screen_name": "george_slayy", "lang": "en", "location": "Houston/Co.Springs", "create_at": date("2011-01-25"), "description": "Gay af✨", "followers_count": 1402, "friends_count": 1288, "statues_count": 34685 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Four Corners, TX", "id": "cddb718d9306e2f9", "name": "Four Corners", "place_type": "city", "bounding_box": rectangle("-95.71547,29.64897 -95.643201,29.71046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4827102, "cityName": "Four Corners" } }
+{ "create_at": datetime("2016-01-02T10:00:17.000Z"), "id": 683347091280769025, "text": "I want to snuggle today", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 270751076, "name": "yassss queen", "screen_name": "varoc0", "lang": "en", "location": "null", "create_at": date("2011-03-22"), "description": "be peace itself ॐ नमः शिवाय,", "followers_count": 367, "friends_count": 707, "statues_count": 19751 }, "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-02T10:00:17.000Z"), "id": 683347091767230464, "text": "Only niggas who can have a house party that lasts til 4am", "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": "Johnny", "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 KDL", "followers_count": 2090, "friends_count": 2016, "statues_count": 47701 }, "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-02T10:00:17.000Z"), "id": 683347091792465920, "text": "Seven more followers to make 400!! Thank you everyone for your support https://t.co/UBz3msv9Cg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4615928596, "name": "Mr.Fred", "screen_name": "sweetlittlefred", "lang": "en", "location": "canbridge, MA", "create_at": date("2015-12-26"), "description": "Hi there! I am Fred and I am a very special little pup! I was rescued from a kill shelter in Texas and I now live in MA with my new family!", "followers_count": 393, "friends_count": 1903, "statues_count": 32 }, "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-01-02T10:00:17.000Z"), "id": 683347091805061120, "text": "@dddvnielysx a picture of somebody that can help", "in_reply_to_status": 683345936219672576, "in_reply_to_user": 416662530, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 416662530 }}, "user": { "id": 1423683108, "name": "PoloDeño", "screen_name": "thatniggacedeno", "lang": "en", "location": "Barol Bity ", "create_at": date("2013-05-12"), "description": "Don't come here looking for love #TML #GirlsWithCurls #RIPMarcus", "followers_count": 844, "friends_count": 368, "statues_count": 146532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Princeton, FL", "id": "12bb5154d46fecc0", "name": "Princeton", "place_type": "city", "bounding_box": rectangle("-80.428566,25.510691 -80.362933,25.559653") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1258975, "cityName": "Princeton" } }
+{ "create_at": datetime("2016-01-02T10:00:17.000Z"), "id": 683347091880554496, "text": "No one does it better than twp�� #SQUAAA @ 2nd Street https://t.co/UMNaWtWP4V", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.14917169,39.92702297"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SQUAAA" }}, "user": { "id": 574205412, "name": "Palak Patel", "screen_name": "palakypoo", "lang": "en", "location": "null", "create_at": date("2012-05-07"), "description": "ku'19 l ΑΣΤ⚓️", "followers_count": 949, "friends_count": 786, "statues_count": 31290 }, "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-02T10:00:17.000Z"), "id": 683347091922395136, "text": "Really missing my best friend and boyfriend rn @DylaneCalabro @Papa_Q96", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 748641278, 501319095 }}, "user": { "id": 1967813125, "name": "Nycole Ash", "screen_name": "NycoleMahomie", "lang": "en", "location": "Nebraska, USA", "create_at": date("2013-10-17"), "description": "Rest easy my angel 9.26.15 December Grad Taken by the best 12.20.15", "followers_count": 311, "friends_count": 987, "statues_count": 1870 }, "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-02T10:00:17.000Z"), "id": 683347092098662401, "text": "just need to move onnnnn man", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2868343208, "name": "Jezel✨", "screen_name": "Jezelll_", "lang": "en", "location": "null", "create_at": date("2014-10-20"), "description": "Norman Reedus// MHS volleyball", "followers_count": 329, "friends_count": 311, "statues_count": 2843 }, "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-02T10:00:17.000Z"), "id": 683347092157288448, "text": "@HorseShort 51% of US Muslims Want Sharia Law; 25% OK w/ Violence Against Americans►https://t.co/upJfB0tUAw #NoIslam https://t.co/ftSvJVZlOo", "in_reply_to_status": -1, "in_reply_to_user": 399745111, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NoIslam" }}, "user_mentions": {{ 399745111 }}, "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! Take care of American citizens only! No Islam & no foreigners!", "followers_count": 2177, "friends_count": 3116, "statues_count": 6828 }, "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-01-02T10:00:17.000Z"), "id": 683347092207759360, "text": "Verses crew member @floskee rocking \"Happy\" at #wednesdayverses! WE LIVE LOVE. :) #dc… https://t.co/dS5DZfmtuB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.02689574,38.91706152"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wednesdayverses", "dc" }}, "user_mentions": {{ 26002654 }}, "user": { "id": 24171344, "name": "Lorna Pinckney", "screen_name": "lpverses", "lang": "en", "location": "Richmond, Washington D.C., NY", "create_at": date("2009-03-13"), "description": "The host of Tue/Wed Verses: the open mic, & owner of UPSIDE Marketing, llc", "followers_count": 1460, "friends_count": 1625, "statues_count": 13444 }, "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-02T10:00:17.000Z"), "id": 683347092488744960, "text": "BFF/FYM https://t.co/lXM0E1GFVL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "sv", "is_retweet": false, "user": { "id": 18366565, "name": "Polaroid Papi", "screen_name": "iimrod", "lang": "en", "location": "216 NYC", "create_at": date("2008-12-24"), "description": "Visual Artist | #ThreeSeas IG: iimrod", "followers_count": 1105, "friends_count": 753, "statues_count": 28403 }, "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-02T10:00:17.000Z"), "id": 683347092606169088, "text": "stop this https://t.co/3yYsZSfkAK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2194988149, "name": "ash", "screen_name": "jvnghoseoks", "lang": "en", "location": "sylvia • 1/24/15", "create_at": date("2013-11-14"), "description": "I do think about erotic stuffs, but I mostly imagine myself being Iron Man.", "followers_count": 1633, "friends_count": 112, "statues_count": 109734 }, "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-02T10:00:17.000Z"), "id": 683347092618752000, "text": "GEORGIA ARE YOU KIDDING ME", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2921857594, "name": "meredith kirbo", "screen_name": "mermerkirbooo", "lang": "en", "location": "colquitt county", "create_at": date("2014-12-14"), "description": "Colquitt High School cheer. pretty happy with the life God gave me|R.I.P. Gav❤️", "followers_count": 347, "friends_count": 441, "statues_count": 1482 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moultrie, GA", "id": "224f34087a5a72d5", "name": "Moultrie", "place_type": "city", "bounding_box": rectangle("-83.835962,31.116348 -83.706539,31.224797") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13071, "countyName": "Colquitt", "cityID": 1353060, "cityName": "Moultrie" } }
+{ "create_at": datetime("2016-01-02T10:00:17.000Z"), "id": 683347093142908928, "text": "The most important thing is being kind to others. \n\nThat's it. 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": 24216455, "name": "Brittney✨", "screen_name": "honeybstew", "lang": "en", "location": "Trap City", "create_at": date("2009-03-13"), "description": "AAFA CPT. YOU'RE in charge of your own happiness...act like it!", "followers_count": 2504, "friends_count": 2007, "statues_count": 83238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-02T10:00:17.000Z"), "id": 683347093210017792, "text": "Partly cloudy this afternoon, high 52 (11 C). Low 34 (1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575444, "name": "Little Rock Weather", "screen_name": "LittleRockWX", "lang": "en", "location": "Little Rock, AR", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Little Rock, AR", "followers_count": 1836, "friends_count": 92, "statues_count": 7886 }, "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-02T10:00:17.000Z"), "id": 683347093268873216, "text": "My Grandad just told my dad , \" he's all bark and no bite \" �������� like what ? Who says that to their child ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 238666359, "name": "ChelseaMari Andrews", "screen_name": "_chelseamari", "lang": "en", "location": "Powder Springs, GA", "create_at": date("2011-01-15"), "description": "I Am The Princess❤️", "followers_count": 850, "friends_count": 606, "statues_count": 79143 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gresham Park, GA", "id": "2aacaaae6a17755c", "name": "Gresham Park", "place_type": "city", "bounding_box": rectangle("-84.330162,33.688117 -84.299422,33.724667") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1335240, "cityName": "Gresham Park" } }
+{ "create_at": datetime("2016-01-02T10:00:17.000Z"), "id": 683347093285658625, "text": "Slowly getting over it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 222702532, "name": "Thays Uchiha", "screen_name": "_Thathyzinha_", "lang": "en", "location": "Wonderland", "create_at": date("2010-12-03"), "description": "Daddy Issues", "followers_count": 1675, "friends_count": 690, "statues_count": 48822 }, "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-02T10:00:17.000Z"), "id": 683347093306617856, "text": "St Paul, MN: What a normal day, it's fair and 22ºF.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.17,44.93"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3195747619, "name": "Weather By Brian", "screen_name": "WeatherByBrian", "lang": "en", "location": "null", "create_at": date("2015-05-14"), "description": "Tweets the weather about wherever Brian last tweets.\n\nRun by @bman4789", "followers_count": 11, "friends_count": 1, "statues_count": 1589 }, "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.89076 -93.003514,44.992279") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27123, "countyName": "Ramsey", "cityID": 2758000, "cityName": "St. Paul" } }
+{ "create_at": datetime("2016-01-02T10:00:17.000Z"), "id": 683347093323444225, "text": "LIKE KOBE SHAQ DROSE AND BUTLER WE BE BALLIN https://t.co/4Om1JakNYg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2519744962, "name": "new year new zac", "screen_name": "thotfulzac", "lang": "en", "location": "konoha village", "create_at": date("2014-04-30"), "description": "NY ✈️ FL // 17 // Senior", "followers_count": 769, "friends_count": 715, "statues_count": 12803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Benson, NC", "id": "74e0013ce8c59d04", "name": "Benson", "place_type": "city", "bounding_box": rectangle("-78.559422,35.362207 -78.526611,35.404987") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37101, "countyName": "Johnston", "cityID": 3705040, "cityName": "Benson" } }
+{ "create_at": datetime("2016-01-02T10:00:17.000Z"), "id": 683347093340078080, "text": "@ellie_mueller thanks Ellie!!!! ���� love ya !!", "in_reply_to_status": 683346951962628096, "in_reply_to_user": 381288273, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 381288273 }}, "user": { "id": 2228577625, "name": "Cam♑", "screen_name": "CamreePearson2", "lang": "en", "location": "null", "create_at": date("2013-12-03"), "description": "My dance style ranges from a dad at a barbeque.. to a stripper whose rent is due.", "followers_count": 720, "friends_count": 622, "statues_count": 5808 }, "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-02T10:00:17.000Z"), "id": 683347093696700417, "text": "@ShakaWulu ��", "in_reply_to_status": 683347005800902656, "in_reply_to_user": 474714060, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 474714060 }}, "user": { "id": 2182822879, "name": "Kady", "screen_name": "kadycox_", "lang": "en", "location": "Michigan State University", "create_at": date("2013-11-08"), "description": "Michigan State Spartan. Feel the pressure or apply it. sc: kadyc ig: kady.baby", "followers_count": 2263, "friends_count": 736, "statues_count": 38005 }, "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-02T10:00:17.000Z"), "id": 683347093763706880, "text": "in need of a massage", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3545451494, "name": "jac jac", "screen_name": "lmaojaci", "lang": "en", "location": "tacompton", "create_at": date("2015-09-12"), "description": "4'11 / it's coo", "followers_count": 803, "friends_count": 753, "statues_count": 14671 }, "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-02T10:00:17.000Z"), "id": 683347093931601920, "text": "First session of the New Year! Big things coming for the SETS Team in 2016! #SETSbuilt… https://t.co/XrEf23YH5e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.286761,40.136357"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SETSbuilt" }}, "user": { "id": 1676573564, "name": "SETS", "screen_name": "ShoreElite", "lang": "en", "location": "Jackson, NJ", "create_at": date("2013-08-16"), "description": "A certified personal training & bootcamp company. INSTAGRAM: @ShoreElite", "followers_count": 328, "friends_count": 292, "statues_count": 558 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hyson, NJ", "id": "016727834eac6588", "name": "Hyson", "place_type": "city", "bounding_box": rectangle("-74.302459,40.113353 -74.241608,40.15691") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-01-02T10:00:17.000Z"), "id": 683347093998690304, "text": "#toptweetsof2015 https://t.co/dDz1IOXKil", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "toptweetsof2015" }}, "user": { "id": 358254990, "name": "iCEBERGSiMPSON", "screen_name": "CKlenk131", "lang": "en", "location": "NJ", "create_at": date("2011-08-19"), "description": "My presence is a present just to kick it is a blessing.", "followers_count": 423, "friends_count": 356, "statues_count": 34531 }, "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-01-02T10:00:17.000Z"), "id": 683347094006939649, "text": "@lane_shaw Happy Birthday ��", "in_reply_to_status": -1, "in_reply_to_user": 867190718, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 867190718 }}, "user": { "id": 2921089298, "name": "cass", "screen_name": "CassidyAllane", "lang": "en", "location": "null", "create_at": date("2014-12-06"), "description": "#40 ❤️ Psalm 73:26", "followers_count": 686, "friends_count": 555, "statues_count": 4727 }, "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-01-02T10:00:17.000Z"), "id": 683347094124498944, "text": "Up north has more hurricanes than Florida in the past couple years, try again. https://t.co/0ekOlch9Ez", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1546465909, "name": "Kat", "screen_name": "PINENIPPLESS", "lang": "en", "location": "South Florida", "create_at": date("2013-06-25"), "description": "Miami sports & wine fanatic", "followers_count": 4616, "friends_count": 597, "statues_count": 178355 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pine Manor, FL", "id": "bb3c878553134384", "name": "Pine Manor", "place_type": "city", "bounding_box": rectangle("-81.884278,26.567963 -81.871876,26.581244") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12071, "countyName": "Lee", "cityID": 1257025, "cityName": "Pine Manor" } }
+{ "create_at": datetime("2016-01-02T10:00:17.000Z"), "id": 683347094162268160, "text": "So weird change of events these next few days. Fins will be wearing their Aqua unis for an early afternoon home game & Hickey is out as GM.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 77588440, "name": "Armando Velez", "screen_name": "Mandoman12", "lang": "en", "location": "Miami, FL", "create_at": date("2009-09-26"), "description": "FIU '16. Management & Int'l Business Major. 22 years old. Musician. Golf Channel. A kid from Miami. #Dolphins #Marlins #Heat #FlaPanthers #OrlandoCitySC #Noles.", "followers_count": 423, "friends_count": 802, "statues_count": 43602 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ponce Inlet, FL", "id": "42e1363aa818ac7b", "name": "Ponce Inlet", "place_type": "city", "bounding_box": rectangle("-80.957717,29.075438 -80.919618,29.122109") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1258200, "cityName": "Ponce Inlet" } }
+{ "create_at": datetime("2016-01-02T10:00:17.000Z"), "id": 683347094283890688, "text": "Niggas gotta stop actin like they not inspired. I motivate you. Don't hate. Its ok to be motivated by 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": 172866544, "name": "TF", "screen_name": "Taefresh_", "lang": "en", "location": "Columbus, Ohio", "create_at": date("2010-07-30"), "description": "Instagram : @taefresh_ ( Taefreshmusic@gmail.com )", "followers_count": 3117, "friends_count": 360, "statues_count": 36473 }, "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-02T10:00:17.000Z"), "id": 683347094334214145, "text": "Wind 2.0 mph WSW. Barometer 30.028 in, Falling Rapidly. Temperature 40.1F. Rain today 0.00 in. Humidity 53% | 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": 580, "friends_count": 773, "statues_count": 35225 }, "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-02T10:00:17.000Z"), "id": 683347094510292992, "text": "We're playing @artfactorykc TONIGHT. See you at 7PM. https://t.co/ataxWT7dD9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3280063645 }}, "user": { "id": 2740503778, "name": "Tall Tales", "screen_name": "tall_tales_band", "lang": "en", "location": "Kansas City, MO", "create_at": date("2014-08-11"), "description": "IT'S JUST A GAME - EP AVAILABLE NOW. Shows at http://talltalesband.com", "followers_count": 642, "friends_count": 321, "statues_count": 427 }, "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-02T10:00:17.000Z"), "id": 683347094632067072, "text": "Nope, decided that's never gonna happen.. Lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 451142147, "name": "K A Y L I E ✨", "screen_name": "BakerTwin_1", "lang": "en", "location": "Mansfield, Ohio", "create_at": date("2011-12-30"), "description": "Madison Comprehensive '16 ⚽️ IG:kayliebakerrrr Snapchat:Baker.27", "followers_count": 1435, "friends_count": 1316, "statues_count": 12867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mansfield, OH", "id": "f136163002bd51f6", "name": "Mansfield", "place_type": "city", "bounding_box": rectangle("-82.572489,40.681009 -82.378653,40.815657") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39139, "countyName": "Richland", "cityID": 3947138, "cityName": "Mansfield" } }
+{ "create_at": datetime("2016-01-02T10:00:18.000Z"), "id": 683347094678188033, "text": "I'm 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": 2810544613, "name": "Ali Katato ❤️", "screen_name": "AKhalKat", "lang": "en", "location": "Detroit/Troy", "create_at": date("2014-09-14"), "description": "Love is life. Pre-medicine at WSU. King. Student.Skater.Romantic.Rapper.Poet. Sweetheart ❤ I love MT.\nhttps://www.youtube.com/user/AliKatato", "followers_count": 116, "friends_count": 124, "statues_count": 2569 }, "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-02T10:00:18.000Z"), "id": 683347094707507200, "text": "Sergio Busquets is like a fine exotic dish. He might not be what you would have, but you can’t deny it’s quality.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2400973298, "name": "Juan Arango", "screen_name": "JuanG_Arango", "lang": "en", "location": "Miami", "create_at": date("2014-03-20"), "description": "On TV, radio, internets and classroom. Pxp/color @MiamiHurricanes on @ESPN3, contributor @GazettaWorld @SPortsMax_Carib Earned. Not inherited. #BACANO", "followers_count": 28563, "friends_count": 5917, "statues_count": 41207 }, "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-02T10:00:18.000Z"), "id": 683347094783016960, "text": "Finally figured out how to get my hands on a pair of Yeezys: Become a rapper, befriend Kanye, sign with G.O.O.D. Music. Ye 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": 364006456, "name": "Jalen Courts", "screen_name": "AwwYeaCourts", "lang": "en", "location": "Raleigh, NC", "create_at": date("2011-08-28"), "description": "get it got it good", "followers_count": 844, "friends_count": 639, "statues_count": 9438 }, "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-02T10:00:18.000Z"), "id": 683347094804021248, "text": "1st #workout of #2016! #legday #letsgo #fueledbyherbalife @ LA Fitness https://t.co/cdATD5jVpr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.15649787,40.83929028"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "workout", "legday", "letsgo", "fueledbyherbalife" }}, "user": { "id": 153250480, "name": "Natasha Nicole", "screen_name": "Tasha_Nicole84", "lang": "en", "location": "Newark", "create_at": date("2010-06-07"), "description": "Herbalife Health Coach!! Health is a journey with endless benefits. Try it and see! Spelman Alumna.", "followers_count": 269, "friends_count": 518, "statues_count": 19667 }, "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-01-02T10:00:18.000Z"), "id": 683347095122657280, "text": "@devincf love it... just wish it was shot on film.", "in_reply_to_status": 683186218960470016, "in_reply_to_user": 15396479, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15396479 }}, "user": { "id": 15841045, "name": "DJ Kento", "screen_name": "DJKentoLive", "lang": "en", "location": "San Francisco, CA", "create_at": date("2008-08-13"), "description": "Your favorite Ex-Human DJ from the Bay Area.", "followers_count": 923, "friends_count": 1044, "statues_count": 29771 }, "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-01-02T10:00:18.000Z"), "id": 683347095428841472, "text": "Can you recommend anyone for this #Hospitality #job? https://t.co/RgeAN5jMHV #Pearland, TX #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.3676564,29.5568211"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Pearland", "Hiring" }}, "user": { "id": 1648169364, "name": "Raising Cane's Jobs", "screen_name": "CaniacCareers", "lang": "en", "location": "null", "create_at": date("2013-08-05"), "description": "null", "followers_count": 145, "friends_count": 40, "statues_count": 480 }, "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-02T10:00:18.000Z"), "id": 683347095772762112, "text": "Temp 42.6°F Rising, Pressure 30.291in Falling, Dew point 27.1°, Wind SW 4mph, Rain today 0.00in https://t.co/gPtnr2dDtV #STLwx #MOwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.35555556,38.47694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "STLwx", "MOwx" }}, "user": { "id": 1191304230, "name": "Oakville Weather", "screen_name": "OakvilleWx", "lang": "en", "location": "Oakville, Missouri, USA", "create_at": date("2013-02-17"), "description": "Current conditions on the hour brought to you by @DaveStLou", "followers_count": 101, "friends_count": 80, "statues_count": 24536 }, "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-01-02T10:00:18.000Z"), "id": 683347095835643904, "text": "#Hospitality alert: Banquet Cook... | Mandarin Oriental Hotel Las Vegas | #LasVegas, NV https://t.co/BOHyjnICzm https://t.co/G0EebST3Ev", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.1745805,36.1062236"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "Hospitality", "LasVegas" }}, "user": { "id": 3089847157, "name": "Mandarin Las Vegas", "screen_name": "MO_LVJobs", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-03-16"), "description": "Nevada's only Triple Forbes Five-Star luxury destination located on the world-famous #LasVegas Strip. Apply here to open #jobs.", "followers_count": 81, "friends_count": 59, "statues_count": 44 }, "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-02T10:00:18.000Z"), "id": 683347095877582848, "text": "@DivaMariee happy birfday. Big Booty. Mad Love over here. Enjoy it. ������", "in_reply_to_status": -1, "in_reply_to_user": 54810096, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 54810096 }}, "user": { "id": 248512311, "name": "..", "screen_name": "Og_LiP", "lang": "en", "location": "null", "create_at": date("2011-02-06"), "description": "don't get sent for ..", "followers_count": 1232, "friends_count": 891, "statues_count": 43278 }, "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-02T10:00:18.000Z"), "id": 683347095949017088, "text": "I make my bed everyday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 552434499, "name": "ᎯЅℋ", "screen_name": "InkGoldSkin", "lang": "en", "location": "Huntsville, AL", "create_at": date("2012-04-12"), "description": "my apologies toward anyone who's ever missed out on my love you lost a good thing. ✨", "followers_count": 1862, "friends_count": 1463, "statues_count": 19347 }, "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-02T10:00:18.000Z"), "id": 683347096326377472, "text": "Sunny this afternoon, high 45 (7 C). Low 25 (-4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26195145, "name": "Wichita Weather", "screen_name": "WichitaWeather", "lang": "en", "location": "Wichita, KS", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Wichita, KS", "followers_count": 1847, "friends_count": 88, "statues_count": 7741 }, "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-02T10:00:18.000Z"), "id": 683347097190404100, "text": "I wanna see scooter tonight !!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 601271711, "name": "Mrs.Way", "screen_name": "_Bossy_Flossy_", "lang": "en", "location": "Soulja Boys Houseeee ", "create_at": date("2012-06-06"), "description": "G L A M O R O U S✨\n\nDeAndre Cortez Way's Wife", "followers_count": 938, "friends_count": 1037, "statues_count": 24841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pascagoula, MS", "id": "b5d9160030d685ba", "name": "Pascagoula", "place_type": "city", "bounding_box": rectangle("-88.607587,30.3343 -88.500496,30.393797") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28059, "countyName": "Jackson", "cityID": 2855360, "cityName": "Pascagoula" } }
+{ "create_at": datetime("2016-01-02T10:00:18.000Z"), "id": 683347097219801088, "text": "pattyfergs found a good photo opp #detour @ Lake Louise, Alberta https://t.co/foFwSVTHZM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-116.183,51.4333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "detour" }}, "user": { "id": 862235071, "name": "Kiddle Kat", "screen_name": "KiddleKat56", "lang": "en", "location": "null", "create_at": date("2012-10-04"), "description": "Where does the family start? It starts with a young man falling in love with a girl–no superior alternative has yet been found. - Winston Churchill", "followers_count": 37, "friends_count": 58, "statues_count": 1263 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Improvement District No. 9 Banff, Alberta", "id": "2a71c43fd7a708ca", "name": "Improvement District No. 9 Banff", "place_type": "city", "bounding_box": rectangle("-117.320398,50.705371 -115.170174,52.270998") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-02T10:00:18.000Z"), "id": 683347097421262850, "text": "Wasup on 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": 625912893, "name": "#️⃣OsoStrapped", "screen_name": "BN4L_dblack", "lang": "en", "location": "#️⃣RedZone", "create_at": date("2012-07-03"), "description": "Gotta win can't lose do it for my nigga Gabe #RipGabeD #FreeJames #FreeWayne #FreeDevi it ain't nothing but ah set back", "followers_count": 2052, "friends_count": 462, "statues_count": 50549 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beecher, MI", "id": "8d5dfd9dd04ba8a7", "name": "Beecher", "place_type": "city", "bounding_box": rectangle("-83.763336,43.075092 -83.674217,43.133995") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2606820, "cityName": "Beecher" } }
+{ "create_at": datetime("2016-01-02T10:00:18.000Z"), "id": 683347097475760128, "text": "Happy Gameday WVU. Missin the fam������ #cactusbowl #hailwv @ Mountaineer Field https://t.co/LX8GttXBUG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.9588035,39.65120946"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "cactusbowl", "hailwv" }}, "user": { "id": 279220047, "name": "Lydia Alexander", "screen_name": "LilMissLydia", "lang": "en", "location": "Almost Heaven", "create_at": date("2011-04-08"), "description": "Do no harm but take no shit", "followers_count": 325, "friends_count": 447, "statues_count": 23808 }, "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-02T10:00:18.000Z"), "id": 683347097727447040, "text": "im pretty sure i got 50+ almond joys for Christmas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3541690696, "name": "o-dog", "screen_name": "oliviazara16", "lang": "en", "location": "null", "create_at": date("2015-09-03"), "description": "all good things take time", "followers_count": 173, "friends_count": 141, "statues_count": 1151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxford, MI", "id": "00c6aa3d045d7743", "name": "Oxford", "place_type": "city", "bounding_box": rectangle("-83.334197,42.795097 -83.215202,42.861512") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2662020, "cityName": "Oxford" } }
+{ "create_at": datetime("2016-01-02T10:00:18.000Z"), "id": 683347097739870208, "text": "Sales Associate - Sterling Jewelers: (#Cumberland, MD) https://t.co/uqWgBauVLL #Retail #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.8348529,39.6276489"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Cumberland", "Retail", "Job", "Jobs", "Hiring" }}, "user": { "id": 3248774184, "name": "Kay Jewelers", "screen_name": "JobsAtKay", "lang": "en", "location": "null", "create_at": date("2015-06-18"), "description": "Your Future Starts Here.", "followers_count": 5, "friends_count": 0, "statues_count": 1108 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Vale, MD", "id": "7e66033ba16f60db", "name": "La Vale", "place_type": "city", "bounding_box": rectangle("-78.845421,39.600233 -78.784937,39.691146") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24001, "countyName": "Allegany", "cityID": 2446075, "cityName": "La Vale" } }
+{ "create_at": datetime("2016-01-02T10:00:18.000Z"), "id": 683347097932967937, "text": "(NoPhonesGetLeftBehind) ain't nobody gone caught nobody slipping ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2540974146, "name": "March8_B2L", "screen_name": "DelvinEllis", "lang": "en", "location": "null", "create_at": date("2014-06-02"), "description": "IG @quez_built2last", "followers_count": 255, "friends_count": 203, "statues_count": 4133 }, "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-02T10:00:18.000Z"), "id": 683347097987448834, "text": "You've got to be kidding me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 478968679, "name": "David Heim", "screen_name": "davidheim12", "lang": "en", "location": "New Jersey", "create_at": date("2012-01-30"), "description": "Seton Hall '16 | @Setonian Executive Editor | Associate Producer- @SetonHallPTV's Hall Talk | Dream large. Live larger.", "followers_count": 287, "friends_count": 259, "statues_count": 10012 }, "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-02T10:00:18.000Z"), "id": 683347098062958592, "text": "It's just crazy how people just settle and don't want to do better than what they are doing . Upgrade your life don't 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": 277482673, "name": "#BallingLikeImCJAY", "screen_name": "Realist_PJay", "lang": "en", "location": "CleveLAND", "create_at": date("2011-04-05"), "description": "it's just Twitter", "followers_count": 1140, "friends_count": 1327, "statues_count": 17118 }, "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-02T10:00:18.000Z"), "id": 683347098226409472, "text": "#1jan2016@10km#overtonBaRk my toes got cold at least I #rode #trekCrokett5crossBikeCooL https://t.co/4iWWBmNhOL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.949131,35.129178"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "1jan2016", "rode", "trekCrokett5crossBikeCooL" }}, "user": { "id": 177298421, "name": "Mitchael", "screen_name": "xspindoc", "lang": "en", "location": "Planet Earth", "create_at": date("2010-08-11"), "description": "iG@outspokin * 7072001257 * flickr@outspokin * fb/md.childress *", "followers_count": 511, "friends_count": 2002, "statues_count": 12998 }, "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-02T10:00:18.000Z"), "id": 683347098247512064, "text": "everything is going to be ok this year. I may get betrayed, I may get hurt. as long as I'm in good health & progressing it's ok.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 619328351, "name": "lalaaa", "screen_name": "Kenja__", "lang": "en", "location": "null", "create_at": date("2012-06-26"), "description": "null", "followers_count": 3903, "friends_count": 3353, "statues_count": 42020 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rockledge, FL", "id": "ebc38e5de1c3a878", "name": "Rockledge", "place_type": "city", "bounding_box": rectangle("-80.763206,28.278095 -80.700557,28.349513") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12009, "countyName": "Brevard", "cityID": 1261500, "cityName": "Rockledge" } }
+{ "create_at": datetime("2016-01-02T10:00:18.000Z"), "id": 683347098532712449, "text": "Bib https://t.co/M22kAVHP8Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1940905470, "name": "Re〽️ainReal❗️", "screen_name": "RichTre_", "lang": "en", "location": "R.I.P Uncle R.I.P GRAND", "create_at": date("2013-10-06"), "description": "#FreeRob Follow me @____cvl", "followers_count": 2026, "friends_count": 1569, "statues_count": 69673 }, "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-02T10:00:18.000Z"), "id": 683347098549547013, "text": "Deep in da feels", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2214495299, "name": "Caleb harp", "screen_name": "calebharp24", "lang": "en", "location": "null", "create_at": date("2013-12-08"), "description": "RRHS | Cobalt ss", "followers_count": 216, "friends_count": 196, "statues_count": 109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roanoke Rapids, NC", "id": "22983f7058794d47", "name": "Roanoke Rapids", "place_type": "city", "bounding_box": rectangle("-77.695269,36.420078 -77.619507,36.481282") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37083, "countyName": "Halifax", "cityID": 3756900, "cityName": "Roanoke Rapids" } }
+{ "create_at": datetime("2016-01-02T10:00:18.000Z"), "id": 683347098691997696, "text": "@gabek you can buy a USB extension cable for a few bucks", "in_reply_to_status": 683323014541262848, "in_reply_to_user": 740913, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 740913 }}, "user": { "id": 257078220, "name": "Adam Caldwell", "screen_name": "JavaJoker", "lang": "en", "location": "Omaha, NE", "create_at": date("2011-02-24"), "description": "null", "followers_count": 31, "friends_count": 90, "statues_count": 150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chalco, NE", "id": "01d7ae2510775a65", "name": "Chalco", "place_type": "city", "bounding_box": rectangle("-96.196696,41.140119 -96.101091,41.19082") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31153, "countyName": "Sarpy", "cityID": 3108640, "cityName": "Chalco" } }
+{ "create_at": datetime("2016-01-02T10:00:18.000Z"), "id": 683347098712936450, "text": "@FeelDaRebirth23 touché", "in_reply_to_status": 683346123986042881, "in_reply_to_user": 1632053012, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user_mentions": {{ 1632053012 }}, "user": { "id": 498586047, "name": "ChandlerFromFriends", "screen_name": "Rchandler_11", "lang": "en", "location": "null", "create_at": date("2012-02-20"), "description": "When life throws you a curveball, sit back on it and hit a dinger⚾️", "followers_count": 661, "friends_count": 611, "statues_count": 5905 }, "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-01-02T10:00:18.000Z"), "id": 683347098834612225, "text": "still stuntin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 231290861, "name": "neely", "screen_name": "lilneelz_", "lang": "en", "location": "land of misfit toys", "create_at": date("2010-12-27"), "description": "swimming in the deep end", "followers_count": 921, "friends_count": 401, "statues_count": 24980 }, "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-02T10:00:18.000Z"), "id": 683347098834632705, "text": "@realestrozay yea I hate having fun too. Have a great life!", "in_reply_to_status": 683326575471898624, "in_reply_to_user": 161114383, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 161114383 }}, "user": { "id": 335713874, "name": "D-wight", "screen_name": "JAKE_DWIGHT", "lang": "en", "location": "Los Spurs", "create_at": date("2011-07-14"), "description": "Ole Miss", "followers_count": 599, "friends_count": 201, "statues_count": 4604 }, "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-02T10:00:19.000Z"), "id": 683347098876547072, "text": "Sunny this afternoon, high 33 (1 C). Low 14 (-10 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26199170, "name": "Omaha Weather", "screen_name": "OmahaWX", "lang": "en", "location": "Omaha, Nebraska", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Omaha, Nebraska", "followers_count": 2590, "friends_count": 88, "statues_count": 7734 }, "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-02T10:00:19.000Z"), "id": 683347098918469632, "text": "Make today a great day people", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26931547, "name": "Baby Spice", "screen_name": "abbzhoward", "lang": "en", "location": "null", "create_at": date("2009-03-26"), "description": "i like good vibes and good people and i like to dance.", "followers_count": 590, "friends_count": 1977, "statues_count": 9556 }, "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-02T10:00:19.000Z"), "id": 683347099069452288, "text": "You can't be upset for someone not trusting you when all you've told them were lies.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3272194322, "name": "feyah", "screen_name": "faithChewbacca", "lang": "en", "location": "null", "create_at": date("2015-07-08"), "description": "it's all about me", "followers_count": 241, "friends_count": 293, "statues_count": 6322 }, "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-01-02T10:00:19.000Z"), "id": 683347099107328000, "text": "Just bought ever ninja turtle thing target has for a 4 year old", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 245183706, "name": "Queen B", "screen_name": "BAlysee", "lang": "en", "location": "New York--Atlanta ", "create_at": date("2011-01-30"), "description": "Future MILF", "followers_count": 428, "friends_count": 285, "statues_count": 15443 }, "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-02T10:00:19.000Z"), "id": 683347099187019776, "text": "@MyDigitalEscap3 you don't have any Virginia dates on your tour, so i cant go :(", "in_reply_to_status": -1, "in_reply_to_user": 2873522944, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2873522944 }}, "user": { "id": 3002317822, "name": "megan", "screen_name": "damnmegann", "lang": "en", "location": "0/4 :(", "create_at": date("2015-01-27"), "description": "nothing like the rain when you're in outer space.. || bassist || ig: @damnmegan ||", "followers_count": 725, "friends_count": 855, "statues_count": 3548 }, "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-02T10:00:19.000Z"), "id": 683347099199483904, "text": "partly cloudy -> mostly cloudy\ntemperature up 12°F -> 14°F\nhumidity down 76% -> 73%\nwind 6mph -> 3mph\npressure 30.26in rising", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.06176,37.67795"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 132361176, "name": "Cedar City Weather", "screen_name": "CedarCityUT", "lang": "en", "location": "Cedar City, UT", "create_at": date("2010-04-12"), "description": "Weather updates, forecast, warnings and information for Cedar City, UT. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 105, "friends_count": 137, "statues_count": 25838 }, "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-01-02T10:00:19.000Z"), "id": 683347099379974144, "text": "@Komps4 @theprasiddha @footballitalia you fucking loser stop. City are shit and will die once Arab money does.", "in_reply_to_status": 683346831875661829, "in_reply_to_user": 333132360, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 333132360, 376230582, 18841928 }}, "user": { "id": 2460552407, "name": "Salvatore Finazzo", "screen_name": "JuvCalcio", "lang": "en", "location": "NJ----Carini", "create_at": date("2014-04-04"), "description": "Amunì, Ⓢⓘⓒⓘⓛⓘⓐⓝⓞ, Dai Dai Dai.", "followers_count": 429, "friends_count": 506, "statues_count": 4141 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bradley Gardens, NJ", "id": "01fd51395038e02e", "name": "Bradley Gardens", "place_type": "city", "bounding_box": rectangle("-74.687875,40.547704 -74.647892,40.599992") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34035, "countyName": "Somerset", "cityID": 3407000, "cityName": "Bradley Gardens" } }
+{ "create_at": datetime("2016-01-02T10:00:19.000Z"), "id": 683347099434352640, "text": "Bro it's only Saturday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 479140817, "name": "Nawfside Shawty ⛵️", "screen_name": "KingJourdain", "lang": "en", "location": "Monroe, LA", "create_at": date("2012-01-30"), "description": "19. God first | #WhoDatNation | college student | snapchat: jourdain96", "followers_count": 3883, "friends_count": 1982, "statues_count": 62720 }, "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-02T10:00:19.000Z"), "id": 683347099438694404, "text": "#HolaSebastian tweeteame una selfie tuya de ahora! @SEBTSB x27", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "HolaSebastian" }}, "user_mentions": {{ 1631128196 }}, "user": { "id": 2621545545, "name": "sarah", "screen_name": "confusemendes", "lang": "en", "location": "New York, USA", "create_at": date("2014-06-18"), "description": "* ✧ · . ☾ *| njh | jw | sm |* ✧ ·. ☾ *", "followers_count": 2704, "friends_count": 2079, "statues_count": 22267 }, "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-01-02T10:00:19.000Z"), "id": 683347099698769921, "text": "@flanelnotchanel no problem at all honey. Hope you have a wonderful day", "in_reply_to_status": 683345791868604416, "in_reply_to_user": 537562021, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 537562021 }}, "user": { "id": 3472838241, "name": "Writergirl1034", "screen_name": "Writergirl10341", "lang": "en", "location": "Florence, SC", "create_at": date("2015-08-28"), "description": "null", "followers_count": 655, "friends_count": 1046, "statues_count": 13705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florence, SC", "id": "d49251f0d0a2652d", "name": "Florence", "place_type": "city", "bounding_box": rectangle("-79.918523,34.110516 -79.642093,34.254807") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45041, "countyName": "Florence", "cityID": 4525810, "cityName": "Florence" } }
+{ "create_at": datetime("2016-01-02T10:00:19.000Z"), "id": 683347099769946112, "text": "I am looking for a buyer on 7846 Golf Estates Drive #PequotLakes #MN https://t.co/sKHsmbGYR5 #realestate https://t.co/Lq8J07BAc4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.238986,46.653956"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PequotLakes", "MN", "realestate" }}, "user": { "id": 214631153, "name": "Sean M. McDonald", "screen_name": "realtorSeanMac", "lang": "en", "location": "Longville, MN", "create_at": date("2010-11-11"), "description": "25 years of full time real estate service, moved to my long time favorite area in the world, Longville MN., and enjoying life!", "followers_count": 50, "friends_count": 51, "statues_count": 1575 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minnesota, USA", "id": "9807c5c5f7a2c6ce", "name": "Minnesota", "place_type": "admin", "bounding_box": rectangle("-97.239256,43.499362 -89.483385,49.384359") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27035, "countyName": "Crow Wing" } }
+{ "create_at": datetime("2016-01-02T10:00:19.000Z"), "id": 683347099820273665, "text": "Why the fuck do I have like 8 things not graded. This must be how teachers feel when students don't do assignments https://t.co/bmxBpuBnUY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2222330094, "name": "Jordan's Room", "screen_name": "Jordan_Lee310", "lang": "en", "location": "null", "create_at": date("2013-11-29"), "description": "Baby girl, look both ways before your cross my mind.", "followers_count": 199, "friends_count": 193, "statues_count": 2818 }, "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-02T10:00:19.000Z"), "id": 683347099900088320, "text": "Major L https://t.co/yZ1x3y9yxF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 458994289, "name": "CurveDaddy", "screen_name": "DannyPrice23", "lang": "en", "location": "Amherst, NY", "create_at": date("2012-01-08"), "description": "18yr|Georgia Forever Colga | Bishop Timon'16 http://www.hudl.com/athlete/3073553/highlights", "followers_count": 1935, "friends_count": 768, "statues_count": 60508 }, "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-02T10:00:19.000Z"), "id": 683347100004921345, "text": "�������� https://t.co/k68jB5HUYB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 46982289, "name": "✨KingArthur✨", "screen_name": "_kingarthur__", "lang": "en", "location": "〽️-Town ", "create_at": date("2009-06-13"), "description": "Loyalty + Business = Success. Tenn State. Snapchat: sirkingarthur", "followers_count": 828, "friends_count": 633, "statues_count": 16910 }, "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-02T10:00:19.000Z"), "id": 683347100021702656, "text": "I'm so tired and cranky.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15042348, "name": "million dollar man", "screen_name": "daddychasey", "lang": "en", "location": "null", "create_at": date("2008-06-07"), "description": "it's just me and my magic pussy.", "followers_count": 464, "friends_count": 503, "statues_count": 33851 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beavercreek, OH", "id": "050c276f7bcbb37e", "name": "Beavercreek", "place_type": "city", "bounding_box": rectangle("-84.106764,39.676687 -83.988342,39.779185") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39057, "countyName": "Greene", "cityID": 3904720, "cityName": "Beavercreek" } }
+{ "create_at": datetime("2016-01-02T10:00:19.000Z"), "id": 683347100076146689, "text": "Run faster!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 39745267, "name": "Michael Dendor", "screen_name": "MichaelDendor", "lang": "en", "location": "Streamwood, IL", "create_at": date("2009-05-13"), "description": "Die Hard Seminole fan. I also love the Bulls, Bears, and White Sox. Very competitive fantasy sports player and gambler. I love you Jessica.", "followers_count": 369, "friends_count": 1696, "statues_count": 22449 }, "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-01-02T10:00:19.000Z"), "id": 683347100155777025, "text": "https://t.co/OWtyARq905", "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": 1322, "friends_count": 1374, "statues_count": 20683 }, "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-02T10:00:19.000Z"), "id": 683347100311117825, "text": "God loves you in spite of everything you're using as an excuse not to surrender to Him.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 445617036, "name": "Mike Check", "screen_name": "MC11SIX", "lang": "en", "location": "Bowling Green, KY", "create_at": date("2011-12-24"), "description": "Believer. Husband. Father. Lead Vocals for @OneRenown. Avid TV/Film watcher and sports aficionado. Proud #BBN & #11SIX clique member.", "followers_count": 251, "friends_count": 475, "statues_count": 7058 }, "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-02T10:00:19.000Z"), "id": 683347100571045893, "text": "@AlseidOreades @ArsVampyre @MargaretsBelly To each his own. I suspect its how the SJW-Lites or whatnot got in and swarming around TBH.", "in_reply_to_status": 683346441209778177, "in_reply_to_user": 248485957, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 248485957, 33540385, 2705002822 }}, "user": { "id": 2992643773, "name": "Fox Gibson", "screen_name": "SpartacusFoxGib", "lang": "en", "location": "United States. Mostly.", "create_at": date("2015-01-22"), "description": "I was manufatcured sometime. Looks Like I'm Going to Mars. LOL.", "followers_count": 1742, "friends_count": 1538, "statues_count": 88220 }, "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-02T10:00:19.000Z"), "id": 683347100579450882, "text": "@mreynoso4380", "in_reply_to_status": 679017105996431361, "in_reply_to_user": 2539728647, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2539728647 }}, "user": { "id": 2539728647, "name": "Manuel Reynoso", "screen_name": "mreynoso4380", "lang": "en", "location": "null", "create_at": date("2014-05-10"), "description": "null", "followers_count": 19, "friends_count": 104, "statues_count": 13 }, "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-01-02T10:00:19.000Z"), "id": 683347100822716416, "text": "#Rockville, MD #Nursing #Job: Geriatric Nursing Assistants (Long-term care GNA) at Hebrew Home https://t.co/AF4lkJPKMG #GNA #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.123972,39.0533434"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Rockville", "Nursing", "Job", "GNA", "Jobs", "Hiring" }}, "user": { "id": 2532895171, "name": "CESLC Careers", "screen_name": "CESLCCareers", "lang": "en", "location": "Rockville, MD", "create_at": date("2014-05-29"), "description": "At CESLC we are committed to providing outstanding care for our seniors. Come see why over 200 staff members have been with us for more than 10 years!", "followers_count": 32, "friends_count": 3, "statues_count": 47 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Bethesda, MD", "id": "d3c4d514e13810f3", "name": "North Bethesda", "place_type": "city", "bounding_box": rectangle("-77.155886,39.012634 -77.084619,39.074304") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2456337, "cityName": "North Bethesda" } }
+{ "create_at": datetime("2016-01-02T10:00:19.000Z"), "id": 683347101393268736, "text": "So far, looks like 2016 will go down as the Year of Abruptly Deciding to Go Home Immediately.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 50754545, "name": "Melissa Sorcic", "screen_name": "explodingknife", "lang": "en", "location": "Oshkosh, WI", "create_at": date("2009-06-25"), "description": "Topics. Love 'em.", "followers_count": 155, "friends_count": 728, "statues_count": 607 }, "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-01-02T10:00:19.000Z"), "id": 683347101804179456, "text": "#NewYear Family #Comedy Show TONIGHT 5:30pm All Tix $6 + 50% of Jan Tix Sales > @RedCrossDFW for #DFWTornado Victims https://t.co/I1tnmlcrcF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NewYear", "Comedy", "DFWTornado" }}, "user_mentions": {{ 17983739 }}, "user": { "id": 2883690657, "name": "Zingerz Comedy Club", "screen_name": "Zingerz_Comedy", "lang": "en", "location": "Bedford, TX", "create_at": date("2014-11-18"), "description": "Family-Friendly Comedy Club that's FUNNY for ALL AGES! Zingerz' Clean Improv Comedy Show is an Interactive Entertainment Experience Unlike Any Other! LAUGH ON!", "followers_count": 68, "friends_count": 110, "statues_count": 506 }, "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-01-02T10:00:19.000Z"), "id": 683347101993025537, "text": "Breosha is snoringg��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "hi", "is_retweet": false, "user": { "id": 724452781, "name": "livinglavish♉️✨", "screen_name": "AmbriaW_", "lang": "en", "location": "Newberry, SC", "create_at": date("2012-07-29"), "description": "snapchat: Ambria_34 ❣", "followers_count": 3390, "friends_count": 3439, "statues_count": 50774 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newberry, SC", "id": "a83e4dd26ae7b004", "name": "Newberry", "place_type": "city", "bounding_box": rectangle("-81.657544,34.255599 -81.571202,34.315219") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45071, "countyName": "Newberry", "cityID": 4549570, "cityName": "Newberry" } }
+{ "create_at": datetime("2016-01-02T10:00:19.000Z"), "id": 683347102135549952, "text": "@OvenOtter I’m headin’ to HOLLYWOOD! ��", "in_reply_to_status": 683346931645468672, "in_reply_to_user": 835529784, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 835529784 }}, "user": { "id": 286668374, "name": "Tiger Twink Today", "screen_name": "Kwandryb", "lang": "en", "location": "Anaheim, CALIFORNRAWR", "create_at": date("2011-04-23"), "description": "RAWR! (don't worry WAHS will return soon)", "followers_count": 723, "friends_count": 408, "statues_count": 50455 }, "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-02T10:00:19.000Z"), "id": 683347102223601666, "text": "Can you recommend anyone for this #job? Scrum Master III - https://t.co/87dSYH3Pbx #McLean, VA #WebDesign #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.1774801,38.9342776"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "McLean", "WebDesign", "Hiring", "CareerArc" }}, "user": { "id": 113036236, "name": "TMJ-VAV IT Jobs", "screen_name": "tmj_VAV_it", "lang": "en", "location": "Tysons, VA", "create_at": date("2010-02-10"), "description": "Follow this account for geo-targeted Software Dev. - General/IT job tweets in Tysons, VA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 336, "friends_count": 308, "statues_count": 43 }, "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-02T10:00:19.000Z"), "id": 683347102282444805, "text": "Gave a stranger a coupon today and her high school-aged son helped me carry my bags to the car. What a kind gesture! #PayItForward", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PayItForward" }}, "user": { "id": 2832531628, "name": "CCPS Research&Assmt", "screen_name": "CCPSAssmt", "lang": "en", "location": "La Plata, MD", "create_at": date("2014-10-15"), "description": "Official Twitter account of the Charles County Public Schools Department of Research and Assessment.", "followers_count": 408, "friends_count": 261, "statues_count": 2028 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waldorf, MD", "id": "4ddec4770a850303", "name": "Waldorf", "place_type": "city", "bounding_box": rectangle("-76.985159,38.608878 -76.841929,38.660379") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24017, "countyName": "Charles", "cityID": 2481175, "cityName": "Waldorf" } }
+{ "create_at": datetime("2016-01-02T10:00:20.000Z"), "id": 683347103242948608, "text": "@aliz0411 @PAPPA___M wish i could say the same", "in_reply_to_status": 683346649444253696, "in_reply_to_user": 410960798, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 410960798, 553670175 }}, "user": { "id": 2388278840, "name": "Fern", "screen_name": "trvpgodfern", "lang": "en", "location": " ΔΤΔ ", "create_at": date("2014-03-13"), "description": "...though I walk through the valley of the shadow of death...", "followers_count": 277, "friends_count": 265, "statues_count": 7831 }, "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-01-02T10:00:20.000Z"), "id": 683347103267958784, "text": "@IamMicaela_K you're welcome Micaela and thank you!", "in_reply_to_status": 683346380119797760, "in_reply_to_user": 2477727313, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2477727313 }}, "user": { "id": 930552433, "name": "Harold Smith", "screen_name": "CU86Grad", "lang": "en", "location": "Buies Creek, NC", "create_at": date("2012-11-06"), "description": "Live in NC but follow bunch of Canadians. Go figure, eh? CU86Grad IG and SnapChat too. #CowboysNation! Thanks to Canadian Sisters #LeafsNation Lover of Bow Ties", "followers_count": 2649, "friends_count": 2277, "statues_count": 124117 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fuquay-Varina, NC", "id": "000b2b6f4a781bba", "name": "Fuquay-Varina", "place_type": "city", "bounding_box": rectangle("-78.844741,35.560069 -78.697863,35.670874") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3725300, "cityName": "Fuquay-Varina" } }
+{ "create_at": datetime("2016-01-02T10:00:20.000Z"), "id": 683347103435911169, "text": "@SMACKHighIL is this someone's real opinion....", "in_reply_to_status": 683332074783293444, "in_reply_to_user": 2249861844, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2249861844 }}, "user": { "id": 1045909795, "name": "Abby", "screen_name": "nissssen", "lang": "en", "location": "815", "create_at": date("2012-12-29"), "description": "99 problems is a lot of problems.", "followers_count": 329, "friends_count": 237, "statues_count": 20425 }, "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-02T10:00:20.000Z"), "id": 683347103582662656, "text": "@Mooflower it's PBS. There aren't commercials", "in_reply_to_status": 683330363800862721, "in_reply_to_user": 21202133, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21202133 }}, "user": { "id": 152224898, "name": "anibundel", "screen_name": "anibundel", "lang": "en", "location": "wildsofFairfax", "create_at": date("2010-06-05"), "description": "Founder of http://Anibundel.com. EiC of @wizards_whatnot News Editor @WiCnet, @dorksideoforce Snarky anglophilic pop culture. All posts approved by my cats.", "followers_count": 938, "friends_count": 786, "statues_count": 68734 }, "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-02T10:00:20.000Z"), "id": 683347103612035073, "text": "Forgive me. I'm just in my feelings 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": 4472652743, "name": "Angelica Marie", "screen_name": "_dearmvrie", "lang": "en", "location": "Chicago, IL", "create_at": date("2015-12-13"), "description": "null", "followers_count": 39, "friends_count": 92, "statues_count": 476 }, "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-01-02T10:00:20.000Z"), "id": 683347103720996870, "text": "Check out my #listing in #Brainerd #MN https://t.co/AFhOBNethA #realestate #realtor https://t.co/aQsLo4apdr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.262507,46.370355"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "Brainerd", "MN", "realestate", "realtor" }}, "user": { "id": 214631153, "name": "Sean M. McDonald", "screen_name": "realtorSeanMac", "lang": "en", "location": "Longville, MN", "create_at": date("2010-11-11"), "description": "25 years of full time real estate service, moved to my long time favorite area in the world, Longville MN., and enjoying life!", "followers_count": 50, "friends_count": 51, "statues_count": 1576 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baxter, MN", "id": "00f0bcb04c0518a1", "name": "Baxter", "place_type": "city", "bounding_box": rectangle("-94.312073,46.31474 -94.22301,46.375874") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27035, "countyName": "Crow Wing", "cityID": 2704042, "cityName": "Baxter" } }
+{ "create_at": datetime("2016-01-02T10:00:20.000Z"), "id": 683347104127954944, "text": "My moms got jokes �� https://t.co/5GjpNaaavI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 626002453, "name": "Cassieeeee ✨", "screen_name": "Cassie_Baybee01", "lang": "en", "location": "drunk", "create_at": date("2012-07-03"), "description": "new life, who dis?", "followers_count": 612, "friends_count": 376, "statues_count": 52738 }, "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-02T10:00:20.000Z"), "id": 683347104157184000, "text": "Sunny this afternoon, high 42 (6 C). Low 22 (-6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433608, "name": "Kansas City Weather", "screen_name": "KansasCityWX", "lang": "en", "location": "Kansas City Metro Area", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Kansas City Metro Area", "followers_count": 1932, "friends_count": 88, "statues_count": 7827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "ddabe75a8f44fe5a", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.765917,38.827176 -94.385522,39.356662") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-01-02T10:00:20.000Z"), "id": 683347104983433217, "text": "She was excited to go to Mexico because she \"just started watching Narcos...\"", "in_reply_to_status": 683346523246088193, "in_reply_to_user": 10138412, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 10138412, "name": "Oscar Godson", "screen_name": "oscargodson", "lang": "en", "location": "Portland, OR", "create_at": date("2007-11-10"), "description": "Co-founder of @getpiggybank, Founding partner of @yorkshireinc. Formerly of @simple and @yammer. Husband of @ciarragodson.", "followers_count": 2083, "friends_count": 711, "statues_count": 75903 }, "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-02T10:00:20.000Z"), "id": 683347105394651137, "text": "@CarlosRamz1 duuuudeeeeeee ������������������������", "in_reply_to_status": 683347035370553344, "in_reply_to_user": 88729451, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 88729451 }}, "user": { "id": 168965882, "name": "Lizbeth Sanchez", "screen_name": "LizzJizzz", "lang": "en", "location": "Port Isabel, TX", "create_at": date("2010-07-20"), "description": "null", "followers_count": 657, "friends_count": 418, "statues_count": 34982 }, "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-01-02T10:00:20.000Z"), "id": 683347105814048772, "text": "Genesis Healthc is hiring! Certified Nursi #jobs in GWYNEDD Apply today https://t.co/L4hZPZfbWA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.245957,40.200758"), "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": 167, "friends_count": 42, "statues_count": 71983 }, "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": 42091, "countyName": "Montgomery", "cityID": 4273264, "cityName": "Spring House" } }
+{ "create_at": datetime("2016-01-02T10:00:20.000Z"), "id": 683347105897930753, "text": "@Raeseti yeah not much else. Cardinals?", "in_reply_to_status": 683346517378215937, "in_reply_to_user": 223174420, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 223174420 }}, "user": { "id": 70641240, "name": "Shane P. Hallam", "screen_name": "ShanePHallam", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2009-09-01"), "description": "Fantasy Football/NFL Draft/College Football Analyst. College Football Expert for @AboutCom, Fantasy Writer @Fanspeak, & President of @DraftTV FWAA & FSWA Member", "followers_count": 13716, "friends_count": 10200, "statues_count": 98759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Castle Shannon, PA", "id": "52b8e53ae833638f", "name": "Castle Shannon", "place_type": "city", "bounding_box": rectangle("-80.038088,40.354859 -80.001567,40.379642") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4211680, "cityName": "Castle Shannon" } }
+{ "create_at": datetime("2016-01-02T10:00:20.000Z"), "id": 683347105943949312, "text": "If you know a dude is a f*ckboy, don't think he's really gonna settle down with you. You're just a temporary thing to him.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 191990134, "name": "Shi'Ann", "screen_name": "Shiannaihs", "lang": "en", "location": "null", "create_at": date("2010-09-17"), "description": "pdx•ama•wf\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#MSU ❤ DBAB", "followers_count": 323, "friends_count": 335, "statues_count": 33735 }, "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-01-02T10:00:20.000Z"), "id": 683347105956548608, "text": "Ready for #NewYear https://t.co/JZ55Hf8Qdq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NewYear" }}, "user": { "id": 27902863, "name": "Kim Rescate", "screen_name": "krescate", "lang": "en", "location": "Oakland, CA", "create_at": date("2009-03-31"), "description": "#Designer + #Photographer + #Immigrant + #Lesbian. #Communicator for @SEIU_UHW. #GAIQ Certified.", "followers_count": 1498, "friends_count": 1601, "statues_count": 11665 }, "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-01-02T10:00:20.000Z"), "id": 683347105990115328, "text": "she took the midnight train 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": 181242953, "name": "em ☾lyn", "screen_name": "emelyyynn", "lang": "en", "location": "California, USA", "create_at": date("2010-08-21"), "description": "null", "followers_count": 462, "friends_count": 388, "statues_count": 11505 }, "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-02T10:00:20.000Z"), "id": 683347106342424577, "text": "#Nursing #Job in #Westfield, MA: RN at Baystate Noble Hospital https://t.co/5GLRBMVYNt #RN #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.7592109,42.11807"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "Job", "Westfield", "RN", "Jobs", "Hiring" }}, "user": { "id": 2688167298, "name": "Noble Hospital Jobs", "screen_name": "JobsatNobleHosp", "lang": "en", "location": "Westfield, MA", "create_at": date("2014-07-28"), "description": "Noble Hospital is a regional community hospital, providing a broad range of expert services to Greater Westfield and surrounding areas.", "followers_count": 17, "friends_count": 0, "statues_count": 94 }, "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-01-02T10:00:20.000Z"), "id": 683347106543828992, "text": "Me on social media this morning https://t.co/IDbwMsbIej", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 371894011, "name": "ChefLuBu @ #SWC", "screen_name": "Chef_Lu_Bu", "lang": "en", "location": "Atlanta, Ga", "create_at": date("2011-09-11"), "description": "Social Media Manager @GASMMedia http://youtube.com/cheflubu \n Disabled veteran and handsome fatty.", "followers_count": 7794, "friends_count": 2456, "statues_count": 89555 }, "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-02T10:00:20.000Z"), "id": 683347106594029569, "text": "somehow everyone in my family managed to get sick. even little kayden.��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 436901753, "name": "Claire Stucker", "screen_name": "ClaireStucker17", "lang": "en", "location": "Fresno, CA", "create_at": date("2011-12-14"), "description": "fresno state softball #17", "followers_count": 747, "friends_count": 664, "statues_count": 16944 }, "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-02T10:00:20.000Z"), "id": 683347106682114048, "text": "This #Healthcare #job might be a great fit for you: Clinical Assistant/Medical Assistant II - Sleep Medicine... - https://t.co/x2zBJIWebd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.2363548,37.4852152"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job" }}, "user": { "id": 3405932663, "name": "SHC Jobs", "screen_name": "SHC_Jobs", "lang": "en", "location": "null", "create_at": date("2015-08-06"), "description": "We’re looking for people like you to help us change the world. Fearless leaders, unstoppable patient advocates, and healing partners. Find your opportunity.", "followers_count": 34, "friends_count": 7, "statues_count": 641 }, "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-01-02T10:00:20.000Z"), "id": 683347106837430272, "text": "are you livin your life or just waitin to die?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 411086579, "name": "Adria Mae Roberts", "screen_name": "AdriaRoberts3", "lang": "en", "location": "null", "create_at": date("2011-11-12"), "description": "she's a good girl who gets ready to trap music", "followers_count": 563, "friends_count": 720, "statues_count": 4378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenville, OH", "id": "1707e796e8dd6496", "name": "Greenville", "place_type": "city", "bounding_box": rectangle("-84.647525,40.074038 -84.595954,40.136045") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39037, "countyName": "Darke", "cityID": 3932340, "cityName": "Greenville" } }
+{ "create_at": datetime("2016-01-02T10:00:20.000Z"), "id": 683347107131035648, "text": "@_E4C_ QC�� Happy G-Day Fool����", "in_reply_to_status": -1, "in_reply_to_user": 243858537, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 243858537 }}, "user": { "id": 2397625264, "name": "D1", "screen_name": "kentland_steph", "lang": "en", "location": "w/ Nijah❤️", "create_at": date("2014-03-09"), "description": "Grind #23 RipMommy❤️", "followers_count": 548, "friends_count": 618, "statues_count": 13923 }, "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-01-02T10:00:20.000Z"), "id": 683347107139325953, "text": "@PaigeEriks @Tprent77 luck? I think it's cuz she's cute ��", "in_reply_to_status": 683346864775770113, "in_reply_to_user": 903168732, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 903168732, 1321243512 }}, "user": { "id": 3164445103, "name": "courtney", "screen_name": "_courtneycollis", "lang": "en", "location": "logansport ", "create_at": date("2015-04-19"), "description": "null", "followers_count": 271, "friends_count": 289, "statues_count": 1243 }, "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-02T10:00:20.000Z"), "id": 683347107181244416, "text": "I am looking for a buyer on 24940 Kipling Avenue N #Scandia #MN https://t.co/kPVnnST6Tv #realestate https://t.co/zURHHY0Qys", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.8900365,45.3101611"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Scandia", "MN", "realestate" }}, "user": { "id": 214631153, "name": "Sean M. McDonald", "screen_name": "realtorSeanMac", "lang": "en", "location": "Longville, MN", "create_at": date("2010-11-11"), "description": "25 years of full time real estate service, moved to my long time favorite area in the world, Longville MN., and enjoying life!", "followers_count": 50, "friends_count": 51, "statues_count": 1577 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minnesota, USA", "id": "9807c5c5f7a2c6ce", "name": "Minnesota", "place_type": "admin", "bounding_box": rectangle("-97.239256,43.499362 -89.483385,49.384359") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27025, "countyName": "Chisago", "cityID": 2711350, "cityName": "Chisago City" } }
+{ "create_at": datetime("2016-01-02T10:00:21.000Z"), "id": 683347107273543680, "text": "Partly cloudy this afternoon, high 49 (9 C). Low 32 (0 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433972, "name": "Memphis Weather", "screen_name": "MemphisWX", "lang": "en", "location": "Memphis, TN", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Memphis, TN", "followers_count": 1688, "friends_count": 88, "statues_count": 7739 }, "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-02T10:00:21.000Z"), "id": 683347107353358336, "text": "Xmas https://t.co/Hyn1PQUGuv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24717099, "name": "Kathleen Olson", "screen_name": "olsok05", "lang": "en", "location": "Bloomington, MN", "create_at": date("2009-03-16"), "description": "null", "followers_count": 4, "friends_count": 226, "statues_count": 123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomington, MN", "id": "080b8d8543aab399", "name": "Bloomington", "place_type": "city", "bounding_box": rectangle("-93.399443,44.78542 -93.203245,44.863519") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2706616, "cityName": "Bloomington" } }
+{ "create_at": datetime("2016-01-02T10:00:21.000Z"), "id": 683347107361628161, "text": "@YOANDRl i'll buy you your plane ticket so you can come to kent, WA i'll even pick you up at the airport, you dont even have to pay for a +", "in_reply_to_status": -1, "in_reply_to_user": 2998906691, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2998906691 }}, "user": { "id": 3255225180, "name": "Bianca Garcia", "screen_name": "_biancagarciaaa", "lang": "en", "location": "Kent, WA", "create_at": date("2015-06-24"), "description": "joelandri ❤️ •mexicana |16| |Snapchat; biancagarciaaa| |Instagram;_biancagarciaaa| @itsjoelpimentel se robó mi corazón |joel follows| @YOANDRl", "followers_count": 138, "friends_count": 189, "statues_count": 1563 }, "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-02T10:00:21.000Z"), "id": 683347107407872000, "text": "Typing this tweet with a semi-frozen finger", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 587990570, "name": "Daeng Chelak", "screen_name": "saichapika", "lang": "en", "location": "Nashville, TN", "create_at": date("2012-05-22"), "description": "i shall creep you in your dreams", "followers_count": 406, "friends_count": 238, "statues_count": 9739 }, "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-01-02T10:00:21.000Z"), "id": 683347107609210880, "text": "@CinchGaming where you find the 3.5mm for astro a40 i just got my sucf controller today", "in_reply_to_status": 671668164741214208, "in_reply_to_user": 2271855696, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2271855696 }}, "user": { "id": 2760173699, "name": "Jeremy Patterson", "screen_name": "Jeremy24in24", "lang": "en", "location": "Marion Country IN", "create_at": date("2014-09-03"), "description": "My favorite Nascar Drivers are Danica Patrick Lebron James my favorite two golfers is Tiger Woods and Natalie Gulbis", "followers_count": 61, "friends_count": 259, "statues_count": 335 }, "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-01-02T10:00:21.000Z"), "id": 683347107898507264, "text": "all I wanna know is who keeps egging my house bruh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 310009285, "name": "elysia hernandez", "screen_name": "__ehernandez", "lang": "en", "location": "johns creek", "create_at": date("2011-06-02"), "description": "snap: elysiahernandez", "followers_count": 431, "friends_count": 183, "statues_count": 1884 }, "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-02T10:00:21.000Z"), "id": 683347108213035009, "text": "Medical Director - Absolute Total Care: (#NorthCharleston, SC) https://t.co/xg1wo8iT2r #Physician #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.9748103,32.8546197"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "hashtags": {{ "NorthCharleston", "Physician", "Job", "Jobs", "Hiring" }}, "user": { "id": 36865943, "name": "TMJ-SCC Health Jobs", "screen_name": "tmj_scc_health", "lang": "en", "location": "Charleston, SC", "create_at": date("2009-04-30"), "description": "Follow this account for geo-targeted Healthcare job tweets in Charleston, SC. Need help? Tweet us at @CareerArc!", "followers_count": 396, "friends_count": 303, "statues_count": 116 }, "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-02T10:00:21.000Z"), "id": 683347108372463616, "text": "I guess I'm gonna be bored at home until tonight :(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 50456200, "name": "Z", "screen_name": "zarrifletcher", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2009-06-24"), "description": "we are what we choose to be ✨", "followers_count": 1525, "friends_count": 1363, "statues_count": 51143 }, "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-02T10:00:21.000Z"), "id": 683347108464742400, "text": "Almost time. @ Slovak Club of Lincoln https://t.co/ZlyQw0YTi4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.67175207,40.82621247"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17322758, "name": "Rob Blue", "screen_name": "rblue78", "lang": "en", "location": "Hooker Corner, Indiana", "create_at": date("2008-11-11"), "description": "I'm basically into beer, scotch, aviation, travel, anthropodermic bibliopegy and screwing around. One-time winner of nine Potbelly sandwiches.", "followers_count": 599, "friends_count": 618, "statues_count": 60832 }, "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-02T10:00:21.000Z"), "id": 683347108464844805, "text": "Genesis Healthc is hiring! Dietary Aide #jobs in MONMOUTH JCT Apply today https://t.co/fAD82UBEqb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.548626,40.394588"), "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": 167, "friends_count": 42, "statues_count": 71984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Brunswick, NJ", "id": "01eff32204eda465", "name": "South Brunswick", "place_type": "city", "bounding_box": rectangle("-74.557382,40.381056 -74.499006,40.44139") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3447190, "cityName": "Monmouth Junction" } }
+{ "create_at": datetime("2016-01-02T10:00:21.000Z"), "id": 683347108548640768, "text": "Every night by @HandsomeHarlow is ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 61642247 }}, "user": { "id": 2898868890, "name": "Joseph Hayford", "screen_name": "hayfordjoseph", "lang": "en", "location": "spokane WA", "create_at": date("2014-11-13"), "description": "snapchat hayford.joe", "followers_count": 161, "friends_count": 482, "statues_count": 211 }, "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-02T10:00:21.000Z"), "id": 683347108594761728, "text": "I guess I can start getting ready 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": 1319439908, "name": "DAN❣ELLE", "screen_name": "danielledahlber", "lang": "en", "location": "Texas, USA", "create_at": date("2013-03-31"), "description": "atx •rip Nathan Carranza ❤️", "followers_count": 364, "friends_count": 428, "statues_count": 8803 }, "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-01-02T10:00:21.000Z"), "id": 683347108670377986, "text": "This is my \" #OMGItsStarting! \" face and Bruce's \" #OMGAnotherSelfie? \" face... #StarWars… https://t.co/0nYMQi16Hv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.8561401,39.0927544"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "OMGItsStarting", "OMGAnotherSelfie", "StarWars" }}, "user": { "id": 460204931, "name": "Anne Kendrick", "screen_name": "AOKendrick", "lang": "en", "location": "Baltimore, Maryland, USA", "create_at": date("2012-01-10"), "description": "Don't dream it. Be it!\nInterests include archaeology, books, museums, travel, movies, politics, sarcasm, and smashing the patriarchy.", "followers_count": 46, "friends_count": 46, "statues_count": 806 }, "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-01-02T10:00:21.000Z"), "id": 683347108930269184, "text": "Some of y'all need Hooked on phonics", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 199929577, "name": "Your Favorite Guy", "screen_name": "TheRealEricMack", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-10-07"), "description": "DC to LA College Grad C•o•15 Team No Chill TφG Giving Up Is Not An Option. I Can Do All Things Through Christ who Strengthens Me.(Phil413)", "followers_count": 1180, "friends_count": 939, "statues_count": 94056 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-02T10:00:21.000Z"), "id": 683347109211422720, "text": "hate when a nigga say shit to me like, \" You don't know what I'll do to you.\" you're right & I don't want to know �� did I ask? ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 381833686, "name": "❥ sugarhoneyicetea ❥", "screen_name": "cvssidy_", "lang": "en", "location": "Sc: @mookeybaby", "create_at": date("2011-09-28"), "description": "keep your heart 3stacks", "followers_count": 1812, "friends_count": 401, "statues_count": 72824 }, "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-02T10:00:21.000Z"), "id": 683347109320392704, "text": "Lol I feel like I gained weight from all the food I ate in Peru", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3312340960, "name": "Romina Martin", "screen_name": "romimartin1303", "lang": "en", "location": "null", "create_at": date("2015-06-07"), "description": "null", "followers_count": 187, "friends_count": 159, "statues_count": 1639 }, "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-02T10:00:21.000Z"), "id": 683347109391630336, "text": "@kellyortega__ @YouAndBabe ya tu sabe ��", "in_reply_to_status": 683201630985048064, "in_reply_to_user": 3187171585, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 3187171585, 2848149332 }}, "user": { "id": 2149956307, "name": "ismael", "screen_name": "diana_baee", "lang": "en", "location": "null", "create_at": date("2013-10-22"), "description": "// a Lancer has my ❤ . // * RHS *", "followers_count": 403, "friends_count": 291, "statues_count": 15110 }, "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-02T10:00:21.000Z"), "id": 683347109844791297, "text": "I'm at Sam’s Morning Glory Diner in Philadelphia, PA https://t.co/bPx62HKgmT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.158869,39.94091606"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30044382, "name": "Mad Franz", "screen_name": "madfranz", "lang": "en", "location": "destination: unknown", "create_at": date("2009-04-09"), "description": "everyday is an adventure. \nlover of: boston college, france, crosswords, chiweenies, all things kitsch.", "followers_count": 375, "friends_count": 821, "statues_count": 2965 }, "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-02T10:00:21.000Z"), "id": 683347110243127296, "text": "Sitting at a gate at the airport. Guy has large guitar case to carry on. 2nd guy almost shut down terminal by walking past TSA. Jerks", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 381650666, "name": "John Hilgert", "screen_name": "JohnHilgert", "lang": "en", "location": "Alton/Godfrey, IL USA", "create_at": date("2011-09-28"), "description": "Spreading sarcasm one meaningless Tweet at a time", "followers_count": 224, "friends_count": 351, "statues_count": 2823 }, "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-01-02T10:00:21.000Z"), "id": 683347110251655168, "text": "I really regret not beating her ass yesterday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 63586015, "name": "LEAFY", "screen_name": "Stanley__Green", "lang": "en", "location": "null", "create_at": date("2009-08-06"), "description": "The Situation your in doesn't define your character but what you do to better that situation does | 1.16.15 Brianna S Haynes ❤️", "followers_count": 1300, "friends_count": 1537, "statues_count": 12204 }, "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-02T10:00:21.000Z"), "id": 683347110356500484, "text": "@21Shawna us af https://t.co/wUGUpEbfxS", "in_reply_to_status": -1, "in_reply_to_user": 1441859353, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1441859353 }}, "user": { "id": 2842355084, "name": "mads", "screen_name": "maddyahouse1", "lang": "en", "location": "null", "create_at": date("2014-10-06"), "description": "null", "followers_count": 256, "friends_count": 187, "statues_count": 5432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kingston, NY", "id": "1385e8d7843ce9d7", "name": "Kingston", "place_type": "city", "bounding_box": rectangle("-74.051642,41.875827 -73.961854,41.957802") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36111, "countyName": "Ulster", "cityID": 3639727, "cityName": "Kingston" } }
+{ "create_at": datetime("2016-01-02T10:00:21.000Z"), "id": 683347110398439424, "text": "Front on me now \nTaste that salt later", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1062338244, "name": "_", "screen_name": "FuckkCash", "lang": "en", "location": "null", "create_at": date("2013-01-04"), "description": "SC : smokepounds", "followers_count": 520, "friends_count": 447, "statues_count": 26427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elmira, NY", "id": "e9b6805c69f635ba", "name": "Elmira", "place_type": "city", "bounding_box": rectangle("-76.842135,42.064833 -76.776861,42.119876") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36015, "countyName": "Chemung", "cityID": 3624229, "cityName": "Elmira" } }
+{ "create_at": datetime("2016-01-02T10:00:21.000Z"), "id": 683347110436012033, "text": "happy birthday @camshelb������❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3718939994 }}, "user": { "id": 1155605005, "name": "ashlee", "screen_name": "ashleeevansss", "lang": "en", "location": "null", "create_at": date("2013-02-06"), "description": "null", "followers_count": 546, "friends_count": 403, "statues_count": 11774 }, "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-02T10:00:21.000Z"), "id": 683347110473777152, "text": "@PettisTyra @M_Berdan when an ugly black girl try's to tell a really hot girl that she is ugly... I don't think that's how it works :/", "in_reply_to_status": 683165411886641152, "in_reply_to_user": 2705125003, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2705125003, 520948204 }}, "user": { "id": 883032494, "name": "Just Ryan Things", "screen_name": "ryawelsh", "lang": "en", "location": "snap: ryanshot", "create_at": date("2012-10-15"), "description": "Mahtomedi has so much diversity -Abigail Hunn", "followers_count": 103, "friends_count": 49, "statues_count": 158 }, "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.890822 -92.983891,45.036878") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27123, "countyName": "Ramsey", "cityID": 2740382, "cityName": "Maplewood" } }
+{ "create_at": datetime("2016-01-02T10:00:21.000Z"), "id": 683347110515716096, "text": "See our latest #Portland, OR #job and click to apply: FILE CLERK - https://t.co/Ns76mQl2Oo #Clerical #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.6762071,45.5234515"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Portland", "job", "Clerical", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 23022397, "name": "TMJ-PDX Cleric. Jobs", "screen_name": "tmj_pdx_cler", "lang": "en", "location": "Portland, OR", "create_at": date("2009-03-05"), "description": "Follow this account for geo-targeted Office/Clerical job tweets in Portland, OR. Need help? Tweet us at @CareerArc!", "followers_count": 290, "friends_count": 201, "statues_count": 86 }, "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-02T10:00:21.000Z"), "id": 683347110561890305, "text": "Can you recommend anyone for this #job? Learning Business Partner - https://t.co/9M6VvsyntN #FarmingtonHills, MI #HR #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.324296,42.491843"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "FarmingtonHills", "HR", "Hiring", "CareerArc" }}, "user": { "id": 23130397, "name": "TMJ-DTW HR Jobs", "screen_name": "tmj_dtw_hr", "lang": "en", "location": "Detroit, MI", "create_at": date("2009-03-06"), "description": "Follow this account for geo-targeted Human Resources job tweets in Detroit, MI. Need help? Tweet us at @CareerArc!", "followers_count": 387, "friends_count": 294, "statues_count": 19 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Farmington Hills, MI", "id": "0b40afc99807b6ac", "name": "Farmington Hills", "place_type": "city", "bounding_box": rectangle("-83.437523,42.439001 -83.316839,42.529556") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2627440, "cityName": "Farmington Hills" } }
+{ "create_at": datetime("2016-01-02T10:00:21.000Z"), "id": 683347110738038784, "text": "Check out my #listing in #Walker #MN https://t.co/DUy2sNxF4o #realestate #realtor https://t.co/SyG2z3fBxa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.571781,47.050761"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "Walker", "MN", "realestate", "realtor" }}, "user": { "id": 214631153, "name": "Sean M. McDonald", "screen_name": "realtorSeanMac", "lang": "en", "location": "Longville, MN", "create_at": date("2010-11-11"), "description": "25 years of full time real estate service, moved to my long time favorite area in the world, Longville MN., and enjoying life!", "followers_count": 50, "friends_count": 51, "statues_count": 1578 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minnesota, USA", "id": "9807c5c5f7a2c6ce", "name": "Minnesota", "place_type": "admin", "bounding_box": rectangle("-97.239256,43.499362 -89.483385,49.384359") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27021, "countyName": "Cass" } }
+{ "create_at": datetime("2016-01-02T10:00:21.000Z"), "id": 683347110952087552, "text": "GENESIS REHABIL is hiring! Speech-Language #jobs in AQUA Apply today https://t.co/eNqNfHedWG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.658566,39.931062"), "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": 167, "friends_count": 42, "statues_count": 71985 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chambersburg, PA", "id": "97f1e7e6f315d56a", "name": "Chambersburg", "place_type": "city", "bounding_box": rectangle("-77.685651,39.907363 -77.623028,39.960692") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42055, "countyName": "Franklin", "cityID": 4212536, "cityName": "Chambersburg" } }
+{ "create_at": datetime("2016-01-02T10:00:21.000Z"), "id": 683347111073693697, "text": "@DJDIRTYBEANZ @LLLYNDAAA @Joshochos @Ruff1o_ that shit is lit I've never not had fun at a gay bar", "in_reply_to_status": 683347029423206400, "in_reply_to_user": 4008517762, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4008517762, 27355260, 3709331, 371781758 }}, "user": { "id": 617851798, "name": "Bill Bosby", "screen_name": "stogestevens", "lang": "en", "location": "BROOKLANDIA", "create_at": date("2012-06-24"), "description": "HOST OF THE STOGE STEVENS SHOW AND FREELANCE WRITER http://SOUNDCLOUD.COM/STOGESTEVENS", "followers_count": 313, "friends_count": 186, "statues_count": 55312 }, "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-02T10:00:21.000Z"), "id": 683347111203737600, "text": "okokokok i have an idea\nmer and her friend genevitalia or whatever\ntheir names are really\n\nmerewho and genitals DCFVGBHNJMK CRYING", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2646194502, "name": "Laur", "screen_name": "DesiredDolans", "lang": "en", "location": "Ethan's ________", "create_at": date("2014-07-14"), "description": "we're squished as fuck\n\n\n#DolanRoyaltyFam #UnholyDolans #BubbleBuppies #DDE #PussayPack", "followers_count": 1115, "friends_count": 159, "statues_count": 13525 }, "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-01-02T10:00:21.000Z"), "id": 683347111254093824, "text": "@chinna_cocainne fb ? ❤", "in_reply_to_status": -1, "in_reply_to_user": 241388478, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 241388478 }}, "user": { "id": 327979402, "name": "#ÝŇïÇ❗", "screen_name": "_26stFinest", "lang": "en", "location": "null", "create_at": date("2011-07-02"), "description": "#FreeTrell #FreeMuniraa #ABL #FreeRif\n#DasKader #YNIC", "followers_count": 1388, "friends_count": 1111, "statues_count": 14251 }, "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-02T10:00:21.000Z"), "id": 683347111434321920, "text": "IM HOME!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.381001,37.618651"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1497371646, "name": "dasha", "screen_name": "duhsha", "lang": "en", "location": "null", "create_at": date("2013-06-09"), "description": "— california", "followers_count": 706, "friends_count": 516, "statues_count": 17752 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco International Airport (SFO)", "id": "07d9dafeb5c82002", "name": "San Francisco International Airport (SFO)", "place_type": "poi", "bounding_box": rectangle("-122.38100109999999,37.6186509 -122.381001,37.618651") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo" } }
+{ "create_at": datetime("2016-01-02T10:00:22.000Z"), "id": 683347111480537089, "text": "#2 https://t.co/qvPU5lfPnS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 303492563, "name": "February,15❤", "screen_name": "BitchyouuMAD__", "lang": "en", "location": " Philly✈Jamaica", "create_at": date("2011-05-22"), "description": "@__bdw , @__anijah , @_prertygirlMEL ❤ SC:pinnk_cocaine February,15 A King Will Be Born ❤ March24 A Queen Will Be Born❤", "followers_count": 1743, "friends_count": 1440, "statues_count": 38112 }, "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-02T10:00:22.000Z"), "id": 683347111568621573, "text": "@taxslayerbowl", "in_reply_to_status": 683332370750111744, "in_reply_to_user": 3321360648, "favorite_count": 0, "coordinate": point("-81.63828733,30.32429608"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 50307161 }}, "user": { "id": 3321360648, "name": "Tory Donahoo", "screen_name": "ToryDonahoo", "lang": "en", "location": "null", "create_at": date("2015-08-20"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 10 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midtown, Jacksonville", "id": "74d045c1a394f792", "name": "Midtown", "place_type": "neighborhood", "bounding_box": rectangle("-81.647881,30.318083 -81.62476,30.333091") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12031, "countyName": "Duval", "cityID": 1235000, "cityName": "Jacksonville" } }
+{ "create_at": datetime("2016-01-02T10:00:22.000Z"), "id": 683347111597993985, "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": 2345866834, "name": "M A R I A H", "screen_name": "_mariiahh_", "lang": "en", "location": "null", "create_at": date("2014-02-15"), "description": "if you aren't contributing to my bank account , goals or my happiness you don't mattaaaaaa", "followers_count": 236, "friends_count": 194, "statues_count": 5245 }, "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-01-02T10:00:22.000Z"), "id": 683347111610564609, "text": "@Mitch__Kupchak Lmao I actually didn't even do anything for New Years ��", "in_reply_to_status": 682965660570390532, "in_reply_to_user": 194133508, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 194133508 }}, "user": { "id": 236694886, "name": "....", "screen_name": "courtneakim", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2011-01-10"), "description": "null", "followers_count": 709, "friends_count": 315, "statues_count": 10518 }, "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-02T10:00:22.000Z"), "id": 683347111836958720, "text": "If you're a #Nursing professional in #LosAngeles, CA, check out this #job: https://t.co/v9gqF4ZC0m #nursingjobs https://t.co/4l45QscYyD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.2452633,33.9239643"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "LosAngeles", "job", "nursingjobs" }}, "user": { "id": 3100019028, "name": "YourMLKCHJobs", "screen_name": "yourMLKCHjobs", "lang": "en", "location": "Los Angeles, CA ", "create_at": date("2015-03-20"), "description": "Now hiring qualified, committed and passionate individuals. Join the MLKCH Team!", "followers_count": 230, "friends_count": 539, "statues_count": 51 }, "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-02T10:00:22.000Z"), "id": 683347111866417152, "text": "@VanossGaming https://t.co/1VW10X8UuY", "in_reply_to_status": -1, "in_reply_to_user": 380020286, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 380020286 }}, "user": { "id": 1666817125, "name": "Jake Smith", "screen_name": "yacob10298", "lang": "en", "location": "Please check out my youtube", "create_at": date("2013-08-12"), "description": "null", "followers_count": 698, "friends_count": 2110, "statues_count": 1393 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bethel Park, PA", "id": "7c1a8644282d6a92", "name": "Bethel Park", "place_type": "city", "bounding_box": rectangle("-80.072649,40.287528 -79.998015,40.359255") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4206064, "cityName": "Bethel Park" } }
+{ "create_at": datetime("2016-01-02T10:00:22.000Z"), "id": 683347111916666880, "text": "@fejonredd sadly ���� https://t.co/ANP1wHc2cM", "in_reply_to_status": -1, "in_reply_to_user": 3775719492, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3775719492 }}, "user": { "id": 433711132, "name": "kellie-ann", "screen_name": "kelliehovorka", "lang": "en", "location": "fairfield", "create_at": date("2011-12-10"), "description": "Know your worth baby then add tax", "followers_count": 495, "friends_count": 519, "statues_count": 2201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, IA", "id": "e5e49e2acfcfccd0", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-92.0065,40.988295 -91.919161,41.032375") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19101, "countyName": "Jefferson", "cityID": 1926445, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-01-02T10:00:22.000Z"), "id": 683347112109584384, "text": "��I just luv slam�� @ Holiday Inn Wilkes Barre https://t.co/nvTsuwruRc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.81420399,41.25078083"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 527527044, "name": "Kelly Clarke", "screen_name": "_clarkelly", "lang": "en", "location": "null", "create_at": date("2012-03-17"), "description": "I think the sun's a little brighter today SCRANTON 2020", "followers_count": 787, "friends_count": 578, "statues_count": 23604 }, "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": 42079, "countyName": "Luzerne" } }
+{ "create_at": datetime("2016-01-02T10:00:22.000Z"), "id": 683347112289906688, "text": "Mostly sunny this afternoon, high 46 (8 C). Low 30 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119697197, "name": "Huntsville Weather", "screen_name": "HuntsvilleWX", "lang": "en", "location": "null", "create_at": date("2010-03-04"), "description": "null", "followers_count": 668, "friends_count": 65, "statues_count": 6050 }, "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-02T10:00:22.000Z"), "id": 683347112306671618, "text": "@ProSyndicate makes it a bit more personal lol. And at least you can see what you're doing.", "in_reply_to_status": 683346793631985664, "in_reply_to_user": 204089551, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 204089551 }}, "user": { "id": 2924417611, "name": "David (Skilletfan93)", "screen_name": "thedustyroad10", "lang": "en", "location": "null", "create_at": date("2014-12-09"), "description": "why are you here?", "followers_count": 180, "friends_count": 365, "statues_count": 2366 }, "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-02T10:00:22.000Z"), "id": 683347112466067457, "text": "@Cushaveli ohhh I don't remember none of that ��", "in_reply_to_status": 683347002013294592, "in_reply_to_user": 167036809, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 167036809 }}, "user": { "id": 58608549, "name": "MCJ3.", "screen_name": "Myray_Cyntrese", "lang": "en", "location": "loading...", "create_at": date("2009-07-20"), "description": "I'm Myrie. God Hold it down ! ΔΣΘ ΘΞ ❤️ 225.337 Less Huggin mo hustlin. UL16.", "followers_count": 1461, "friends_count": 580, "statues_count": 86709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gramercy, LA", "id": "00a2525a2f622b41", "name": "Gramercy", "place_type": "city", "bounding_box": rectangle("-90.712119,30.040513 -90.662068,30.079008") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22093, "countyName": "St. James", "cityID": 2230550, "cityName": "Gramercy" } }
+{ "create_at": datetime("2016-01-02T10:00:22.000Z"), "id": 683347112822587394, "text": "@_hollycarney dude me too", "in_reply_to_status": 683346654284496896, "in_reply_to_user": 1031332748, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1031332748 }}, "user": { "id": 2766539153, "name": "trina roth", "screen_name": "wanabeaROTHstar", "lang": "en", "location": "Omaha, NE", "create_at": date("2014-09-10"), "description": "if you're dreams don't scare you, they aren't big enough| skutt catholic☻", "followers_count": 309, "friends_count": 315, "statues_count": 3594 }, "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-02T10:00:22.000Z"), "id": 683347113246343168, "text": "#BeardCrew #TeamVegeta https://t.co/cuGBys4cUg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "BeardCrew", "TeamVegeta" }}, "user": { "id": 393242944, "name": "Joe", "screen_name": "FatalStryke", "lang": "en", "location": "NY, USA", "create_at": date("2011-10-18"), "description": "Proud partnered @Twitch streamer. Gears of war competitor & Caster. #iQHQ Co-Owner. Love my #BeardCrew Business inquiries email: iqhqtv@gmail.com", "followers_count": 4623, "friends_count": 1396, "statues_count": 37706 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Farmingdale, NY", "id": "c55c9fed765e35a1", "name": "South Farmingdale", "place_type": "city", "bounding_box": rectangle("-73.468517,40.704497 -73.429787,40.730502") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3669001, "cityName": "South Farmingdale" } }
+{ "create_at": datetime("2016-01-02T10:00:22.000Z"), "id": 683347113342705664, "text": "I'm sick af' ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3308784973, "name": "myaa", "screen_name": "myam510", "lang": "en", "location": "null", "create_at": date("2015-08-07"), "description": "chs // sc: love_mya", "followers_count": 166, "friends_count": 131, "statues_count": 1953 }, "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-02T10:00:22.000Z"), "id": 683347113376354304, "text": "@hnusss3 https://t.co/2Iw1wpRWjP", "in_reply_to_status": -1, "in_reply_to_user": 386562865, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 386562865 }}, "user": { "id": 2882767224, "name": "Catherine Nusser", "screen_name": "catnusser", "lang": "en", "location": "Mooresville, NC", "create_at": date("2014-10-29"), "description": "MHS '19", "followers_count": 266, "friends_count": 175, "statues_count": 1077 }, "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-01-02T10:00:22.000Z"), "id": 683347113850322944, "text": "GENESIS REHABIL is hiring! Speech-Language #jobs in FRITZTOWN Apply today https://t.co/dDL0KLbqby", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.033437,40.317246"), "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": 167, "friends_count": 42, "statues_count": 71986 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sinking Spring, PA", "id": "2cfae8c2b76b4440", "name": "Sinking Spring", "place_type": "city", "bounding_box": rectangle("-76.037842,40.315175 -76.006632,40.335486") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42011, "countyName": "Berks", "cityID": 4270880, "cityName": "Sinking Spring" } }
+{ "create_at": datetime("2016-01-02T10:00:22.000Z"), "id": 683347114328371200, "text": "El sexo es bueno bajo cualquier efecto de drogas o alcohol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2635203026, "name": "_milanno", "screen_name": "__mota_milano", "lang": "es", "location": "California, USA", "create_at": date("2014-07-12"), "description": "La liberad no existe. La lógica es solo una palabra. Eres libre de alma. La gente es un asco. Piensa diferente. ACUARIANO, mente activa. Aire.", "followers_count": 338, "friends_count": 246, "statues_count": 5175 }, "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-02T10:00:22.000Z"), "id": 683347114479325184, "text": "don't be afraid to confess ur love for me 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": 1250397360, "name": "beth claire", "screen_name": "bethchamilton", "lang": "en", "location": "Bryan, TX", "create_at": date("2013-03-07"), "description": "fifteen. poet, coffee lover, big fan of Jesus. bhs '19.", "followers_count": 202, "friends_count": 114, "statues_count": 5241 }, "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-02T10:00:22.000Z"), "id": 683347114496233473, "text": "The amount of fake accounts on Instagram liking my photos is almost enough for me to delete that app.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 55488643, "name": "Ian Soper", "screen_name": "Ian_Soper", "lang": "en", "location": "Cincinnati, USA", "create_at": date("2009-07-09"), "description": "The United States of America, ATΩ, Reds, Seahawks, Blackhawks, Bearcats, Jonny Gomes, Graham Rahal, Conor Daly, Fernando Alonso, & The Indy 500 is what I like.", "followers_count": 346, "friends_count": 689, "statues_count": 21921 }, "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-02T10:00:22.000Z"), "id": 683347114521300992, "text": "#TAMUC Follow @commerce_LIVE RIGHT NOW!!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TAMUC" }}, "user_mentions": {{ 2978602121 }}, "user": { "id": 441157727, "name": "the8ThLetter", "screen_name": "_JossySays", "lang": "en", "location": "null", "create_at": date("2011-12-19"), "description": "#TAMUC #LongLiveClyde", "followers_count": 1643, "friends_count": 1265, "statues_count": 89674 }, "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-02T10:00:22.000Z"), "id": 683347114806493185, "text": "Partly cloudy this afternoon, high 47 (8 C). Low 32 (0 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119698223, "name": "Birmingham Weather", "screen_name": "BirminghamWX", "lang": "en", "location": "Birmingham, AL", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Birmingham, AL", "followers_count": 503, "friends_count": 68, "statues_count": 6062 }, "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-02T10:00:22.000Z"), "id": 683347114903011328, "text": "Want to work in #Clearwater, FL? View our latest opening: https://t.co/LELMClAPvT #Retail #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.7090158,28.0133179"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Clearwater", "Retail", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 28479033, "name": "TMJ-TPA Retail Jobs", "screen_name": "tmj_tpa_retail", "lang": "en", "location": "Tampa Bay, FL", "create_at": date("2009-04-02"), "description": "Follow this account for geo-targeted Retail job tweets in Tampa Bay, FL. Need help? Tweet us at @CareerArc!", "followers_count": 420, "friends_count": 309, "statues_count": 1326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Safety Harbor, FL", "id": "ab944f41faea7e2f", "name": "Safety Harbor", "place_type": "city", "bounding_box": rectangle("-82.712772,27.977811 -82.675347,28.049126") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1262425, "cityName": "Safety Harbor" } }
+{ "create_at": datetime("2016-01-02T10:00:22.000Z"), "id": 683347115234455552, "text": "They're taking it too far!!!! https://t.co/hDArq89oLA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2446027866, "name": "Denise Julian", "screen_name": "_BossedUpDee", "lang": "en", "location": "Cleveland, Ohio", "create_at": date("2014-04-15"), "description": "Singer-Songwriter | Female Cancer | My Favorite Color is Red | Instagram: @_BossedUpDee | SEND ALL INQUIRIES TO CRAIG EVANS sleep905@gmail.com", "followers_count": 583, "friends_count": 503, "statues_count": 5487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beachwood, OH", "id": "fa5bd676191716ee", "name": "Beachwood", "place_type": "city", "bounding_box": rectangle("-81.532635,41.449669 -81.48037,41.501448") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3904500, "cityName": "Beachwood" } }
+{ "create_at": datetime("2016-01-02T10:00:22.000Z"), "id": 683347115251118080, "text": "Packing to go back to college and I feel like it's not gonna fit in 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": 270633921, "name": "Justice Dumlao", "screen_name": "Justice_DDG", "lang": "en", "location": "Nowhere in Particular", "create_at": date("2011-03-22"), "description": "UCSB//booty//woke queer boy//generally confused", "followers_count": 501, "friends_count": 619, "statues_count": 3167 }, "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-01-02T10:00:22.000Z"), "id": 683347115347677184, "text": "Buenos días a todos ☝��️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2847939110, "name": "Patycaballero", "screen_name": "pcaballero7", "lang": "es", "location": "null", "create_at": date("2014-10-08"), "description": "Chilanga", "followers_count": 553, "friends_count": 1103, "statues_count": 2375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-02T10:00:22.000Z"), "id": 683347115527897088, "text": "Today's game at Nicholls can be heard live at 1 p.m. (Central) on KLVI (AM 560). #PeckEm https://t.co/CtvRyJeszF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PeckEm" }}, "user": { "id": 2722542254, "name": "Lamar Women's Hoops", "screen_name": "LamarWBB", "lang": "en", "location": "Beaumont, Texas", "create_at": date("2014-08-10"), "description": "The official twitter page of Lamar University Women's Basketball #LU #GoBigRed #PeckEm", "followers_count": 553, "friends_count": 458, "statues_count": 795 }, "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-01-02T10:00:23.000Z"), "id": 683347115662168064, "text": "@jackieealvarez I wish I could.. The way my job set up though ��", "in_reply_to_status": 683343025737740288, "in_reply_to_user": 374279362, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 374279362 }}, "user": { "id": 431025761, "name": "kristen arias", "screen_name": "kriiistenarias", "lang": "en", "location": "OK", "create_at": date("2011-12-07"), "description": "null", "followers_count": 1341, "friends_count": 355, "statues_count": 47558 }, "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-02T10:00:23.000Z"), "id": 683347115678928897, "text": "I am looking for a buyer on 5118 Kings Trail NE #Longville #MN https://t.co/JiGMCohtwU #realestate https://t.co/1SKcJmxV08", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.235048,46.988428"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Longville", "MN", "realestate" }}, "user": { "id": 214631153, "name": "Sean M. McDonald", "screen_name": "realtorSeanMac", "lang": "en", "location": "Longville, MN", "create_at": date("2010-11-11"), "description": "25 years of full time real estate service, moved to my long time favorite area in the world, Longville MN., and enjoying life!", "followers_count": 50, "friends_count": 51, "statues_count": 1579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minnesota, USA", "id": "9807c5c5f7a2c6ce", "name": "Minnesota", "place_type": "admin", "bounding_box": rectangle("-97.239256,43.499362 -89.483385,49.384359") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27021, "countyName": "Cass" } }
+{ "create_at": datetime("2016-01-02T10:00:23.000Z"), "id": 683347115846832128, "text": "Me lo robe de @jenni_amp pero es que esta muy bueno más para… https://t.co/Ahf9N6mZYl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.33477945,29.9880206"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 121755957 }}, "user": { "id": 26226159, "name": "G@briel", "screen_name": "elpapixs", "lang": "es", "location": "Bogota", "create_at": date("2009-03-24"), "description": "Bohemio, padre, hijo, hermano, amigo... Confiado en que las cosas buenas llegan en cualquier momento, solo hay q ser constante!!!", "followers_count": 112, "friends_count": 98, "statues_count": 1174 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-02T10:00:23.000Z"), "id": 683347116090109952, "text": "@naldodavis hahaha that's old af. And if I remember correctly we were joking. But idk lol", "in_reply_to_status": 683346155577696256, "in_reply_to_user": 113030654, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 113030654 }}, "user": { "id": 251707058, "name": "Extra Macsauce", "screen_name": "MvcPlvce", "lang": "en", "location": "Snortwood University", "create_at": date("2011-02-13"), "description": "4 out of 5 dentists recommend me snapchat: macplace20", "followers_count": 756, "friends_count": 278, "statues_count": 35243 }, "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-01-02T10:00:23.000Z"), "id": 683347116274528257, "text": "@lovelala__ you're welcome ��", "in_reply_to_status": 683346771943161856, "in_reply_to_user": 241813690, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 241813690 }}, "user": { "id": 460712728, "name": "S", "screen_name": "princessmaree_", "lang": "en", "location": "Wichita Falls, TX", "create_at": date("2012-01-10"), "description": "C/O 2016", "followers_count": 1794, "friends_count": 1413, "statues_count": 22673 }, "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-02T10:00:23.000Z"), "id": 683347116421451776, "text": "I hate niggas bruh ������ https://t.co/BKMghRJ0D3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 253851375, "name": "5'4 = I got no hoes", "screen_name": "IPutSugarOnRice", "lang": "en", "location": "South Florida ", "create_at": date("2011-02-17"), "description": "Intellectual foreplay is my forte. I live my life as if my parents are out of town for the weekend. Navy affiliated. SC: noSHIRTnDUNKS IG: iputsugaronmyrice", "followers_count": 768, "friends_count": 719, "statues_count": 33561 }, "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-01-02T10:00:23.000Z"), "id": 683347116509540352, "text": "bruhhhhhhhhhh that was so close", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 633137626, "name": "Cece Benjamin", "screen_name": "cecebenjamin", "lang": "en", "location": "null", "create_at": date("2012-07-11"), "description": "instagram|cecebenjamin", "followers_count": 290, "friends_count": 96, "statues_count": 15065 }, "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.282413 -71.156939,42.367744") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2545560, "cityName": "Newton" } }
+{ "create_at": datetime("2016-01-02T10:00:23.000Z"), "id": 683347116555644928, "text": "30 for 30 - Drake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 491627816, "name": "Friisky..", "screen_name": "Friisky_2Times", "lang": "en", "location": "in the clouds..", "create_at": date("2012-02-13"), "description": "Match one? [Snapchat: Friisky_bit] #GoodVibesOnly", "followers_count": 948, "friends_count": 751, "statues_count": 60263 }, "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-01-02T10:00:23.000Z"), "id": 683347116580810752, "text": "@haleynnicole thanks for the spoiler bud��", "in_reply_to_status": 683335965734211584, "in_reply_to_user": 473406082, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 473406082 }}, "user": { "id": 218370456, "name": "AdamJernberg", "screen_name": "AdamJernberg12", "lang": "en", "location": "Cape Cod | Sweden", "create_at": date("2010-11-21"), "description": "I'm Adam and I obsess over Bradley Cooper and Jump Rope✌️| Cirque Performer", "followers_count": 1243, "friends_count": 361, "statues_count": 13954 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brewster, MA", "id": "c752649c81d897f4", "name": "Brewster", "place_type": "city", "bounding_box": rectangle("-70.145872,41.719573 -70.005238,41.787695") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25001, "countyName": "Barnstable", "cityID": 2507945, "cityName": "Brewster" } }
+{ "create_at": datetime("2016-01-02T10:00:23.000Z"), "id": 683347116681379841, "text": "I actually really enjoyed the new Mission Impossible Rouge Nation :)��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 59076997, "name": "Jeff Bartrom", "screen_name": "DocCupCake84", "lang": "en", "location": "CA", "create_at": date("2009-07-22"), "description": "#XBOXSUPERFAN, XBOX GT: Doccupcake, #docsgiveaways , US NAVY VET⚓️, USD Student , #DocsCarePackages for US Military, Royal Blade Studio Dev (Project Knighthood)", "followers_count": 1620, "friends_count": 198, "statues_count": 9198 }, "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-02T10:00:23.000Z"), "id": 683347116727611392, "text": "GENESIS REHABIL is hiring! Occupational Th #jobs in PLAINFIELD Apply today https://t.co/cCyZgc6aKY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.442228,40.641608"), "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": 167, "friends_count": 42, "statues_count": 71987 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Watchung, NJ", "id": "3c28264aa7ba72d2", "name": "Watchung", "place_type": "city", "bounding_box": rectangle("-74.482433,40.618215 -74.401743,40.667749") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34035, "countyName": "Somerset", "cityID": 3477600, "cityName": "Watchung" } }
+{ "create_at": datetime("2016-01-02T10:00:23.000Z"), "id": 683347116769546240, "text": "FREE Daytona Beach Kennel Club Saturday evening picks from \"Post Time Perry\". https://t.co/QpLRnuKPrg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.1324502,25.9948596"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 42911831, "name": "GreyhoundNews", "screen_name": "GreyhoundNews", "lang": "en", "location": "Hallandale,FL", "create_at": date("2009-05-27"), "description": "Greyhound Racing News from around the U.S.A. Greyhound Racing, Handicapping, Free Picks, Free Tip Sheets, Parimutuel Wagering, Dog Racing, Betting, Kennel Club", "followers_count": 3535, "friends_count": 3134, "statues_count": 153245 }, "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-01-02T10:00:23.000Z"), "id": 683347116773781504, "text": "Music was fucked pretty much when all the post punk revival bands couldn't deliver anything great after their second albums", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 352118393, "name": "kandy is dead", "screen_name": "tompuopolo", "lang": "en", "location": "null", "create_at": date("2011-08-09"), "description": "i tweet about dawsons creek", "followers_count": 402, "friends_count": 295, "statues_count": 26354 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forked River, NJ", "id": "00d88eb470b73bd2", "name": "Forked River", "place_type": "city", "bounding_box": rectangle("-74.223558,39.809802 -74.143,39.879125") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean", "cityID": 3424180, "cityName": "Forked River" } }
+{ "create_at": datetime("2016-01-02T10:00:23.000Z"), "id": 683347117067350016, "text": "@XlX_tigerwoods wassup??", "in_reply_to_status": 683270360058433536, "in_reply_to_user": 1458626660, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1458626660 }}, "user": { "id": 24125771, "name": "#BlackMenDraggin", "screen_name": "_freakydre", "lang": "en", "location": "The 410", "create_at": date("2009-03-12"), "description": "towson. photographer. writer for the @thedemotape_. filmmaker kinda. #HeavyBoys", "followers_count": 2105, "friends_count": 1503, "statues_count": 153497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Randallstown, MD", "id": "ceaed3406b427e05", "name": "Randallstown", "place_type": "city", "bounding_box": rectangle("-76.843251,39.343366 -76.746531,39.392708") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2464950, "cityName": "Randallstown" } }
+{ "create_at": datetime("2016-01-02T10:00:23.000Z"), "id": 683347117176299521, "text": "Hello could you wake up please @tramp0lineb00ty", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 620211710 }}, "user": { "id": 2783350951, "name": "Overweight Wrestler", "screen_name": "BigTLangston", "lang": "en", "location": "null", "create_at": date("2014-08-31"), "description": "I love two things food , and @tramp0lineb00ty", "followers_count": 175, "friends_count": 145, "statues_count": 3629 }, "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-01-02T10:00:23.000Z"), "id": 683347117176434688, "text": "\"I want the weeknd to yell in my pussy real loud\" lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1648220550, "name": "luis", "screen_name": "YungSavageLou", "lang": "en", "location": "null", "create_at": date("2013-08-05"), "description": "Dominican", "followers_count": 374, "friends_count": 303, "statues_count": 5146 }, "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-02T10:00:23.000Z"), "id": 683347117289652225, "text": "Today while my boys went to do \"Heli-snowboarding\" with \"dangerous\"… https://t.co/y927yqKByj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-106.82321425,39.19131438"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34735464, "name": "Fernando S.Cobourg", "screen_name": "saxecobourg", "lang": "en", "location": "New York, USA", "create_at": date("2009-04-23"), "description": "Name,Fernando Saxe Cobourg und Gotha,born in France,married,PHD,Partner at NY's brokerage company.Love,Polo,Tennis,Sail,Scubadive and travel.", "followers_count": 1282, "friends_count": 2050, "statues_count": 39639 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aspen, CO", "id": "930b3358e4bd7658", "name": "Aspen", "place_type": "city", "bounding_box": rectangle("-106.879146,39.176776 -106.801346,39.237152") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8097, "countyName": "Pitkin", "cityID": 803620, "cityName": "Aspen" } }
+{ "create_at": datetime("2016-01-02T10:00:23.000Z"), "id": 683347117386014721, "text": "Yo wtf I live in Arizona now...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1035781231, "name": "BoomerSooner", "screen_name": "2014jaime", "lang": "en", "location": " Oklahoma", "create_at": date("2012-12-25"), "description": "Chasin the dream one step at a time. ⚾️ Half the talent, twice the heart.", "followers_count": 638, "friends_count": 868, "statues_count": 24447 }, "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.342063,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-01-02T10:00:23.000Z"), "id": 683347117390168064, "text": "Partly cloudy this afternoon, high 53 (12 C). Low 41 (5 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119698549, "name": "Mobile Weather", "screen_name": "MobileWX", "lang": "en", "location": "Mobile, Al", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Mobile, Al", "followers_count": 404, "friends_count": 68, "statues_count": 6137 }, "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-02T10:00:23.000Z"), "id": 683347117532811264, "text": "My only problem with #ForceAwakens - not enough product integration. My cat only had ONE choice of scratch boards. https://t.co/5JywLMCXxm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ForceAwakens" }}, "user": { "id": 50874127, "name": "Seth Holst", "screen_name": "SethHolst", "lang": "en", "location": "MN", "create_at": date("2009-06-25"), "description": "I like tacos.\n\nOpinions are my own & also correct.", "followers_count": 141, "friends_count": 240, "statues_count": 1259 }, "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-01-02T10:00:23.000Z"), "id": 683347117851672576, "text": "Of course I'm sitting next to a screaming street urchin... Bracelet courtesy of @miajanco… https://t.co/TB64RUxNxA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.15277778,26.0725"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 87729749 }}, "user": { "id": 1053787441, "name": "Doug Brewer", "screen_name": "Bernadizz", "lang": "en", "location": "Kansas City, MO", "create_at": date("2013-01-01"), "description": "null", "followers_count": 129, "friends_count": 233, "statues_count": 1568 }, "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-01-02T10:00:23.000Z"), "id": 683347118283698176, "text": "Lmfaoooo what https://t.co/DECHBXUYlA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29846449, "name": "K", "screen_name": "KenDollsHouse_", "lang": "en", "location": "Chicago, IL", "create_at": date("2009-04-08"), "description": "Glowing & Growing. Howard University'19", "followers_count": 7492, "friends_count": 993, "statues_count": 48846 }, "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-02T10:00:23.000Z"), "id": 683347118426214401, "text": "@king_platano y r we friends wtf ur mean as shit 2 me https://t.co/625yTbqm1v", "in_reply_to_status": -1, "in_reply_to_user": 616389176, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 616389176 }}, "user": { "id": 539553745, "name": "dolores haze", "screen_name": "dancegianadance", "lang": "en", "location": "sin wagon ", "create_at": date("2012-03-28"), "description": "so magnetic | Alex ❤️ | #HeadSpace2k16", "followers_count": 471, "friends_count": 1028, "statues_count": 13348 }, "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-02T10:00:23.000Z"), "id": 683347118451458049, "text": "Temp 39.3°F RH 54% Wind 4.4 WSW Gust 9.0 WSW SLP 30.240 in Falling Rain 0.00 in Solar 524 UV 2.3 #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": 102, "friends_count": 65, "statues_count": 32233 }, "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-02T10:00:23.000Z"), "id": 683347118497599489, "text": "Beyoncé is such a inspiration ❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2200074997, "name": "Cece", "screen_name": "cecedominguez_", "lang": "en", "location": "null", "create_at": date("2013-11-17"), "description": "11112 / Sweeter than a swisher.", "followers_count": 2459, "friends_count": 872, "statues_count": 22637 }, "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-02T10:00:23.000Z"), "id": 683347118682148864, "text": "Yooooo the accuracy here ���� https://t.co/uR4tVMCCMT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 37789884, "name": "Dapper Deejay", "screen_name": "DJ_CoolAss", "lang": "en", "location": "421 N. Albany St", "create_at": date("2009-05-04"), "description": "K I N G", "followers_count": 2426, "friends_count": 1934, "statues_count": 79354 }, "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-02T10:00:23.000Z"), "id": 683347118933716992, "text": "I will be dabbin in the grave", "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": 1022, "friends_count": 419, "statues_count": 71082 }, "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-02T10:00:23.000Z"), "id": 683347119021928449, "text": "So we're at a graveyard a lot of my family is at and there is a family of gypsies here with huge ass gravestones.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3188217792, "name": "✱jade✱", "screen_name": "hannah_jadeee", "lang": "en", "location": "null", "create_at": date("2015-05-07"), "description": "16|SCHS|libra|bisexual", "followers_count": 25, "friends_count": 37, "statues_count": 329 }, "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-02T10:00:23.000Z"), "id": 683347119474917376, "text": "about to 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": 292399450, "name": "j a y", "screen_name": "Devonte_Glenn", "lang": "en", "location": "ohio", "create_at": date("2011-05-03"), "description": "heartless af", "followers_count": 510, "friends_count": 2044, "statues_count": 16997 }, "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-02T10:00:23.000Z"), "id": 683347119529439234, "text": "Getting my suntan on for my upcoming shoots for another studio next week. #BBBH #notanlines #pig #nude #Haulover https://t.co/kRVkadvJDN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.121025,25.919884"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BBBH", "notanlines", "pig", "nude", "Haulover" }}, "user": { "id": 4345215861, "name": "Jay Ricci XXX", "screen_name": "jayriccixxx", "lang": "en", "location": "Fort Lauderdale, FL", "create_at": date("2015-12-01"), "description": "18+ NSFW. Adult gay porn actor. Gay, all bareback porn. Leather daddy type. Top/vers. \nJayRicciXXX@gmail.com\nhttp://LeatherTopFTL.tumblr.com", "followers_count": 71, "friends_count": 65, "statues_count": 39 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Haulover Beach Park Nudist Beach", "id": "08161a2c7e165000", "name": "Haulover Beach Park Nudist Beach", "place_type": "poi", "bounding_box": rectangle("-80.1210251,25.9198839 -80.121025,25.919884") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade" } }
+{ "create_at": datetime("2016-01-02T10:00:23.000Z"), "id": 683347119726395392, "text": "OMG ����. Mines 19th. https://t.co/3fArzoHzva", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 621280466, "name": "Magma", "screen_name": "OyinOdulaja", "lang": "en", "location": "chicago ", "create_at": date("2012-06-28"), "description": "Mad Ting", "followers_count": 4144, "friends_count": 2530, "statues_count": 64706 }, "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-02T10:00:24.000Z"), "id": 683347119927881729, "text": "baby's first copic set ft. breezy https://t.co/ybCrN3wUKW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174452122, "name": "peach ma", "screen_name": "lolewbye", "lang": "en", "location": "nyc • she/her", "create_at": date("2010-08-03"), "description": "whose chair is that?", "followers_count": 493, "friends_count": 213, "statues_count": 36359 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baldwin, NY", "id": "c2270f3c90deeb93", "name": "Baldwin", "place_type": "city", "bounding_box": rectangle("-73.623737,40.638174 -73.596821,40.687876") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3604143, "cityName": "Baldwin" } }
+{ "create_at": datetime("2016-01-02T10:00:24.000Z"), "id": 683347119957131265, "text": "Mostly sunny this afternoon, high 32 (0 C). Low 25 (-4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119751854, "name": "Chicago Weather", "screen_name": "Chicago_WX", "lang": "en", "location": "Chicago, IL", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Chicago, IL", "followers_count": 1059, "friends_count": 68, "statues_count": 6419 }, "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-02T10:00:24.000Z"), "id": 683347120091340800, "text": "Lorenzo's Italian pizza; made by real Mexicans", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3179915986, "name": "Jacob Rock", "screen_name": "Jrock2312", "lang": "en", "location": "null", "create_at": date("2015-04-18"), "description": "bless up\n\n\nUSA", "followers_count": 89, "friends_count": 149, "statues_count": 645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Carmel, NY", "id": "9d510608e32022ec", "name": "Lake Carmel", "place_type": "city", "bounding_box": rectangle("-73.691105,41.436676 -73.650361,41.497116") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36079, "countyName": "Putnam", "cityID": 3640398, "cityName": "Lake Carmel" } }
+{ "create_at": datetime("2016-01-02T10:00:24.000Z"), "id": 683347120187822081, "text": "I am looking for a buyer on 5109 Shangri La Drive NE #Longville #MN https://t.co/ntqhI8W5ci #realestate https://t.co/4z5UPrz4F1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.251576,46.987802"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Longville", "MN", "realestate" }}, "user": { "id": 214631153, "name": "Sean M. McDonald", "screen_name": "realtorSeanMac", "lang": "en", "location": "Longville, MN", "create_at": date("2010-11-11"), "description": "25 years of full time real estate service, moved to my long time favorite area in the world, Longville MN., and enjoying life!", "followers_count": 50, "friends_count": 51, "statues_count": 1580 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minnesota, USA", "id": "9807c5c5f7a2c6ce", "name": "Minnesota", "place_type": "admin", "bounding_box": rectangle("-97.239256,43.499362 -89.483385,49.384359") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27021, "countyName": "Cass" } }
+{ "create_at": datetime("2016-01-02T10:00:24.000Z"), "id": 683347120221372416, "text": "We're #hiring! Click to apply: Security Officer - https://t.co/pDiLm951wR #security #Shakopee, MN #Veterans #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.5272861,44.7973962"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "security", "Shakopee", "Veterans", "Job", "Jobs", "CareerArc" }}, "user": { "id": 118914397, "name": "TMJ-MSP Security Job", "screen_name": "tmj_MSP_secure", "lang": "en", "location": "St. Paul, MN", "create_at": date("2010-03-01"), "description": "Follow this account for geo-targeted Security job tweets in St. Paul, MN from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 293, "friends_count": 266, "statues_count": 15 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shakopee, MN", "id": "00221a12113d93c6", "name": "Shakopee", "place_type": "city", "bounding_box": rectangle("-93.582038,44.752728 -93.398642,44.804724") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27139, "countyName": "Scott", "cityID": 2759350, "cityName": "Shakopee" } }
+{ "create_at": datetime("2016-01-02T10:00:24.000Z"), "id": 683347120267636737, "text": "@BTS_twt stay healthy ☺️��", "in_reply_to_status": 683300258554642433, "in_reply_to_user": 335141638, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 335141638 }}, "user": { "id": 3384126261, "name": "GetWellSoonRM❤️", "screen_name": "jungcock1997", "lang": "en", "location": "Kris's Weave", "create_at": date("2015-07-19"), "description": "#Getwellsoonnamjoon! #HandsomeRM all you haters need to evaporate @ Namjoon JIMIN GOT YES JAMS #Getwellsoontaegi", "followers_count": 58, "friends_count": 118, "statues_count": 1794 }, "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-02T10:00:24.000Z"), "id": 683347120305352704, "text": "I've seen #TheForceAwakens so many times that my phone predicted the Oaks 10 as my next destination. �� https://t.co/utooKKbXm1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TheForceAwakens" }}, "user": { "id": 15553138, "name": "Johnny", "screen_name": "jonathancrll", "lang": "en", "location": "Melbourne, FL", "create_at": date("2008-07-22"), "description": "null", "followers_count": 258, "friends_count": 260, "statues_count": 3393 }, "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-02T10:00:24.000Z"), "id": 683347120640913408, "text": "I dont like the helmets they wearing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2504588336, "name": "DOUBLEO11", "screen_name": "TezarooMercer_", "lang": "en", "location": "Between I Want It & I Got It ", "create_at": date("2014-05-18"), "description": "• Old Dominion University Football Commit • #FreeTheGoodManSqueak", "followers_count": 954, "friends_count": 834, "statues_count": 7405 }, "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-02T10:00:24.000Z"), "id": 683347121257451520, "text": "@caitlingarrity https://t.co/7KZw9R8vqz", "in_reply_to_status": -1, "in_reply_to_user": 561549368, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 561549368 }}, "user": { "id": 834116856, "name": "jacy", "screen_name": "jacex_09", "lang": "en", "location": "null", "create_at": date("2012-09-19"), "description": "null", "followers_count": 375, "friends_count": 318, "statues_count": 10316 }, "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-02T10:00:24.000Z"), "id": 683347121349595136, "text": "@Spidey_Grips https://t.co/Dv8Gc7Cb7g today is the day stream for autism i will have giveaways music and fun please join and help the cause", "in_reply_to_status": -1, "in_reply_to_user": 3198600293, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3198600293 }}, "user": { "id": 4168351522, "name": "MAK S4DISTIC JOKER", "screen_name": "S4DISTICJOKER", "lang": "en", "location": "Gotham City", "create_at": date("2015-11-11"), "description": "xb1 S4DISTIC JOKER MAK team member,sponsored by @Spidey_Grips, @brimanscustoms, and @westcoastchill. Gamer,streamer, father, http://youtube.com/channel/UCNW52…", "followers_count": 711, "friends_count": 681, "statues_count": 10284 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenfield, WI", "id": "7f42154616cee8fa", "name": "Greenfield", "place_type": "city", "bounding_box": rectangle("-88.069295,42.930385 -87.948504,42.988186") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5531175, "cityName": "Greenfield" } }
+{ "create_at": datetime("2016-01-02T10:00:24.000Z"), "id": 683347121353830400, "text": "Last �� https://t.co/9I12gX02rQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2951384420, "name": "Kristèn .", "screen_name": "Whitegalkris", "lang": "en", "location": "null", "create_at": date("2014-12-29"), "description": "Follow Me On Instagram @Whitegalkris__✈️ AMOSC Whitegalkris⌛️", "followers_count": 1750, "friends_count": 986, "statues_count": 15910 }, "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-02T10:00:24.000Z"), "id": 683347121425088512, "text": "I love when girls dress up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2906718100, "name": "BVNKROLL BEEZY", "screen_name": "_rbeezy_", "lang": "en", "location": "null", "create_at": date("2014-12-05"), "description": "i cant change at all ♎️ GOLDEN VOICE", "followers_count": 746, "friends_count": 769, "statues_count": 3453 }, "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-02T10:00:24.000Z"), "id": 683347121546915844, "text": "This was my favorite part of 2015 by god I LOVE #NXT @TheWWEBlake https://t.co/tN0iaZCIPc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NXT" }}, "user_mentions": {{ 1705382556 }}, "user": { "id": 30223633, "name": "MaryAnn Mason", "screen_name": "gravenbabies", "lang": "en", "location": "Lakeland,Fl ", "create_at": date("2009-04-10"), "description": "NXT Stalker & Marker of Gravenbabies Horror Dolls.All photos copyrighted @gravenbabies DON'T STEAL MY PICS", "followers_count": 1623, "friends_count": 270, "statues_count": 31476 }, "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-01-02T10:00:24.000Z"), "id": 683347121693679618, "text": "@Daveybaby49 I apologize for my sloppy self and and also demanding spaghetti", "in_reply_to_status": -1, "in_reply_to_user": 317134702, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 317134702 }}, "user": { "id": 372302332, "name": "sharkbait", "screen_name": "monica_lnees", "lang": "en", "location": "Florida, USA", "create_at": date("2011-09-12"), "description": "null", "followers_count": 367, "friends_count": 244, "statues_count": 9394 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Wales, PA", "id": "19c3652cfd1c0c1b", "name": "North Wales", "place_type": "city", "bounding_box": rectangle("-75.284446,40.203558 -75.264241,40.219245") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42091, "countyName": "Montgomery", "cityID": 4255512, "cityName": "North Wales" } }
+{ "create_at": datetime("2016-01-02T10:00:24.000Z"), "id": 683347121781641218, "text": "I say lets all skip practice number 2��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2588941825, "name": ".kristian.", "screen_name": "kristian_dawwn", "lang": "en", "location": "Frederick, OK", "create_at": date("2014-06-25"), "description": "|FHS| •James 1:2-4••Jarrett Phillips•", "followers_count": 474, "friends_count": 533, "statues_count": 2837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Frederick, OK", "id": "14ccf074980d5cd9", "name": "Frederick", "place_type": "city", "bounding_box": rectangle("-99.029438,34.376597 -98.993478,34.409456") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40141, "countyName": "Tillman", "cityID": 4027800, "cityName": "Frederick" } }
+{ "create_at": datetime("2016-01-02T10:00:24.000Z"), "id": 683347121815171073, "text": "@TwickiB New Year New Priorities New Outlook OUT With The Old New Goals Growth", "in_reply_to_status": 683346067232964609, "in_reply_to_user": 563944543, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 563944543 }}, "user": { "id": 174143478, "name": "Tommy", "screen_name": "NautieWon", "lang": "en", "location": "null", "create_at": date("2010-08-02"), "description": "~~ La faveur de Dieu ~~", "followers_count": 780, "friends_count": 545, "statues_count": 26581 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bothell, WA", "id": "a285b156487ecf55", "name": "Bothell", "place_type": "city", "bounding_box": rectangle("-122.248238,47.732453 -122.163654,47.814554") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5307380, "cityName": "Bothell" } }
+{ "create_at": datetime("2016-01-02T10:00:24.000Z"), "id": 683347121882443776, "text": "LETS GO JACK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1546465909, "name": "Kat", "screen_name": "PINENIPPLESS", "lang": "en", "location": "South Florida", "create_at": date("2013-06-25"), "description": "Miami sports & wine fanatic", "followers_count": 4616, "friends_count": 597, "statues_count": 178356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pine Manor, FL", "id": "bb3c878553134384", "name": "Pine Manor", "place_type": "city", "bounding_box": rectangle("-81.884278,26.567963 -81.871876,26.581244") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12071, "countyName": "Lee", "cityID": 1257025, "cityName": "Pine Manor" } }
+{ "create_at": datetime("2016-01-02T10:00:24.000Z"), "id": 683347121899237380, "text": "Planet Fitness is so packed!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 115757110, "name": "Charli Baltimore", "screen_name": "callme_ren", "lang": "en", "location": "тeхaѕ & new jerѕey", "create_at": date("2010-02-19"), "description": "ѕenιor | ғυтυre denтιѕт | ĸeep waтcнιng", "followers_count": 7903, "friends_count": 2092, "statues_count": 101893 }, "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-01-02T10:00:24.000Z"), "id": 683347121957810176, "text": "My dinner party is tonight ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2734509730, "name": "rip sarde ❤️", "screen_name": "pearlswithjays", "lang": "en", "location": "#ImSoPortArthur ", "create_at": date("2014-08-05"), "description": "http://www.hudl.com/athlete/5541413/highlights/304430386/v2# Go look at my brother highlights", "followers_count": 1399, "friends_count": 1330, "statues_count": 14186 }, "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-02T10:00:24.000Z"), "id": 683347122125541377, "text": "like your life https://t.co/8Qi8F5BXB0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2863496420, "name": "Big Daddy", "screen_name": "iamUSELESSSS", "lang": "en", "location": "ROCKIN THREW TEXAS", "create_at": date("2014-10-18"), "description": "WARNING : DONT COME FOR ME! PLEASE! IM WARNING YOU ☹ rhs18'", "followers_count": 347, "friends_count": 280, "statues_count": 15040 }, "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-02T10:00:24.000Z"), "id": 683347122192650240, "text": "@julestolents ehe ehe�� enebe ehe hende nemen teme leng ehe ehe", "in_reply_to_status": 683346864045830144, "in_reply_to_user": 3848418733, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 3848418733 }}, "user": { "id": 1690868456, "name": "apple", "screen_name": "iamkylanicoleee", "lang": "en", "location": "Montana, USA", "create_at": date("2013-08-22"), "description": "God is good, all the time!", "followers_count": 1095, "friends_count": 1032, "statues_count": 22780 }, "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.083903 -104.676843,47.136719") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30021, "countyName": "Dawson", "cityID": 3031450, "cityName": "Glendive" } }
+{ "create_at": datetime("2016-01-02T10:00:24.000Z"), "id": 683347122192793601, "text": "Focused on growing my faith and making moves 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": 954464599, "name": "molly jane mitchell", "screen_name": "ItsMollyMitch", "lang": "en", "location": "null", "create_at": date("2012-11-17"), "description": "Get baked at the beach • Sunkissed Tanning", "followers_count": 804, "friends_count": 557, "statues_count": 9007 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fenwick Island, DE", "id": "5f1c213c381b1a3b", "name": "Fenwick Island", "place_type": "city", "bounding_box": rectangle("-75.058406,38.452831 -75.048715,38.467491") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10005, "countyName": "Sussex", "cityID": 1026880, "cityName": "Fenwick Island" } }
+{ "create_at": datetime("2016-01-02T10:00:24.000Z"), "id": 683347122419150848, "text": "I need a fucking forklift to get my ass out of bed 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": 4340968454, "name": "Ash", "screen_name": "ashley_nic5", "lang": "en", "location": "null", "create_at": date("2015-11-23"), "description": "Mommy to Audriegh ❤️ M.A.C Artist", "followers_count": 95, "friends_count": 95, "statues_count": 71 }, "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-01-02T10:00:24.000Z"), "id": 683347122444439552, "text": "@flatisjustiice @SayaTheFox or the ones that say, 'I'll be right back.'", "in_reply_to_status": 683328293081329664, "in_reply_to_user": 3429906503, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3429906503, 2880860322 }}, "user": { "id": 1646979768, "name": "Snarfy Snarf", "screen_name": "ben_shoten1979", "lang": "en", "location": "Richmond, VA", "create_at": date("2013-08-04"), "description": "I don't know much about myself but I do know I'm a big gamer. In love with anime, manga, video games & the people who bring them to life (cosplayers).\n\nNyahaha!", "followers_count": 49, "friends_count": 261, "statues_count": 3065 }, "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-02T10:00:24.000Z"), "id": 683347122662473728, "text": "Partly cloudy this afternoon, high 51 (11 C). Low 37 (3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119753180, "name": "Jackson Weather", "screen_name": "JacksonWX", "lang": "en", "location": "Jackson, MS", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Jackson, MS", "followers_count": 295, "friends_count": 68, "statues_count": 6368 }, "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-02T10:00:24.000Z"), "id": 683347122691948546, "text": "#HolaSebastian tweeteame una selfie tuya de ahora! @SEBTSB x28", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "HolaSebastian" }}, "user_mentions": {{ 1631128196 }}, "user": { "id": 2621545545, "name": "sarah", "screen_name": "confusemendes", "lang": "en", "location": "New York, USA", "create_at": date("2014-06-18"), "description": "* ✧ · . ☾ *| njh | jw | sm |* ✧ ·. ☾ *", "followers_count": 2704, "friends_count": 2079, "statues_count": 22268 }, "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-01-02T10:00:24.000Z"), "id": 683347122888945664, "text": "Production Planner/Scheduler, Columbus, OH - Philips North America: (#Columbus, OH) https://t.co/4RrIMKM7b5 #SupplyChain #Job #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": {{ "Columbus", "SupplyChain", "Job", "Jobs", "Hiring" }}, "user": { "id": 188185944, "name": "ColumbusOH S-Chain", "screen_name": "tmj_CHH_schn", "lang": "en", "location": "Columbus, OH", "create_at": date("2010-09-07"), "description": "Follow this account for geo-targeted Supply Chain job tweets in Columbus, OH from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 284, "friends_count": 264, "statues_count": 31 }, "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-02T10:00:24.000Z"), "id": 683347123056721920, "text": "AOC to is my second home ��", "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": 370, "friends_count": 371, "statues_count": 2296 }, "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-02T10:00:24.000Z"), "id": 683347123195215873, "text": "������ https://t.co/L8rcfzPeFo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 368429785, "name": "ℱábíε ツ", "screen_name": "iAmSec0nd_", "lang": "en", "location": "F L O R I D A ", "create_at": date("2011-09-05"), "description": "I know that nothing good dwells in me,that is, my flesh. For I have the desire to do whats right but not the ability to carry it out. Romans 7:18 #GodIsGreater", "followers_count": 1536, "friends_count": 945, "statues_count": 83108 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Byron, GA", "id": "9c4add4563eac2c7", "name": "Byron", "place_type": "city", "bounding_box": rectangle("-83.789593,32.606493 -83.708924,32.685579") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13225, "countyName": "Peach", "cityID": 1312260, "cityName": "Byron" } }
+{ "create_at": datetime("2016-01-02T10:00:24.000Z"), "id": 683347123580964864, "text": "See our latest #Cincinnati, OH #job and click to apply: Technical Product Manager - Senior (Near a Siemens... - https://t.co/oCwMw0xhH6", "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": {{ "Cincinnati", "job" }}, "user": { "id": 43709923, "name": "TMJ-USA PM Jobs", "screen_name": "tmj_usa_prod", "lang": "en", "location": "USA", "create_at": date("2009-05-31"), "description": "Follow this account for geo-targeted Product Management job tweets in USA. Need help? Tweet us at @CareerArc!", "followers_count": 287, "friends_count": 179, "statues_count": 731 }, "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-02T10:00:24.000Z"), "id": 683347123694219266, "text": "@RGRDave @justinred87 @ZOMB13_K1LL3R @ZFGNation @NerdDelegation @3seven0z", "in_reply_to_status": 683339353377001472, "in_reply_to_user": 336517351, "favorite_count": 0, "coordinate": point("-96.31172833,30.659445"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 336517351, 3623402892, 1449349878, 4362947054, 3005612941, 120317902 }}, "user": { "id": 336517351, "name": "Ranger Known as Dave", "screen_name": "RGRDave", "lang": "en", "location": "Gainesville, GA", "create_at": date("2011-07-16"), "description": "Bio you can't get my Bio..... But you should follow @zfgnation as well.", "followers_count": 67, "friends_count": 82, "statues_count": 1331 }, "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-02T10:00:24.000Z"), "id": 683347123715325953, "text": "#brunch #lulu #grovelife #miami #miamilife #familyfirst #friends #foodporn #foodie #HappyNewYear… https://t.co/ywsOHmmBnu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.2438736,25.7271194"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "brunch", "lulu", "grovelife", "miami", "miamilife", "familyfirst", "friends", "foodporn", "foodie", "HappyNewYear" }}, "user": { "id": 92895881, "name": "nadia asencio", "screen_name": "newyorknadia13", "lang": "en", "location": "New Yawk, New Yawk ", "create_at": date("2009-11-26"), "description": "#Radio host (Evolution 93.5), #author (#POLITIQUETTE), #MiamiTOX host on #YouTube, and all around #RIOT...I love you; also, can I borrow $20?", "followers_count": 463, "friends_count": 401, "statues_count": 5916 }, "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-02T10:00:24.000Z"), "id": 683347123732115459, "text": "@Vronos @SaffronOlive except the flavor is a jumble due to prev colored drones and never really explaining eldrazi/mana/color relationship", "in_reply_to_status": 683346512663859200, "in_reply_to_user": 867402332, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 867402332, 1306129981 }}, "user": { "id": 72880154, "name": "Andrew Rudiger", "screen_name": "nostrangenames", "lang": "en", "location": "null", "create_at": date("2009-09-09"), "description": "Husband. Father. MtG L1 Judge. Attorney. Occasional tweeter. UWb MTGO: Visla BattleNet: Visla#1508", "followers_count": 83, "friends_count": 223, "statues_count": 3970 }, "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-02T10:00:24.000Z"), "id": 683347123983806464, "text": "@DetroitRedWings @flagstar Tatar #DRWFirstGoal", "in_reply_to_status": 683332958464393216, "in_reply_to_user": 16826656, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "hashtags": {{ "DRWFirstGoal" }}, "user_mentions": {{ 16826656, 19042958 }}, "user": { "id": 548006784, "name": "Tim Osterman", "screen_name": "TimOsterman", "lang": "en", "location": "Western Michigan University ", "create_at": date("2012-04-07"), "description": "Its the way she goes sometimes she goes sometimes she doesn't it's just the way she goes, US Army, Psalm 23:4", "followers_count": 325, "friends_count": 321, "statues_count": 3340 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cutlerville, MI", "id": "d4c37349a0a22355", "name": "Cutlerville", "place_type": "city", "bounding_box": rectangle("-85.703328,42.805038 -85.633255,42.855744") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26081, "countyName": "Kent", "cityID": 2619500, "cityName": "Cutlerville" } }
+{ "create_at": datetime("2016-01-02T10:00:25.000Z"), "id": 683347124063449088, "text": "I did not wanna get up that early but once my baby said he was hungry j hopped right up �� never let my baby boy feel hunger", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 183825210, "name": "Ashleyy", "screen_name": "theyLOVEash_xo", "lang": "en", "location": "New York", "create_at": date("2010-08-27"), "description": "Some sweet psychotic 22 year old girl who likes to eat. @myfabolouslife is babe. instagram-lovelylifeofashxo. my DM's is broke ✌", "followers_count": 1457, "friends_count": 938, "statues_count": 54445 }, "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-02T10:00:25.000Z"), "id": 683347124159823872, "text": "Check out my #listing in #FederalDam #MN https://t.co/vvOWhFxcIo #realestate #realtor https://t.co/htlVZk500J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.257666,47.251993"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "FederalDam", "MN", "realestate", "realtor" }}, "user": { "id": 214631153, "name": "Sean M. McDonald", "screen_name": "realtorSeanMac", "lang": "en", "location": "Longville, MN", "create_at": date("2010-11-11"), "description": "25 years of full time real estate service, moved to my long time favorite area in the world, Longville MN., and enjoying life!", "followers_count": 50, "friends_count": 51, "statues_count": 1581 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minnesota, USA", "id": "9807c5c5f7a2c6ce", "name": "Minnesota", "place_type": "admin", "bounding_box": rectangle("-97.239256,43.499362 -89.483385,49.384359") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27021, "countyName": "Cass", "cityID": 2720798, "cityName": "Federal Dam" } }
+{ "create_at": datetime("2016-01-02T10:00:25.000Z"), "id": 683347124247867392, "text": "See our latest #Wilmington, NC #job and click to apply: Advanced Medical Support Assistant,... - https://t.co/lkT3PQwdZM #Healthcare", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.9447102,34.2257255"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Wilmington", "job", "Healthcare" }}, "user": { "id": 23279706, "name": "TM-WIL Health Jobs", "screen_name": "tmj_wil_health", "lang": "en", "location": "Wilmington, NC", "create_at": date("2009-03-07"), "description": "Follow this account for geo-targeted Healthcare job tweets in Wilmington, NC. Need help? Tweet us at @CareerArc!", "followers_count": 352, "friends_count": 303, "statues_count": 43 }, "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.32187,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37129, "countyName": "New Hanover", "cityID": 3774440, "cityName": "Wilmington" } }
+{ "create_at": datetime("2016-01-02T10:00:25.000Z"), "id": 683347124444991488, "text": "@ch33rio_ tmb", "in_reply_to_status": -1, "in_reply_to_user": 3353628792, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3353628792 }}, "user": { "id": 1561707458, "name": "tydolla$ign", "screen_name": "TyvonRashaud", "lang": "en", "location": "Omaha, NE", "create_at": date("2013-07-01"), "description": "yo nigga tryna get his curls like mine❌", "followers_count": 472, "friends_count": 185, "statues_count": 11126 }, "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-02T10:00:25.000Z"), "id": 683347124629573632, "text": "taper gang from nah on...no more philly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2369895275, "name": "⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀", "screen_name": "WAVY_CO", "lang": "en", "location": "Baton Rouge, LA", "create_at": date("2014-02-27"), "description": "null", "followers_count": 361, "friends_count": 41, "statues_count": 60434 }, "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-02T10:00:25.000Z"), "id": 683347124763897856, "text": "Maki // I hate Carter but I love you so we are nice mutuals�� we used to talk but you totally forgot about me so bye", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3014229517, "name": "lud loves johnson", "screen_name": "flirtingjacks", "lang": "en", "location": "born in Buenos Aires | Cali ", "create_at": date("2015-02-09"), "description": "I jumped on gilinsky and we fell off sorry dude | shawnguitars loml", "followers_count": 4082, "friends_count": 408, "statues_count": 2450 }, "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-01-02T10:00:25.000Z"), "id": 683347124973600768, "text": "Wind 4.0 mph W. Barometer 30.086 in, Falling. Temperature 32.2 °F. Rain today 0.00 in. Humidity 77%", "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": 287, "statues_count": 7201 }, "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-02T10:00:25.000Z"), "id": 683347125242052609, "text": "We're #hiring! Read about our latest #job opening here: Valet/Bellperson- Full-Time - https://t.co/TPggFwe3VJ #ChapelHill, NC #Hospitality", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.0558445,35.9131996"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "ChapelHill", "Hospitality" }}, "user": { "id": 22452620, "name": "TMJ-RDU HRTA Jobs", "screen_name": "tmj_rdu_hrta", "lang": "en", "location": "RDU, NC", "create_at": date("2009-03-01"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Raleigh/Durham, NC. Need help? Tweet us at @CareerArc!", "followers_count": 412, "friends_count": 292, "statues_count": 479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chapel Hill, NC", "id": "23ebe811ea93fd41", "name": "Chapel Hill", "place_type": "city", "bounding_box": rectangle("-79.083153,35.869376 -78.968887,35.969507") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37135, "countyName": "Orange", "cityID": 3711800, "cityName": "Chapel Hill" } }
+{ "create_at": datetime("2016-01-02T10:00:25.000Z"), "id": 683347125367787521, "text": "Actually pretty sad that E has to go back to fort dodge today ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 514620440, "name": "Nicole", "screen_name": "nicoleebradleyy", "lang": "en", "location": "Ames, Iowa", "create_at": date("2012-03-04"), "description": "Everyone you meet is fighting a battle you know nothing about. Be kind. Always. | 19", "followers_count": 269, "friends_count": 175, "statues_count": 11651 }, "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-02T10:00:25.000Z"), "id": 683347125384704000, "text": "Lunch!!! (at @BSpotBurgers in Woodmere, OH) https://t.co/VrRzjiiz5r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.47632599,41.46302254"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 116815354 }}, "user": { "id": 261715088, "name": "Melissa Fairman SPHR", "screen_name": "HrRemix", "lang": "en", "location": "Cleveland, Ohio", "create_at": date("2011-03-06"), "description": "HR Pro. Blogger. Dreamer. Writer. Musician. Persistent Optimist. Dog Lover. Avid Reader.", "followers_count": 13781, "friends_count": 5743, "statues_count": 15299 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodmere, OH", "id": "f10d8719c0027745", "name": "Woodmere", "place_type": "city", "bounding_box": rectangle("-81.488244,41.455564 -81.473327,41.464059") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3986394, "cityName": "Woodmere" } }
+{ "create_at": datetime("2016-01-02T10:00:25.000Z"), "id": 683347125426491393, "text": "Every state got some \"ugly ass\" girls. But come to Houston & your opinion on Texas will change G. https://t.co/ZpfHvc8nmr", "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": "w lil J", "create_at": date("2014-08-15"), "description": "@partyomo | ig:@ mamiemz | happy af.", "followers_count": 1402, "friends_count": 969, "statues_count": 30502 }, "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-02T10:00:25.000Z"), "id": 683347125544030208, "text": "Doin early mornings the right way https://t.co/1dqOnyVXa4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 420117141, "name": "kayla skipper", "screen_name": "TheKaylaSkipper", "lang": "en", "location": "Tampa fl", "create_at": date("2011-11-23"), "description": "In the end, love will be what saves us all.", "followers_count": 292, "friends_count": 299, "statues_count": 11439 }, "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-01-02T10:00:25.000Z"), "id": 683347125644734464, "text": "Interesting way to eat a banana! #JRMac ���� Headed home from a fantastic trip! @ I 95 South Bound https://t.co/tGig3nXF4v", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.70608519,28.08800138"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "JRMac" }}, "user": { "id": 31503126, "name": "Eva Connors McMullin", "screen_name": "evaconnors", "lang": "en", "location": "www.srperrott.com", "create_at": date("2009-04-15"), "description": "VP of @micheleandgroup nationwide Modeling & Talent Agency and Asst GM @srperrott a Beer & Beverage Wholesaler in the Daytona Beach, FL area, wife & a mommy!", "followers_count": 1253, "friends_count": 1040, "statues_count": 12981 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Melbourne, FL", "id": "e49edeea1a3526b1", "name": "West Melbourne", "place_type": "city", "bounding_box": rectangle("-80.706103,28.03518 -80.637804,28.103875") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12009, "countyName": "Brevard", "cityID": 1276500, "cityName": "West Melbourne" } }
+{ "create_at": datetime("2016-01-02T10:00:25.000Z"), "id": 683347126059843584, "text": "Cloudy this afternoon, high 49 (9 C). Low 38 (3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119754993, "name": "Waco Weather", "screen_name": "WacoWeather", "lang": "en", "location": "Waco, TX", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Waco, TX", "followers_count": 816, "friends_count": 68, "statues_count": 6349 }, "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.069786,31.639828") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4876000, "cityName": "Waco" } }
+{ "create_at": datetime("2016-01-02T10:00:25.000Z"), "id": 683347126064054272, "text": "Nice work @blais_louise ! https://t.co/QfLMNA64C3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 578734691 }}, "user": { "id": 251024167, "name": "Marcy Grossman", "screen_name": "MarcyGrossman", "lang": "en", "location": "Denver", "create_at": date("2011-02-12"), "description": "Proudly representing Canada in the US Rocky Mountain region. Views expressed are my own.", "followers_count": 1337, "friends_count": 1210, "statues_count": 6017 }, "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-02T10:00:25.000Z"), "id": 683347126210826240, "text": "@24Reilly I should've huh", "in_reply_to_status": 683347000188796928, "in_reply_to_user": 1967078718, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1967078718 }}, "user": { "id": 1325319096, "name": "a$h", "screen_name": "asssssslyn", "lang": "en", "location": "null", "create_at": date("2013-04-03"), "description": "Jiggy like Madonna but trippy like Nirvana ☻", "followers_count": 1422, "friends_count": 405, "statues_count": 71699 }, "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-02T10:00:25.000Z"), "id": 683347126269644800, "text": "Me the next 3 months https://t.co/fCeppz8MJu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 268009699, "name": "BRADY BIRKEL", "screen_name": "bradybirkel", "lang": "en", "location": "null", "create_at": date("2011-03-17"), "description": "I am who I am. I like what I like. I love what I love. I do what I want. Get off my back and deal with it. It's my life, not yours.", "followers_count": 9123, "friends_count": 6826, "statues_count": 15426 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ironton, OH", "id": "005db8db2559283e", "name": "Ironton", "place_type": "city", "bounding_box": rectangle("-82.716209,38.499248 -82.641339,38.558635") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39087, "countyName": "Lawrence", "cityID": 3937464, "cityName": "Ironton" } }
+{ "create_at": datetime("2016-01-02T10:00:25.000Z"), "id": 683347126630375425, "text": "Wind 3 mph NNE. Barometer 29.93 in, Steady. Temperature 69.1 °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": 51, "friends_count": 70, "statues_count": 24681 }, "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-02T10:00:25.000Z"), "id": 683347126814834688, "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": 61366106, "name": "Kenny Bout It, Not U", "screen_name": "Flyer_thanu2", "lang": "en", "location": "Chicago/Blue Island, IL", "create_at": date("2009-07-29"), "description": "28 years old #80sBaby I'm only here for laughs, sports, and music. Forget a debate #49ers #Lakers #OKC Instagram: Kenny_DatDude", "followers_count": 4939, "friends_count": 3583, "statues_count": 549783 }, "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-01-02T10:00:25.000Z"), "id": 683347127402008576, "text": "2016. Body count reset. Virgin again. ����", "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.", "followers_count": 3759, "friends_count": 558, "statues_count": 152407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-02T10:00:25.000Z"), "id": 683347127578136576, "text": "Yeah, poor little rich girl\nCries her eyes out, it's all part of the show.����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 492843605, "name": "Bodie Johnson", "screen_name": "bodej9", "lang": "en", "location": "Abilene, KS ➡️ Manhattan, KS", "create_at": date("2012-02-14"), "description": "Girl you're thicker than a bowl of oatmeal.", "followers_count": 1177, "friends_count": 708, "statues_count": 20118 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Abilene, KS", "id": "8c381035170f8763", "name": "Abilene", "place_type": "city", "bounding_box": rectangle("-97.251381,38.904411 -97.199353,38.944452") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20041, "countyName": "Dickinson", "cityID": 2000125, "cityName": "Abilene" } }
+{ "create_at": datetime("2016-01-02T10:00:25.000Z"), "id": 683347127632805888, "text": "Lions and tigers and bears oh my! @loganElizabeth_ @KathrynHuey https://t.co/xbcoO5lMMy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 878965442, 563820528 }}, "user": { "id": 614509300, "name": "kenna ❂", "screen_name": "MckennaHaile", "lang": "en", "location": "null", "create_at": date("2012-06-21"), "description": "I'm never in my right mind, but im einstein when i do think. sc | mckenna_lh", "followers_count": 641, "friends_count": 898, "statues_count": 7457 }, "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-02T10:00:25.000Z"), "id": 683347128081604608, "text": "Morning y'all! https://t.co/Z9QZD1xkfZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1941871848, "name": "Ryan Lumbberjacx", "screen_name": "RyanLumbberjacx", "lang": "en", "location": "Niagara Falls", "create_at": date("2013-10-06"), "description": "#NSFW & #X-rated. environmental designer/urbanist. #gay #lumbersexual #truvadawhore #bbbh #teamtop #bareback. semiretired #escort. The 1 momma warned you about.", "followers_count": 841, "friends_count": 259, "statues_count": 875 }, "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-02T10:00:25.000Z"), "id": 683347128111001600, "text": "when we were young", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2770309918, "name": "J O S H U A", "screen_name": "officialclvssic", "lang": "en", "location": "null", "create_at": date("2014-09-14"), "description": "dancer | actor", "followers_count": 127, "friends_count": 46, "statues_count": 5075 }, "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-02T10:00:26.000Z"), "id": 683347128433815554, "text": "If you're a #Education professional in #Portland, OR, check out this #job: https://t.co/dHzKGf8QXz #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.6762071,45.5234515"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Education", "Portland", "job", "Hiring", "CareerArc" }}, "user": { "id": 21838455, "name": "TMJ-PDX Educ. Jobs", "screen_name": "tmj_pdx_edu", "lang": "en", "location": "Portland, OR", "create_at": date("2009-02-24"), "description": "Follow this account for geo-targeted Education/Teaching job tweets in Portland, OR. Need help? Tweet us at @CareerArc!", "followers_count": 353, "friends_count": 235, "statues_count": 31 }, "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-02T10:00:26.000Z"), "id": 683347128433926145, "text": "Somethings I do might look worldly. Sometimes they will be. I'm only flesh, no man's righteous except through the blood of Christ.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.8875164,32.7788386"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 109597619, "name": "M.K", "screen_name": "Whiteboi1024", "lang": "en", "location": "Macon, GA", "create_at": date("2010-01-29"), "description": "Ex-convict. I believe God makes my past irrelevant. Family, Love, Loyalty../Writer/Artist(f.k.a CroSs) founder of Get-Rich-INDependently House Productions.", "followers_count": 1747, "friends_count": 1832, "statues_count": 1972 }, "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": 13021, "countyName": "Bibb", "cityID": 1349008, "cityName": "Macon-Bibb County" } }
+{ "create_at": datetime("2016-01-02T10:00:26.000Z"), "id": 683347128538787841, "text": "@LuceroMusic & @JohnMorelandOK live in Ohio 4/9/16 -- Hey @RoyFnBerry any otther shows like this? https://t.co/k6ywHgZTKe", "in_reply_to_status": -1, "in_reply_to_user": 24752568, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24752568, 39365181, 43076888 }}, "user": { "id": 76921999, "name": "Bobby Wilbert", "screen_name": "bobbyw24", "lang": "en", "location": "Jacksonille, Fla.", "create_at": date("2009-09-24"), "description": "#Bankruptcy #Lawyer in Fla. & Md. #GaslightAnthem, #LuceroMusic #Libertarian #Springsteen #EStreetFans #WebbNation Consumer rights - English & broken German", "followers_count": 1956, "friends_count": 2385, "statues_count": 21637 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jacksonville Beach, FL", "id": "5e281c17a74c170f", "name": "Jacksonville Beach", "place_type": "city", "bounding_box": rectangle("-81.429213,30.252059 -81.380497,30.307701") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12031, "countyName": "Duval", "cityID": 1235050, "cityName": "Jacksonville Beach" } }
+{ "create_at": datetime("2016-01-02T10:00:26.000Z"), "id": 683347128542834688, "text": "Ridem Cowboy https://t.co/LNqB5ysu01", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.333153,32.759588"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 143311525, "name": "TBL Tech Nerds", "screen_name": "TBLAndroidGames", "lang": "en", "location": "New York City", "create_at": date("2010-05-12"), "description": "Android app, iOS app and Mobile Website Development Experts.", "followers_count": 26403, "friends_count": 3651, "statues_count": 2924 }, "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-02T10:00:26.000Z"), "id": 683347128618336256, "text": "Sunny this afternoon, high 44 (7 C). Low 29 (-2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119755728, "name": "Louisville Weather", "screen_name": "Louisville_WX", "lang": "en", "location": "Louisville, KY", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Louisville, KY", "followers_count": 1308, "friends_count": 68, "statues_count": 6375 }, "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-01-02T10:00:26.000Z"), "id": 683347128777859072, "text": "metalparkny very cool metal sculptures! #metalpark #holidayshops #darlek #doctorwho @ Bryant Park https://t.co/PgwC2i08xD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.98373145,40.75391862"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "metalpark", "holidayshops", "darlek", "doctorwho" }}, "user": { "id": 191678790, "name": "Alix-Anne Bush", "screen_name": "xanneb", "lang": "en", "location": "Washington DC", "create_at": date("2010-09-16"), "description": "gymnast, ballet dancer, singer, theatre lover! ...currently living the life of a techie!", "followers_count": 68, "friends_count": 249, "statues_count": 661 }, "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-02T10:00:26.000Z"), "id": 683347128815628288, "text": "\"The Holy Ghost is the only ghost welcome here.\" #PawleysIsland https://t.co/lwSlxpCxRp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PawleysIsland" }}, "user": { "id": 713465186, "name": "Katherine Ballard", "screen_name": "kateoballard", "lang": "en", "location": "Washington, DC", "create_at": date("2012-07-23"), "description": "Foreign Policy", "followers_count": 425, "friends_count": 1202, "statues_count": 4016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pawleys Island, SC", "id": "0098fc7547232eef", "name": "Pawleys Island", "place_type": "city", "bounding_box": rectangle("-79.187234,33.393435 -79.096351,33.490575") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45043, "countyName": "Georgetown", "cityID": 4555015, "cityName": "Pawleys Island" } }
+{ "create_at": datetime("2016-01-02T10:00:26.000Z"), "id": 683347128941346816, "text": "Hate that @jalexander_6ix is a last minute person ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3281957683 }}, "user": { "id": 820665246, "name": "Karisma Ramirez", "screen_name": "SimplyKarismaa", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2012-09-12"), "description": "Family Over Everything ❤", "followers_count": 554, "friends_count": 415, "statues_count": 12986 }, "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-02T10:00:26.000Z"), "id": 683347128966590464, "text": "I always find shit out. I might not say it right away, But 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": 2802989170, "name": "ℑεηη☯", "screen_name": "jmorrrx", "lang": "en", "location": "New Jersey, USA", "create_at": date("2014-10-03"), "description": "21| nj | employed", "followers_count": 1345, "friends_count": 2749, "statues_count": 6103 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westfield, NJ", "id": "777dd5b3b30a625e", "name": "Westfield", "place_type": "city", "bounding_box": rectangle("-74.379383,40.624437 -74.313149,40.682332") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3479040, "cityName": "Westfield" } }
+{ "create_at": datetime("2016-01-02T10:00:26.000Z"), "id": 683347129645965314, "text": "Wind 4 mph SE. Barometer 1011.6 hPa, Falling. Temperature 91.4 °F. Rain today 0.00 in. Humidity 10%", "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": 87, "friends_count": 265, "statues_count": 139957 }, "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-02T10:00:26.000Z"), "id": 683347129688010752, "text": "I find myself so beautiful that on the rare occurrence of someone calling me ugly I just think they're dead lying. Lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3245305351, "name": "Demonican", "screen_name": "goyadelrey", "lang": "en", "location": "NYC", "create_at": date("2015-06-14"), "description": "Lisa Bonet and Selma Hayek's love child.", "followers_count": 1754, "friends_count": 995, "statues_count": 40488 }, "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-02T10:00:26.000Z"), "id": 683347129985810432, "text": "Shopping w/ granny today ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2670173578, "name": "#unfuckwitable", "screen_name": "_0hKaaay__", "lang": "en", "location": "4ever w/ S'Nyah", "create_at": date("2014-07-03"), "description": "Ꮗнѕ||мσѕт нαтє∂||υивσтнєяє∂||bffmfl||fαν ¢συѕιи||410 BITCH||", "followers_count": 2130, "friends_count": 2171, "statues_count": 36493 }, "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-02T10:00:26.000Z"), "id": 683347130111541248, "text": "True!! https://t.co/0xn28OmzIj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 45169293, "name": "Brenda McKinley", "screen_name": "Brendamckinley", "lang": "en", "location": "Alaska", "create_at": date("2009-06-06"), "description": "44 yr old crazy alaskan, married to my best friend. We have 2 amazing kids. God has truly Blessed me!", "followers_count": 55, "friends_count": 116, "statues_count": 630 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmer, AK", "id": "6259f7d715663d4a", "name": "Palmer", "place_type": "city", "bounding_box": rectangle("-149.180191,61.552965 -149.071265,61.628282") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2170, "countyName": "Matanuska-Susitna", "cityID": 258660, "cityName": "Palmer" } }
+{ "create_at": datetime("2016-01-02T10:00:26.000Z"), "id": 683347130614968320, "text": "I'm so in love so deep 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": 1337262114, "name": "התגבשות", "screen_name": "karlycakinnn", "lang": "en", "location": "Kissimmee, FL", "create_at": date("2013-04-08"), "description": "null", "followers_count": 698, "friends_count": 381, "statues_count": 30485 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forest City, FL", "id": "1d5937470e1c08d7", "name": "Forest City", "place_type": "city", "bounding_box": rectangle("-81.459801,28.639911 -81.423037,28.680218") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12117, "countyName": "Seminole", "cityID": 1223375, "cityName": "Forest City" } }
+{ "create_at": datetime("2016-01-02T10:00:26.000Z"), "id": 683347130807939072, "text": "It's still effective https://t.co/m8SMdP3FaJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36548724, "name": "Trixie", "screen_name": "2charmss", "lang": "en", "location": "@chuck_taylord1", "create_at": date("2009-04-29"), "description": "I don't think I'm better than you cause I wear my hair natural & you wear a weave... I'm better than you cause you're a bum ass bitch & I'm not.", "followers_count": 6756, "friends_count": 514, "statues_count": 232357 }, "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-02T10:00:26.000Z"), "id": 683347130996645888, "text": "About to drive my dad to get his car from a party he couldn't drive back from. He ended up puking he partied so hard. Proud 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": 1129241856, "name": "Hannah Ziegman", "screen_name": "Harharnar", "lang": "en", "location": "Elyria/Bowling Green", "create_at": date("2013-01-28"), "description": "♊ |-/", "followers_count": 166, "friends_count": 143, "statues_count": 2895 }, "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-01-02T10:00:26.000Z"), "id": 683347131000877060, "text": "Brkami https://t.co/YOLaNAWOjK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 1963536865, "name": "French Vanilla", "screen_name": "badeaubrookie", "lang": "en", "location": "pcounty", "create_at": date("2013-10-15"), "description": "largo high school junior", "followers_count": 1109, "friends_count": 940, "statues_count": 38528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pinellas Park, FL", "id": "d598cb5ad6f7a09f", "name": "Pinellas Park", "place_type": "city", "bounding_box": rectangle("-82.752428,27.824633 -82.663128,27.897396") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1256975, "cityName": "Pinellas Park" } }
+{ "create_at": datetime("2016-01-02T10:00:26.000Z"), "id": 683347131059548160, "text": "12u FUTURE STARS finalists! #PowerShowcase https://t.co/8dWjPTJOlP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PowerShowcase" }}, "user": { "id": 3358959351, "name": "The Power Showcase", "screen_name": "PowerShowcaseWC", "lang": "en", "location": "Marlins Park in Miami, FL USA", "create_at": date("2015-07-04"), "description": "The official Twitter of The Power Showcase World Classic, featuring the worlds best hitters and rising stars in baseball. Instagram: @powershowcasewc", "followers_count": 229, "friends_count": 123, "statues_count": 158 }, "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-02T10:00:26.000Z"), "id": 683347131466264576, "text": "shoulda got some ink a long time ago. BEEN trying to finish my arm.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3788143279, "name": "big a", "screen_name": "AveOuttaTX", "lang": "en", "location": "REANA IS MY BABY.", "create_at": date("2015-10-04"), "description": "ㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤ⠀⠀⠀ㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤ ⠀king shit♔ㅤㅤㅤㅤㅤㅤㅤ⠀ ⠀[G]", "followers_count": 251, "friends_count": 167, "statues_count": 1506 }, "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-01-02T10:00:26.000Z"), "id": 683347131642568704, "text": "One word to describe my Florida trip? LIT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2303932184, "name": "Sierra Atteberry", "screen_name": "AtteberrySierra", "lang": "en", "location": "null", "create_at": date("2014-01-21"), "description": "instagram: that_blonde_chick09 Dylan's girl❤️ spread positivity✌senior at Clinton", "followers_count": 488, "friends_count": 630, "statues_count": 2760 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sanibel, FL", "id": "6741728259cc0032", "name": "Sanibel", "place_type": "city", "bounding_box": rectangle("-82.114016,26.42166 -82.013547,26.469357") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12071, "countyName": "Lee", "cityID": 1263700, "cityName": "Sanibel" } }
+{ "create_at": datetime("2016-01-02T10:00:26.000Z"), "id": 683347131663433729, "text": "@jferrucci_ thank you so much jar!�� miss you more!!", "in_reply_to_status": 683331415568728064, "in_reply_to_user": 311753290, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 311753290 }}, "user": { "id": 176681043, "name": "asap lyss", "screen_name": "lysssharvs", "lang": "en", "location": "null", "create_at": date("2010-08-09"), "description": "always strive and prosper", "followers_count": 371, "friends_count": 443, "statues_count": 20938 }, "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-02T10:00:26.000Z"), "id": 683347131697115136, "text": "@Eric_Dorris1 @jcachila16 wait he thinks we salty?", "in_reply_to_status": 683346106089115649, "in_reply_to_user": 1268768743, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1268768743, 3228614751 }}, "user": { "id": 161463465, "name": "Outback Bowl Champs", "screen_name": "cable18", "lang": "en", "location": "knoxville tn ", "create_at": date("2010-06-30"), "description": "Christian, Wife, Mother, VFL. Yea Mon!check out the NINDS for bell palsy and strokes.", "followers_count": 1151, "friends_count": 2022, "statues_count": 12513 }, "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-02T10:00:26.000Z"), "id": 683347131898421248, "text": "29 years young https://t.co/JtXvO1PcMT", "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": 1188, "friends_count": 584, "statues_count": 81435 }, "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-02T10:00:26.000Z"), "id": 683347132028456960, "text": "get ur shine on ☀️ @ Holden Beach https://t.co/T2Ny34CJ9h", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.29704054,33.91087647"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 62955235, "name": "e", "screen_name": "emmachristine9", "lang": "en", "location": "910 to 704", "create_at": date("2009-08-04"), "description": "it is what it is ~ UNCC ΚΔ", "followers_count": 1177, "friends_count": 1208, "statues_count": 60815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Holden Beach, NC", "id": "00fd022f7c61bddf", "name": "Holden Beach", "place_type": "city", "bounding_box": rectangle("-78.344153,33.911393 -78.257624,33.962863") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37019, "countyName": "Brunswick", "cityID": 3731960, "cityName": "Holden Beach" } }
+{ "create_at": datetime("2016-01-02T10:00:27.000Z"), "id": 683347132422733824, "text": "@auttuuumnn that's like minimum wage. Sick.", "in_reply_to_status": 683345668925030400, "in_reply_to_user": 379346469, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 379346469 }}, "user": { "id": 245505911, "name": "Dern Smith", "screen_name": "DerrinCLE", "lang": "en", "location": "Cleveland, Ohio", "create_at": date("2011-01-31"), "description": "Marketing the city I love, CLE. All views expressed here are mine. I'm vocal and don't like dumb things.", "followers_count": 555, "friends_count": 740, "statues_count": 29696 }, "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-01-02T10:00:27.000Z"), "id": 683347132430979072, "text": "If you're a #Sales professional in #Piscataway, NJ, check out this #job: https://t.co/44SBuOs36Y #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.4664185,40.5397172"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "Piscataway", "job", "Hiring" }}, "user": { "id": 3004246198, "name": "Colgate Jobs - U.S.", "screen_name": "ColgateJobsUS", "lang": "en", "location": "United States", "create_at": date("2015-01-29"), "description": "null", "followers_count": 1145, "friends_count": 30, "statues_count": 113 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Society Hill, NJ", "id": "63149539fc26b979", "name": "Society Hill", "place_type": "city", "bounding_box": rectangle("-74.471651,40.523186 -74.442662,40.548162") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3468304, "cityName": "Society Hill" } }
+{ "create_at": datetime("2016-01-02T10:00:27.000Z"), "id": 683347132531773443, "text": "I remember this �� https://t.co/5Jpl6dJbru", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2555549832, "name": "Streetz_offdamuscle", "screen_name": "alaqua95", "lang": "en", "location": "Matthews, NC", "create_at": date("2014-06-08"), "description": "704✈️910 #FINAO #FayState", "followers_count": 842, "friends_count": 807, "statues_count": 9392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Matthews, NC", "id": "827f88eab243bdf1", "name": "Matthews", "place_type": "city", "bounding_box": rectangle("-80.764486,35.081194 -80.653965,35.173287") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3741960, "cityName": "Matthews" } }
+{ "create_at": datetime("2016-01-02T10:00:27.000Z"), "id": 683347132850384896, "text": "@shanayaanaiz WELL TOO LATE MOW", "in_reply_to_status": 683344756693311488, "in_reply_to_user": 2286899700, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2286899700 }}, "user": { "id": 2361896072, "name": "Daniel CJ Garcia", "screen_name": "daanieldoee", "lang": "en", "location": "Houston, TX", "create_at": date("2014-02-25"), "description": "YFC htx☁️", "followers_count": 336, "friends_count": 320, "statues_count": 6581 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-02T10:00:27.000Z"), "id": 683347133282521088, "text": "@Zach_Dorsey have you ever fired anyone before?", "in_reply_to_status": -1, "in_reply_to_user": 251796499, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 251796499 }}, "user": { "id": 799176576, "name": "Stephanie ♡", "screen_name": "AMsBigClark", "lang": "en", "location": "New York, NY", "create_at": date("2012-09-02"), "description": "| | 4/4 | | 12.09.14 | 08.11.15 | 08.16.15 | 11.11.15 | 11.12.15 | @AustinMahone everyday", "followers_count": 1889, "friends_count": 988, "statues_count": 20638 }, "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-02T10:00:27.000Z"), "id": 683347133534203904, "text": "#VerbalDisturbance at 2600-2805 Castle Oak Ave, Pine Hills, FL 32808. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4591159,28.5793732"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VerbalDisturbance", "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": 213, "friends_count": 1, "statues_count": 75239 }, "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-02T10:00:27.000Z"), "id": 683347133685219328, "text": "@JRowe094 maybe he's not doing it incorrectly but you are ... you ever think of that Jeremiah?!??!", "in_reply_to_status": 683323455928795137, "in_reply_to_user": 263438791, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 263438791 }}, "user": { "id": 113567363, "name": "αmericαnℓαureℓ.störy", "screen_name": "testiculaurel", "lang": "en", "location": " →♡ @christa10b", "create_at": date("2010-02-11"), "description": "feelin' 22", "followers_count": 1204, "friends_count": 783, "statues_count": 730 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McGovern, PA", "id": "010f5a2b48665dc5", "name": "McGovern", "place_type": "city", "bounding_box": rectangle("-80.243973,40.212004 -80.17736,40.25042") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42125, "countyName": "Washington", "cityID": 4246160, "cityName": "McGovern" } }
+{ "create_at": datetime("2016-01-02T10:00:27.000Z"), "id": 683347133706063872, "text": "When I Lay On My Stomach My Titties Hurt 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": 3672349214, "name": "B R A T ❗️", "screen_name": "Bluuee__", "lang": "en", "location": "null", "create_at": date("2015-09-24"), "description": "K.L.M", "followers_count": 453, "friends_count": 272, "statues_count": 11995 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-02T10:00:27.000Z"), "id": 683347133928501249, "text": "�������������������� https://t.co/pFu5Kq0Ips", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2202054037, "name": "Luis Angel", "screen_name": "LuiggyD_", "lang": "en", "location": "Bronx NYC", "create_at": date("2013-11-18"), "description": "IG & SC kingluiggy", "followers_count": 316, "friends_count": 287, "statues_count": 37544 }, "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-02T10:00:27.000Z"), "id": 683347134771494912, "text": "I woke up and my knee hurts? Why?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3136645950, "name": "kenzie", "screen_name": "wtvrkenz_", "lang": "en", "location": "null", "create_at": date("2015-04-03"), "description": "enemies of the heir beware ✨✨ // probably not ohio // 21// reuben chip enthusiast // photo taker // mckenzie.holivay7@gmail.com // @Underdog_press // @soundpm", "followers_count": 504, "friends_count": 766, "statues_count": 31629 }, "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-02T10:00:27.000Z"), "id": 683347134943371266, "text": "Can you recommend anyone for this #SocialSciences #job? https://t.co/PU7uDxDEu3 #Columbia, MO #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.3340724,38.9517053"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SocialSciences", "job", "Columbia", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 904453002, "name": "MO Social Science", "screen_name": "tmj_mo_socsci", "lang": "en", "location": "Missouri Non-Metro", "create_at": date("2012-10-25"), "description": "Follow this account for geo-targeted Social Sciences job tweets in Missouri Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 92, "friends_count": 67, "statues_count": 14 }, "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": 29019, "countyName": "Boone", "cityID": 2915670, "cityName": "Columbia" } }
+{ "create_at": datetime("2016-01-02T10:00:27.000Z"), "id": 683347135060951041, "text": "#VerbalDisturbance at 600-623 Lee Rd. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3880189,28.6070708"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VerbalDisturbance", "orlpol", "ocso" }}, "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": 74, "friends_count": 1, "statues_count": 9686 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairview Shores, FL", "id": "d5c4f757d5df5fa0", "name": "Fairview Shores", "place_type": "city", "bounding_box": rectangle("-81.423995,28.582063 -81.366389,28.617568") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1221750, "cityName": "Fairview Shores" } }
+{ "create_at": datetime("2016-01-02T10:00:27.000Z"), "id": 683347135258095617, "text": "@TwEAT_Thez_Nuts making big bucks ���� I feel it", "in_reply_to_status": 683347006933315585, "in_reply_to_user": 39097833, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 39097833 }}, "user": { "id": 336435406, "name": "Javier Julio™", "screen_name": "RealDariusChat", "lang": "en", "location": "Cincinnati, OH", "create_at": date("2011-07-16"), "description": "RIP Mr. Willie RIP Uncle Ricky RIP Uncle Howard RIP Pops RIP Darryl RIP Grandma RIP Big Gary STAY STRONG DARIUS", "followers_count": 1264, "friends_count": 1985, "statues_count": 20949 }, "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-02T10:00:27.000Z"), "id": 683347135283134464, "text": "At Lamborghini Newport Beach — https://t.co/Wp2EEEoi4W", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.91676,33.68023"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 495273132, "name": "J • I • I", "screen_name": "jawad_i", "lang": "en", "location": "Tempe, AZ", "create_at": date("2012-02-17"), "description": "DEC/24 kik: J.I.I instagram : @jawad_alibrahim", "followers_count": 226, "friends_count": 156, "statues_count": 7978 }, "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-02T10:00:27.000Z"), "id": 683347135329210368, "text": "Resolution coming smoothly, Domino's rewards ������ https://t.co/t2vEroZh6H", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 458560537, "name": "Debrahh ❥", "screen_name": "debrawLee", "lang": "en", "location": "Texas, baby", "create_at": date("2012-01-08"), "description": "goodVibesOnLy⚡️ ♉️ SC:debrawrr421", "followers_count": 262, "friends_count": 318, "statues_count": 7239 }, "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-02T10:00:27.000Z"), "id": 683347135639744512, "text": "@BridgetteWest no one give my like one my thing maybe i am just dum please mentioned me video", "in_reply_to_status": -1, "in_reply_to_user": 129712915, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 129712915 }}, "user": { "id": 4206218181, "name": "jacquelyn aldrich", "screen_name": "EdithAldrich", "lang": "en", "location": "null", "create_at": date("2015-11-16"), "description": "null", "followers_count": 7, "friends_count": 14, "statues_count": 44 }, "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-01-02T10:00:27.000Z"), "id": 683347135794786304, "text": "O'Reilly Auto Parts: Store Counter Sales (#STOCKBRIDGE, GA) https://t.co/7UwPaXmyDw #Retail #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.2420639,33.5016247"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "STOCKBRIDGE", "Retail", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 28467014, "name": "TMJ-ATL Retail Jobs", "screen_name": "tmj_atl_retail", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-04-02"), "description": "Follow this account for geo-targeted Retail job tweets in Atlanta, GA. Need help? Tweet us at @CareerArc!", "followers_count": 489, "friends_count": 346, "statues_count": 1003 }, "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-02T10:00:27.000Z"), "id": 683347135966920706, "text": "Vietnam Memorial #washington #washingtondc #usa #unitedstatesofamerica #vietnammemorial… https://t.co/cPpttayyQY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.0367,38.8951"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "washington", "washingtondc", "usa", "unitedstatesofamerica", "vietnammemorial" }}, "user": { "id": 204359402, "name": "Mauro Fagnoni", "screen_name": "kingfesen", "lang": "it", "location": "null", "create_at": date("2010-10-18"), "description": "null", "followers_count": 74, "friends_count": 95, "statues_count": 515 }, "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-02T10:00:27.000Z"), "id": 683347136105299968, "text": "Walk in yo trap and take over yo trap", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3304736715, "name": "ThatsmyH0E™", "screen_name": "juice_yrn", "lang": "en", "location": "AtL✈843✈864", "create_at": date("2015-05-31"), "description": "FREE my brother @Bossmandre_ so he can get back to ballin #Y$N #YRN", "followers_count": 172, "friends_count": 279, "statues_count": 1869 }, "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-01-02T10:00:27.000Z"), "id": 683347136331788288, "text": "@Colts If I have to choose between Ryan Grigson and Chuck Pagano I'll come down right beside Pagano every time. You know Grigson needs to go", "in_reply_to_status": -1, "in_reply_to_user": 180884045, "favorite_count": 0, "coordinate": point("-86.1326801,39.84502"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 180884045 }}, "user": { "id": 142926457, "name": "JenWilson", "screen_name": "JenTheBartender", "lang": "en", "location": "Indianapolis", "create_at": date("2010-05-11"), "description": "Blonde hair, blue eyes, big mouth...yeah, I was built for this!", "followers_count": 796, "friends_count": 1278, "statues_count": 2591 }, "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-01-02T10:00:27.000Z"), "id": 683347136579112960, "text": "We're #hiring! Click to apply: LIFE INSURANCE AGENT - https://t.co/7IACwEa0YL #Insurance #Omaha, NE #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.9979883,41.2523634"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Insurance", "Omaha", "Job", "Jobs" }}, "user": { "id": 2366002093, "name": "Celebrity Staff Jobs", "screen_name": "celebstaffjobs", "lang": "en", "location": "Omaha\\Lincoln\\KC\\Des Moines", "create_at": date("2014-02-28"), "description": "Follow @celebstaffjobs for administrative, management, and legal #jobs. \r\nFollow our company @celebritystaff.", "followers_count": 60, "friends_count": 1, "statues_count": 142 }, "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-02T10:00:28.000Z"), "id": 683347136889647106, "text": "@jmturner_author Ahem, Jesus wasnt pasty white... Just sayin", "in_reply_to_status": 683346222170697728, "in_reply_to_user": 2997925069, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2997925069 }}, "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": 117744, "friends_count": 98278, "statues_count": 42139 }, "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-02T10:00:28.000Z"), "id": 683347137019564036, "text": "https://t.co/MMiSxW0Ary", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 599979316, "name": "Troy Geary", "screen_name": "TroyGeary", "lang": "en", "location": "Portland, OR", "create_at": date("2012-06-05"), "description": "Techno", "followers_count": 531, "friends_count": 522, "statues_count": 101273 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Hope, MN", "id": "76462ec13e8f0150", "name": "New Hope", "place_type": "city", "bounding_box": rectangle("-93.40099,45.007312 -93.370138,45.065743") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2745628, "cityName": "New Hope" } }
+{ "create_at": datetime("2016-01-02T10:00:28.000Z"), "id": 683347137283751936, "text": "Pony boy and unicorn �� https://t.co/CVaouxnX8h", "in_reply_to_status": 683337284163256320, "in_reply_to_user": 131331202, "favorite_count": 0, "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": "unavailable, yet here 4 u. follow @loki_logic https://soundcloud.com/howlitout/the-statement-loki-prod-eddie-leaux-kemastree", "followers_count": 4552, "friends_count": 2018, "statues_count": 147398 }, "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-01-02T10:00:28.000Z"), "id": 683347137367687168, "text": "OMMMGGGG YYAASSSSS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "et", "is_retweet": false, "user": { "id": 3313920385, "name": "BabyGirl", "screen_name": "sunshineluuuvv", "lang": "en", "location": "w/ Zo ❤", "create_at": date("2015-08-12"), "description": "SAV", "followers_count": 99, "friends_count": 144, "statues_count": 2880 }, "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-02T10:00:28.000Z"), "id": 683347137527214080, "text": "Driving through Ohio is much better than driving through Indiana #BuffaloBound", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BuffaloBound" }}, "user": { "id": 487255288, "name": "Lautaro", "screen_name": "lazypatagonian", "lang": "en", "location": "Louisville", "create_at": date("2012-02-08"), "description": "I look forward to it", "followers_count": 63, "friends_count": 339, "statues_count": 1617 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Medina, OH", "id": "00eb8de9015a3185", "name": "Medina", "place_type": "city", "bounding_box": rectangle("-81.907176,41.089056 -81.774633,41.215965") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39103, "countyName": "Medina", "cityID": 3948790, "cityName": "Medina" } }
+{ "create_at": datetime("2016-01-02T10:00:28.000Z"), "id": 683347137627840512, "text": "#VerbalDisturbance at 1201-1298 Windmill Ridge Loop. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.1714851,28.5538565"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VerbalDisturbance", "orlpol", "ocso" }}, "user": { "id": 39344591, "name": "Police Calls 32828", "screen_name": "orlpol32828", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-11"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 127, "friends_count": 1, "statues_count": 3706 }, "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-02T10:00:28.000Z"), "id": 683347137694846976, "text": "The breadsticks were incredible @ Olive Garden https://t.co/9xFoMpt21P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.30599064,32.92593029"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1129582591, "name": "avery curry", "screen_name": "averycurry3", "lang": "en", "location": "null", "create_at": date("2013-01-28"), "description": "»take a walk«", "followers_count": 306, "friends_count": 574, "statues_count": 1163 }, "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-02T10:00:28.000Z"), "id": 683347137929850881, "text": "They fuck whoever poppin what you got to offer doesn't interest them anymore homie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3024184655, "name": "DROGAMALA", "screen_name": "mellyfrombeach", "lang": "en", "location": "Jinjuriki ", "create_at": date("2015-02-07"), "description": "https://soundcloud.com/younggodly/commas", "followers_count": 416, "friends_count": 710, "statues_count": 3755 }, "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-02T10:00:28.000Z"), "id": 683347137971658752, "text": "shout to some dude Hennessy I drank, he thought the ladies I was with was gonna drink it but they passed it to me instead. 1UP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18621911, "name": "Shod", "screen_name": "shodgihigashymi", "lang": "en", "location": "AmeriKKKa", "create_at": date("2009-01-04"), "description": "innovator // traveler // creative director", "followers_count": 1134, "friends_count": 500, "statues_count": 69345 }, "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-02T10:00:28.000Z"), "id": 683347138080718848, "text": "@Eritreanpoca_ ����", "in_reply_to_status": 683346889647992832, "in_reply_to_user": 418758560, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 418758560 }}, "user": { "id": 59176407, "name": "Bari⛹", "screen_name": "JabariMoore_", "lang": "en", "location": "Dallas, Tx ", "create_at": date("2009-07-22"), "description": "Chill vibes only | Sc: Jabarimoore3 RIP DAD | Libra♎️", "followers_count": 2976, "friends_count": 944, "statues_count": 154858 }, "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-02T10:00:28.000Z"), "id": 683347138227613697, "text": "#ABE Ready to make it happen! ���� @ Vinings Atlanta, Ga https://t.co/EUdKF0HWfK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.48568853,33.86282366"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ABE" }}, "user": { "id": 42563015, "name": "SYLEAH", "screen_name": "ALISTPRISS", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-05-25"), "description": "And nothing shall be impossible for you ..Matthew 17:20", "followers_count": 548, "friends_count": 351, "statues_count": 22868 }, "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-01-02T10:00:28.000Z"), "id": 683347138235994112, "text": "@brocepher https://t.co/yGTWutQ8J3", "in_reply_to_status": -1, "in_reply_to_user": 537157523, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 537157523 }}, "user": { "id": 2180065839, "name": "genesis", "screen_name": "_itsgenesissss", "lang": "en", "location": "TPA", "create_at": date("2013-11-13"), "description": "SEU'19 || M.", "followers_count": 185, "friends_count": 164, "statues_count": 4545 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gibsonton, FL", "id": "00c23e64064f65f5", "name": "Gibsonton", "place_type": "city", "bounding_box": rectangle("-82.412765,27.777215 -82.326561,27.868938") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1225900, "cityName": "Gibsonton" } }
+{ "create_at": datetime("2016-01-02T10:00:28.000Z"), "id": 683347138365935616, "text": "���� how was it ? Did you feel at home? https://t.co/xiLidReAJf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "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": 1457, "friends_count": 790, "statues_count": 69276 }, "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-01-02T10:00:28.000Z"), "id": 683347138445709312, "text": "@overhealer platonic boyfrwignd", "in_reply_to_status": 683347070275686404, "in_reply_to_user": 2906684799, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2906684799 }}, "user": { "id": 876868524, "name": "binkshamed ™", "screen_name": "flight609", "lang": "en", "location": "rhys • he • ♉️ • INTP", "create_at": date("2012-10-12"), "description": "If we pray to the lord, does he sing on stage? // @thepisswentz", "followers_count": 663, "friends_count": 197, "statues_count": 41996 }, "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-02T10:00:28.000Z"), "id": 683347138466676739, "text": "@MIA_Heat_Index it's not. Just straight forward and un-complicated. Leave that crap to lord of the rings.", "in_reply_to_status": 683346884342202368, "in_reply_to_user": 225260541, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 225260541 }}, "user": { "id": 222673383, "name": "Alfredo Arteaga", "screen_name": "UptownReport", "lang": "en", "location": "Miami, Florida", "create_at": date("2010-12-03"), "description": "News and Opinion on Sports. Can and will Produce TV Shows for Money.", "followers_count": 2208, "friends_count": 893, "statues_count": 48809 }, "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.957039") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1214895, "cityName": "Country Club" } }
+{ "create_at": datetime("2016-01-02T10:00:28.000Z"), "id": 683347138940514306, "text": "Want to work in #Oakland, CA? View our latest opening: https://t.co/z3Ith0z5vm #HR #RHMRJobs #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.2682245,37.8113159"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Oakland", "HR", "RHMRJobs", "Job", "Jobs", "Hiring" }}, "user": { "id": 791930767, "name": "RHMR Jobs", "screen_name": "RHMRJobs", "lang": "en", "location": "null", "create_at": date("2012-08-30"), "description": "Robert Half Management Resources is the world's premier provider of senior-level accounting, finance & business systems professionals. See our newest jobs here!", "followers_count": 105, "friends_count": 8, "statues_count": 929 }, "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-02T10:00:28.000Z"), "id": 683347138948935680, "text": "why does this lil thing hurt like a bitch�� https://t.co/z4HBVEZpQo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3495031098, "name": "stoned malone", "screen_name": "blakelebby", "lang": "en", "location": "Edmond, OK", "create_at": date("2015-09-08"), "description": "what's cookin good lookin / Leo", "followers_count": 404, "friends_count": 442, "statues_count": 5035 }, "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-02T10:00:28.000Z"), "id": 683347139171319808, "text": "MMAC Monthly is here. Get your January issue! (@ Nederland, CO in Nederland, CO) https://t.co/tTVngd9vgb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.509468,39.96114"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 42470826, "name": "Jennifer Pund", "screen_name": "barefoot303", "lang": "en", "location": "Nederland, Co ", "create_at": date("2009-05-25"), "description": "Publisher, Editor, Contributing writer and ad sales at MMAC Monthly. Music addict and a resident of the peak to peak community.", "followers_count": 186, "friends_count": 346, "statues_count": 1390 }, "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": 8013, "countyName": "Boulder", "cityID": 853175, "cityName": "Nederland" } }
+{ "create_at": datetime("2016-01-02T10:00:28.000Z"), "id": 683347139288776704, "text": "Uncle murda- 2015 rap up ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2992645498, "name": "Don C.", "screen_name": "_Soraspy_Studz", "lang": "en", "location": "#GEMS #D.O.S.Boyz", "create_at": date("2015-01-22"), "description": "Yonkers Legend", "followers_count": 262, "friends_count": 267, "statues_count": 4169 }, "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-01-02T10:00:28.000Z"), "id": 683347139292983296, "text": "A Reflection of 2015 And What To Expect In 2016 https://t.co/IhMU0kUN1a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.1193763,41.7353591"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 302272155, "name": "Russell Tanzillo", "screen_name": "RussellTanzillo", "lang": "en", "location": "Chicagoland, IL", "create_at": date("2011-05-20"), "description": "Hey I’m Russell I grew up in the suburbs of Chicago listening to any music I could get my hands/ears on. Now I am on the radio @101WKQX!", "followers_count": 303, "friends_count": 590, "statues_count": 1315 }, "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-01-02T10:00:28.000Z"), "id": 683347139301228545, "text": "@GroverNorquist So when does all these Obama gun seizures start? Is it before or after the \"imminent\" FEMA camps or \"imminent\" Sharia law?", "in_reply_to_status": 683282313078259712, "in_reply_to_user": 16045956, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16045956 }}, "user": { "id": 87869624, "name": "stephen pomes", "screen_name": "stephenpomes", "lang": "en", "location": "Louisiana, USA", "create_at": date("2009-11-05"), "description": "Politically Liberal librarian with interests in history, fine arts, cinema, TV, philosophy, & pet interests in the above fields. #UniteBlue", "followers_count": 732, "friends_count": 1998, "statues_count": 59542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mandeville, LA", "id": "003d28304705c42a", "name": "Mandeville", "place_type": "city", "bounding_box": rectangle("-90.154712,30.348341 -90.003767,30.459998") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22103, "countyName": "St. Tammany", "cityID": 2248225, "cityName": "Mandeville" } }
+{ "create_at": datetime("2016-01-02T10:00:28.000Z"), "id": 683347139364294656, "text": "@BraveSyndicate Do you guys have a competitive team? If so can I prove myself to you if your looking for players? Thank you for your time.", "in_reply_to_status": 681683113223692289, "in_reply_to_user": 3076470992, "favorite_count": 0, "coordinate": point("-73.9827303,40.88950321"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3076470992 }}, "user": { "id": 2324967928, "name": "Wilby Ramos", "screen_name": "NotableHealz", "lang": "en", "location": "null", "create_at": date("2014-02-04"), "description": "16| East Coast | Hybrid Slayer | Tourney Wins:13 | Sponsors: @ProspectGrips use StanDz 5% off | Top AM Player | Sub https://t.co/xTKMBtGc9n Thanks", "followers_count": 210, "friends_count": 113, "statues_count": 4735 }, "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-01-02T10:00:28.000Z"), "id": 683347139401879552, "text": "After the basketball game I plan to switch from my KSU bball shirt to my KSU football shirt for the bowl game. #CatsMan", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CatsMan" }}, "user": { "id": 715287643, "name": "Ben Kohl", "screen_name": "k0hld_blooded", "lang": "en", "location": "boolin", "create_at": date("2012-07-24"), "description": "#Boomsquad - Cats Man", "followers_count": 584, "friends_count": 392, "statues_count": 8791 }, "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-02T10:00:28.000Z"), "id": 683347139603357696, "text": "@BigOShow I try to be optimisitic every play, game and season, yet the Fins never fail to let me down. 3rd and 14--give up 15. No playoffs", "in_reply_to_status": -1, "in_reply_to_user": 23536653, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23536653 }}, "user": { "id": 1048774327, "name": "Neilstradamus", "screen_name": "Neilstradamus", "lang": "en", "location": "Florida, USA", "create_at": date("2012-12-30"), "description": "null", "followers_count": 40, "friends_count": 93, "statues_count": 634 }, "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-01-02T10:00:28.000Z"), "id": 683347139611721729, "text": "This is literally how he eats everyday. So do I though, so... whatever. @ Glenside, Pennsylvania https://t.co/IQpqV0XA0C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.15222222,40.10305556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 215126645, "name": "Jess Craig", "screen_name": "IROCKSOWHAT", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2010-11-12"), "description": "#ecommerce merchandiser & social media lady at @WebLinc", "followers_count": 2609, "friends_count": 1224, "statues_count": 26943 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glenside, PA", "id": "c912e672d14acc42", "name": "Glenside", "place_type": "city", "bounding_box": rectangle("-75.164691,40.092373 -75.134624,40.113888") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42091, "countyName": "Montgomery", "cityID": 4229808, "cityName": "Glenside" } }
+{ "create_at": datetime("2016-01-02T10:00:28.000Z"), "id": 683347139787886593, "text": "Original, chocolate or frosted animal crackers. Discuss.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25994269, "name": "Joey M", "screen_name": "Cinecrab", "lang": "en", "location": "Utica, NY", "create_at": date("2009-03-23"), "description": "Professional Pizza Eater (PPE) Aquaman/DC/Comics-Utica Comets/Buffalo Sabres/Hockey- Atlanta Falcons/Football-Saturday Night Live", "followers_count": 52, "friends_count": 156, "statues_count": 2395 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Utica, NY", "id": "41f575b7eebcd4b7", "name": "Utica", "place_type": "city", "bounding_box": rectangle("-75.295291,43.050251 -75.158681,43.189108") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36065, "countyName": "Oneida", "cityID": 3676540, "cityName": "Utica" } }
+{ "create_at": datetime("2016-01-02T10:00:28.000Z"), "id": 683347139984896001, "text": "We're #hiring! Read about our latest #job opening here: Sales Performance Training Instructor 3 - https://t.co/2o8PlvY4T7 #IT #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.2455364,37.5364134"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "IT", "CareerArc" }}, "user": { "id": 112596930, "name": "TMJ-CA IT Jobs", "screen_name": "tmj_CA_it", "lang": "en", "location": "California", "create_at": date("2010-02-08"), "description": "Follow this account for geo-targeted Software Dev. - General/IT job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 463, "friends_count": 332, "statues_count": 642 }, "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-01-02T10:00:28.000Z"), "id": 683347140559646720, "text": "\"All progress is precarious, and the solution of one problem brings us face to face with another problem.\" - Martin Luther King, Jr.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1099847616, "name": "Miguel Jackson", "screen_name": "GodsWordIsFree", "lang": "en", "location": "NYC now...ATL summer 2016", "create_at": date("2013-01-17"), "description": "Founder of the World Bible Project. We spread the Word by offering free Bibles. Soon will assist, inspire, and empower people to home ownership in ATL area.", "followers_count": 24639, "friends_count": 24387, "statues_count": 53099 }, "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-02T10:00:28.000Z"), "id": 683347140660183040, "text": "#Atlanta, GA #Nursing #Job: LPN, INFUSION, BLAIRSVILLE/ BLUE RIDGE at Northside Hospital https://t.co/TEC6MDig42 #LPN #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.352759,33.909187"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Atlanta", "Nursing", "Job", "LPN", "Jobs", "Hiring" }}, "user": { "id": 3161440380, "name": "Northside Careers", "screen_name": "NorthsideCareer", "lang": "en", "location": "null", "create_at": date("2015-04-17"), "description": "Dedicated to providing clinical excellence balanced with compassionate care. Learn how you can make a difference and continue growing your career.", "followers_count": 16, "friends_count": 1, "statues_count": 427 }, "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-01-02T10:00:28.000Z"), "id": 683347140773429249, "text": "sounds kinda true. https://t.co/gIOAVlL0pG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 316807328, "name": "RavenJ", "screen_name": "raeejones_", "lang": "en", "location": " West$ide, GTX ", "create_at": date("2011-06-13"), "description": "BYE.", "followers_count": 732, "friends_count": 494, "statues_count": 18928 }, "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-02T10:00:29.000Z"), "id": 683347140853280769, "text": "No one makes me feel the way 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": 463477890, "name": "Erin Crane", "screen_name": "Crane_erin", "lang": "en", "location": "Kutztown, PA", "create_at": date("2012-01-13"), "description": "Do not dwell in the past, do not dream of the future, concentrate the mind on the present. Jersey. Kutztown", "followers_count": 453, "friends_count": 761, "statues_count": 6725 }, "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-02T10:00:29.000Z"), "id": 683347141037834240, "text": "Dying in bed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2474738341, "name": "your favorite", "screen_name": "sighearlee", "lang": "en", "location": "sunshine state FL ", "create_at": date("2014-05-02"), "description": "took ya bitch", "followers_count": 910, "friends_count": 1005, "statues_count": 21552 }, "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-01-02T10:00:29.000Z"), "id": 683347141314494464, "text": "throwback �� @yelyahwilliams https://t.co/2jJQZwCCLj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 40981798 }}, "user": { "id": 2424815269, "name": "براندي", "screen_name": "brandykathryn", "lang": "en", "location": "null", "create_at": date("2014-04-02"), "description": "late december with my heart in my chest", "followers_count": 891, "friends_count": 948, "statues_count": 36100 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-02T10:00:29.000Z"), "id": 683347141461471234, "text": "@xoxo_anitaa_ @sudd3nlyyy oh but the sonnet is due on Monday.", "in_reply_to_status": 683346335186194434, "in_reply_to_user": 3161278019, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2569230195, 1078007575 }}, "user": { "id": 3161278019, "name": "Kassandra Colón", "screen_name": "kassandracolonn", "lang": "en", "location": "Fort Lauderdale, FL", "create_at": date("2015-04-12"), "description": "No Beckys pls // Latin // Iowa 20'", "followers_count": 206, "friends_count": 191, "statues_count": 1393 }, "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-02T10:00:29.000Z"), "id": 683347141817970688, "text": "OMG. YESSS! https://t.co/hO7572NAw4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 40077777, "name": "Saige Mize", "screen_name": "iiloveeyuu", "lang": "en", "location": "Sunshine State ♡", "create_at": date("2009-05-14"), "description": "null", "followers_count": 217, "friends_count": 384, "statues_count": 3008 }, "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-01-02T10:00:29.000Z"), "id": 683347141876658176, "text": "He's the ghost tweeter https://t.co/nLk7r0kUsH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2159149793, "name": "FN-2187", "screen_name": "notjaylen_", "lang": "en", "location": "Greenville, NC", "create_at": date("2013-10-29"), "description": "Gpa 1.8 but I'm serving 3.5", "followers_count": 775, "friends_count": 398, "statues_count": 28359 }, "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-02T10:00:29.000Z"), "id": 683347141893451777, "text": "Gordon fouls Garrett on 3-pointer. Uh-oh. No time on clock. Seton Hall up 76-73.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "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": 7099, "friends_count": 2799, "statues_count": 90593 }, "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-02T10:00:29.000Z"), "id": 683347141918474242, "text": "@ecetram @dae_suave @Wherejerry @Jstuntin_ the waves on my head lol", "in_reply_to_status": 683341945511538688, "in_reply_to_user": 2445965682, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2445965682, 3100799984, 378873768, 258416270 }}, "user": { "id": 2881359650, "name": "Justin Grimes ☃", "screen_name": "JayFaneto_", "lang": "en", "location": "null", "create_at": date("2014-10-28"), "description": "Seventeen", "followers_count": 1259, "friends_count": 566, "statues_count": 25949 }, "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-02T10:00:29.000Z"), "id": 683347142279233536, "text": "@Matt2Harrington Yep!", "in_reply_to_status": 683347085626863616, "in_reply_to_user": 35256596, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 35256596 }}, "user": { "id": 23064132, "name": "Matt Chinander", "screen_name": "chinanderm", "lang": "en", "location": "Iowa City, IA", "create_at": date("2009-03-06"), "description": "I build websites for higher education, play guitar, obsess over music, and write for @GDA.\n\n#GoHawks #Skol", "followers_count": 2133, "friends_count": 451, "statues_count": 30948 }, "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-02T10:00:29.000Z"), "id": 683347142396764164, "text": "ooooh she just my type, these other hoes just typos", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2562922304, "name": "Tundra Chinx", "screen_name": "dayoungvillain", "lang": "en", "location": "NYC", "create_at": date("2014-06-12"), "description": "non affiliated", "followers_count": 357, "friends_count": 261, "statues_count": 9891 }, "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-02T10:00:29.000Z"), "id": 683347142656708608, "text": "Interested in a #Hospitality #job near #Monroe, LA? This could be a great fit: https://t.co/XzmsP9DGyU #SONIC #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.0694642,32.4984998"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Monroe", "SONIC", "Hiring", "CareerArc" }}, "user": { "id": 132057683, "name": "TMJ-LAM HRTA Jobs", "screen_name": "tmj_LAM_HRTA", "lang": "en", "location": "Monroe, LA", "create_at": date("2010-04-11"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Monroe, LA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 294, "friends_count": 272, "statues_count": 128 }, "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-02T10:00:29.000Z"), "id": 683347142954582016, "text": "Man I hope we can get at least 20-22 commits for 2016.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1892046097, "name": "Krishton Neeley", "screen_name": "NKrishton", "lang": "en", "location": "Lake City ", "create_at": date("2013-09-21"), "description": "Kristyn. Jesus Lover. 7th Grade Civics Teacher. Duck Hunter. Republican. GO Dawgs. Who Dat.", "followers_count": 280, "friends_count": 718, "statues_count": 5925 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake City, FL", "id": "c9026d2fba3bf4d1", "name": "Lake City", "place_type": "city", "bounding_box": rectangle("-82.709702,30.155154 -82.618276,30.210382") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12023, "countyName": "Columbia", "cityID": 1237775, "cityName": "Lake City" } }
+{ "create_at": datetime("2016-01-02T10:00:29.000Z"), "id": 683347143168507904, "text": "Childish Gambino/heartbeat", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2444690959, "name": "JStragand3️⃣7️⃣", "screen_name": "JoshStragand37", "lang": "en", "location": "prob at baseball", "create_at": date("2014-04-14"), "description": "6'4 And Blessed:PNHS Baseball#22:Delgo Elite Baseball#37~Sophmore-12.22.15", "followers_count": 266, "friends_count": 209, "statues_count": 1224 }, "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-02T10:00:29.000Z"), "id": 683347143386644480, "text": "@LexNicolex @fernyyy12 Lmao this pic is great ��", "in_reply_to_status": 683316015116234752, "in_reply_to_user": 172195188, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 172195188, 232365647 }}, "user": { "id": 159879657, "name": "Jennifer", "screen_name": "jennysmallls", "lang": "en", "location": "Wade County", "create_at": date("2010-06-26"), "description": "null", "followers_count": 1016, "friends_count": 605, "statues_count": 36130 }, "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("2016-01-02T10:00:29.000Z"), "id": 683347143491371011, "text": "@Alessandra8484 You betcha!", "in_reply_to_status": 683346647468892161, "in_reply_to_user": 394796888, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 394796888 }}, "user": { "id": 930552433, "name": "Harold Smith", "screen_name": "CU86Grad", "lang": "en", "location": "Buies Creek, NC", "create_at": date("2012-11-06"), "description": "Live in NC but follow bunch of Canadians. Go figure, eh? CU86Grad IG and SnapChat too. #CowboysNation! Thanks to Canadian Sisters #LeafsNation Lover of Bow Ties", "followers_count": 2649, "friends_count": 2277, "statues_count": 124118 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fuquay-Varina, NC", "id": "000b2b6f4a781bba", "name": "Fuquay-Varina", "place_type": "city", "bounding_box": rectangle("-78.844741,35.560069 -78.697863,35.670874") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3725300, "cityName": "Fuquay-Varina" } }
+{ "create_at": datetime("2016-01-02T10:00:29.000Z"), "id": 683347143537606656, "text": "@mfdwx @mike61400 to what?", "in_reply_to_status": 683347054098276352, "in_reply_to_user": 2338048487, "favorite_count": 0, "coordinate": point("-70.6869768,41.9535379"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2338048487, 1074088987 }}, "user": { "id": 110579152, "name": "Steve-O", "screen_name": "SnowballManny", "lang": "en", "location": "null", "create_at": date("2010-02-01"), "description": "null", "followers_count": 354, "friends_count": 909, "statues_count": 16666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plymouth, MA", "id": "01bbfc630e681dda", "name": "Plymouth", "place_type": "city", "bounding_box": rectangle("-70.758703,41.789998 -70.525503,41.972125") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25023, "countyName": "Plymouth", "cityID": 2554275, "cityName": "Plymouth" } }
+{ "create_at": datetime("2016-01-02T10:00:29.000Z"), "id": 683347143566999552, "text": "Binge watching on Netflix on this sunny 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": 191241953, "name": "Ming", "screen_name": "emsmooth13", "lang": "en", "location": "My girl is Columbian", "create_at": date("2010-09-15"), "description": "in a space where music is playing. #StayWoke\n\nSnap: ming13", "followers_count": 466, "friends_count": 330, "statues_count": 27096 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newington, CT", "id": "89df89b3b7ac521b", "name": "Newington", "place_type": "city", "bounding_box": rectangle("-72.762431,41.647296 -72.700621,41.724482") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 952210, "cityName": "Newington" } }
+{ "create_at": datetime("2016-01-02T10:00:29.000Z"), "id": 683347143852072960, "text": "They said MTV Cribs was abusing the fuck outta the replay button ��������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 995727188, "name": "MOSEL", "screen_name": "moselxasvp", "lang": "en", "location": "dmv", "create_at": date("2012-12-07"), "description": "I GOT N'SYNC DANCING ON MY NECK", "followers_count": 449, "friends_count": 454, "statues_count": 16675 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-02T10:00:29.000Z"), "id": 683347144355516416, "text": "These are my confessions", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 64772552, "name": "Kristi", "screen_name": "Kristi_Pie14", "lang": "en", "location": "prolly somewhere sleeping", "create_at": date("2009-08-11"), "description": "good girl who does her makeup to trap music", "followers_count": 593, "friends_count": 391, "statues_count": 22510 }, "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-02T10:00:29.000Z"), "id": 683347144389095424, "text": "#New #Music Check Out @RobStashiz New Joint \"DRAW BLOOD\" Feat. @therealjimbones Prod.By TantuBeats’ https://t.co/3FEaUK810M … #RDSE��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "New", "Music", "RDSE" }}, "user_mentions": {{ 69172425, 234294014 }}, "user": { "id": 481606532, "name": "DJ COZ ♌", "screen_name": "REEMTHEDREAM215", "lang": "en", "location": "PHILLY", "create_at": date("2012-02-02"), "description": "Forward Is My Progress! Raw Deal Squad Ent., Mobile DJ For Bookings djcozrds@gmail.com or 215-501-9082", "followers_count": 1166, "friends_count": 1841, "statues_count": 52160 }, "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-02T10:00:29.000Z"), "id": 683347144577683457, "text": "@h8rproof82 how was that not a goal tend ? Holy crap", "in_reply_to_status": 683344090721685504, "in_reply_to_user": 793771752, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 793771752 }}, "user": { "id": 3140987918, "name": "blank", "screen_name": "KayNel76", "lang": "en", "location": "null", "create_at": date("2015-04-05"), "description": "#TakeTheCrown #RCJH #LetsGoRoyals #LosReales Love FooFighters & George Strait, you figure it out. Parody is a helluva drug", "followers_count": 301, "friends_count": 606, "statues_count": 16501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Belton, MO", "id": "7235125f54ba7b87", "name": "Belton", "place_type": "city", "bounding_box": rectangle("-94.567317,38.790672 -94.490023,38.845062") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29037, "countyName": "Cass", "cityID": 2904384, "cityName": "Belton" } }
+{ "create_at": datetime("2016-01-02T10:00:30.000Z"), "id": 683347145332781060, "text": "Maybe one more cup of coffee will make me productive. *Pours 5th cup*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 131302830, "name": "Kim DeGidio", "screen_name": "KimDeGidio", "lang": "en", "location": "Lynchburg", "create_at": date("2010-04-09"), "description": "Makai #LiveAKendallLife", "followers_count": 271, "friends_count": 169, "statues_count": 9247 }, "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-02T10:00:30.000Z"), "id": 683347145521364992, "text": "#CareerArc #Hospitality #Job alert: Baker - Night | Panera Bread | #Manassas, VA https://t.co/tHaZTT9PN1 #parttime #Veterans #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.5103528,38.7821922"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CareerArc", "Hospitality", "Job", "Manassas", "parttime", "Veterans", "Jobs", "Hiring" }}, "user": { "id": 88305044, "name": "TMJ-VA HRTA Jobs", "screen_name": "tmj_va_hrta", "lang": "en", "location": "Virginia", "create_at": date("2009-11-07"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Virginia Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 356, "friends_count": 285, "statues_count": 728 }, "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-01-02T10:00:30.000Z"), "id": 683347145538207746, "text": "Want to work at Fidelity Investments? We're #hiring in #Merrimack, NH! Click for details: https://t.co/tTRD1QGWO9 #InvestmentBanking #Job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.4948355,42.8678651"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Merrimack", "InvestmentBanking", "Job" }}, "user": { "id": 207352965, "name": "NH Inv. Banking", "screen_name": "tmj_NH_invbank", "lang": "en", "location": "New Hampshire", "create_at": date("2010-10-24"), "description": "Follow this account for geo-targeted Investment Banking job tweets in New Hampshire Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 239, "friends_count": 235, "statues_count": 12 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merrimack, NH", "id": "000a400a1aba18e8", "name": "Merrimack", "place_type": "city", "bounding_box": rectangle("-71.589882,42.790953 -71.453424,42.910922") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33011, "countyName": "Hillsborough", "cityID": 3321780, "cityName": "East Merrimack" } }
+{ "create_at": datetime("2016-01-02T10:00:30.000Z"), "id": 683347146150653952, "text": "fuckin snitch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 525799766, "name": "Julia ✨", "screen_name": "JuliaaPhillips", "lang": "en", "location": "null", "create_at": date("2012-03-15"), "description": "null", "followers_count": 560, "friends_count": 336, "statues_count": 12278 }, "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-02T10:00:30.000Z"), "id": 683347146494574593, "text": "You're still the best more or less 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": 237071727, "name": "sierra marie.", "screen_name": "thatgirl_gingaa", "lang": "en", "location": "murrland", "create_at": date("2011-01-11"), "description": "I'd rather be a riot than indifferent.", "followers_count": 2042, "friends_count": 1515, "statues_count": 108431 }, "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-02T10:00:30.000Z"), "id": 683347146775515136, "text": "Want to work in #PALOALTO, CA? View our latest opening: https://t.co/RU0hAaU388 #IT #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.1575745,37.4457966"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PALOALTO", "IT", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 20827150, "name": "San Francisco IT Job", "screen_name": "tmj_sfo_it", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-02-13"), "description": "Follow this account for geo-targeted Software Dev. - General/IT job tweets in San Francisco, CA. Need help? Tweet us at @CareerArc!", "followers_count": 719, "friends_count": 360, "statues_count": 1042 }, "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-01-02T10:00:30.000Z"), "id": 683347146926473218, "text": "Maybe 2016 ill get a beach bod", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30682214, "name": "James Campo", "screen_name": "JamesCampo", "lang": "en", "location": "Kansas City, MO & Columbia, MO", "create_at": date("2009-04-12"), "description": "I'm just here so I won't get fined. | MIZ - ΣAE |", "followers_count": 455, "friends_count": 222, "statues_count": 9507 }, "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-02T10:00:30.000Z"), "id": 683347146964348928, "text": "Que como decia mi madre,bailando todo se arregla! Pegate un poco mas, pegado a los tambores olvida los temores, que el tiempo se nos va...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 824419158, "name": "Frank Abagnale Jr.", "screen_name": "LucasTomas22", "lang": "es", "location": "instagram.com/lucastomas22", "create_at": date("2012-09-14"), "description": "Durmiendo no aprendes nada, Salta La Banca. Cuento con serios problemas en el coco. Amante del Fútbol y el Boxeo. Jogo Gomito.", "followers_count": 608, "friends_count": 866, "statues_count": 18460 }, "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-01-02T10:00:30.000Z"), "id": 683347147039858690, "text": "Shouldn't be in-app purchases for kids games https://t.co/cHMCLkJNp6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 228088091, "name": "Money", "screen_name": "AaronTaylorMade", "lang": "en", "location": "FL/Ohio", "create_at": date("2010-12-18"), "description": "I'm out for dead presidents to represent me. #YEEZY2020", "followers_count": 243, "friends_count": 202, "statues_count": 15823 }, "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-02T10:00:30.000Z"), "id": 683347147287322624, "text": "केजरीवाल की बुद्धि भी Odd & Even के हिसाब से काम करती है, और जुबान मुफ्त की......", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "hi", "is_retweet": false, "user": { "id": 1665814687, "name": "Reva D", "screen_name": "BeingReva", "lang": "en", "location": "Mumbai, Maharastra", "create_at": date("2013-08-12"), "description": "Actor Tollywood || follower of Lord Shiva, & Have Faith in @narendramodi #Yoga", "followers_count": 17414, "friends_count": 166, "statues_count": 34352 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Egypt Lake-Leto, FL", "id": "6d4de57fa00a913f", "name": "Egypt Lake-Leto", "place_type": "city", "bounding_box": rectangle("-82.527472,27.996725 -82.485096,28.037298") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1220108, "cityName": "Egypt Lake-Leto" } }
+{ "create_at": datetime("2016-01-02T10:00:30.000Z"), "id": 683347147648020480, "text": "want gene n judes ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 442340950, "name": "jiggythree", "screen_name": "euphorgiaa", "lang": "en", "location": "sherwood forest", "create_at": date("2011-12-20"), "description": "if lost just leave me where i am im probs just high af • @wilsonn_erica ❥ • dylan is pretty cool i guess • KILLA KILLA ૐ #deadhead", "followers_count": 1676, "friends_count": 411, "statues_count": 59484 }, "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-02T10:00:30.000Z"), "id": 683347147903860737, "text": "#HolaSebastian tweeteame una selfie tuya de ahora! @SEBTSB x29", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "HolaSebastian" }}, "user_mentions": {{ 1631128196 }}, "user": { "id": 2621545545, "name": "sarah", "screen_name": "confusemendes", "lang": "en", "location": "New York, USA", "create_at": date("2014-06-18"), "description": "* ✧ · . ☾ *| njh | jw | sm |* ✧ ·. ☾ *", "followers_count": 2704, "friends_count": 2079, "statues_count": 22269 }, "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-01-02T10:00:30.000Z"), "id": 683347148063117312, "text": "#Spokane, Washington #Insurance #Job: Auto Damage Trainee - Spokane, WA at GEICO https://t.co/C1yfapWdcP #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.4260466,47.6587802"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Spokane", "Insurance", "Job", "Jobs", "Hiring" }}, "user": { "id": 201348710, "name": "GEICO Jobs", "screen_name": "GEICO_Jobs", "lang": "en", "location": "Across the U.S.", "create_at": date("2010-10-11"), "description": "#GEICO is stable, growing & #hiring in Sales, CustServ, IT, Legal, Claims & more! This is our #jobs feed. Follow @GEICOCareers or @GEICOCampus to connect.", "followers_count": 455, "friends_count": 96, "statues_count": 2214 }, "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-02T10:00:30.000Z"), "id": 683347148096708608, "text": "We're #hiring! Read about our latest #job opening here: RN-Operating Room Full Time Day - https://t.co/YY9DVaOx1g #TomsRiver, NJ #Nursing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.1979192,39.953729"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "TomsRiver", "Nursing" }}, "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": 455, "friends_count": 292, "statues_count": 557 }, "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-02T10:00:30.000Z"), "id": 683347148415610880, "text": "There's something super sweet and super spiritual about my #yoda PEZ @PEZCandyUSA https://t.co/wFhlXLoV0V", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "yoda" }}, "user_mentions": {{ 264360936 }}, "user": { "id": 94111802, "name": "Nancy Patterson", "screen_name": "ncpatte", "lang": "en", "location": "null", "create_at": date("2009-12-02"), "description": "null", "followers_count": 53, "friends_count": 83, "statues_count": 184 }, "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-02T10:00:30.000Z"), "id": 683347148516229120, "text": "�� https://t.co/O61UCl45L9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2225717655, "name": "Morganne", "screen_name": "morgannedolann", "lang": "en", "location": "Buffalo, NY", "create_at": date("2013-12-14"), "description": "@asvpxrocky Life Is Good", "followers_count": 474, "friends_count": 319, "statues_count": 4658 }, "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-02T10:00:30.000Z"), "id": 683347148633686018, "text": "A look North. [Cam 2] on Saturday, January 2, 2016 @ 1:00:23 PM #CarolinaWx #ClaytonNC https://t.co/gXDYS4Xllj", "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": 262, "friends_count": 35, "statues_count": 713788 }, "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.32187,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37101, "countyName": "Johnston" } }
+{ "create_at": datetime("2016-01-02T10:00:30.000Z"), "id": 683347148742750208, "text": "@spasianmario otb idk. If the building upgraded but https://t.co/0yP1gQMY7m", "in_reply_to_status": 683346911563087872, "in_reply_to_user": 114601784, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 114601784 }}, "user": { "id": 347372806, "name": "⚜sc | neefsharkly⚜", "screen_name": "Neef_Beastly", "lang": "en", "location": "New Orleans, LA", "create_at": date("2011-08-02"), "description": "What We Do In Life Echoes In Eternity |........| Money$ide 13th", "followers_count": 10016, "friends_count": 3216, "statues_count": 168278 }, "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-02T10:00:30.000Z"), "id": 683347148763611136, "text": "This #EmergencyMedicine #job might be a great fit for you: LVN-Urgent Care-San Mateo (Per Diem) - https://t.co/v6Px6cTUqJ #SanMateo, CA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.3255254,37.5629917"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "EmergencyMedicine", "job", "SanMateo" }}, "user": { "id": 120915363, "name": "TMJ-CAP Health Jobs", "screen_name": "tmj_CAP_health", "lang": "en", "location": "The Peninsula, CA", "create_at": date("2010-03-07"), "description": "Follow this account for geo-targeted Healthcare job tweets in The Peninsula, CA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 354, "friends_count": 279, "statues_count": 113 }, "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-02T10:00:30.000Z"), "id": 683347148763697152, "text": "#happybirthday to my #dad who is no longer physically here, but always in my #heart. #photo… https://t.co/haOlBVi9gv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.42417754,42.54112328"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "happybirthday", "dad", "heart", "photo" }}, "user": { "id": 156359478, "name": "Lynn Goldsmith", "screen_name": "Goldsmithphoto", "lang": "en", "location": "NYC/ASPEN", "create_at": date("2010-06-16"), "description": "null", "followers_count": 948, "friends_count": 313, "statues_count": 1823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Bloomfield Township, MI", "id": "93b9db5989615417", "name": "West Bloomfield Township", "place_type": "city", "bounding_box": rectangle("-83.441942,42.52605 -83.32039,42.617858") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland" } }
+{ "create_at": datetime("2016-01-02T10:00:30.000Z"), "id": 683347148935663616, "text": "GODwin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1146599900, "name": "PSU_sincebirth", "screen_name": "PSU_sincebirth", "lang": "en", "location": "null", "create_at": date("2013-02-03"), "description": "null", "followers_count": 104, "friends_count": 269, "statues_count": 4087 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clearfield, PA", "id": "00ae7f2f68947ab6", "name": "Clearfield", "place_type": "city", "bounding_box": rectangle("-78.463671,40.998555 -78.383498,41.047675") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42033, "countyName": "Clearfield", "cityID": 4214064, "cityName": "Clearfield" } }
+{ "create_at": datetime("2016-01-02T10:00:30.000Z"), "id": 683347148994424832, "text": "@TheRichardLewis Loved your interview on @NPR today. You are always genuinely hysterical!", "in_reply_to_status": -1, "in_reply_to_user": 202415747, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 202415747, 5392522 }}, "user": { "id": 252784048, "name": "Kim Miller", "screen_name": "Kimberly9573", "lang": "en", "location": "Redford,MI", "create_at": date("2011-02-15"), "description": "Owner of Kim's Cinnamon Cashew Brittle ... Find it on Facebook.", "followers_count": 91, "friends_count": 216, "statues_count": 260 }, "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-01-02T10:00:30.000Z"), "id": 683347148998602752, "text": "#UnknownTrouble at 6413 Westgate Dr. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.468716,28.530552"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "UnknownTrouble", "orlpol", "opd" }}, "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": 248, "friends_count": 1, "statues_count": 41960 }, "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-02T10:00:31.000Z"), "id": 683347149195640832, "text": "@mpharks @AverageBroz @corygfitness awesome work! Strong hip drive out of the hole!", "in_reply_to_status": 683255549794856960, "in_reply_to_user": 55814232, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 55814232, 1942428164, 225308736 }}, "user": { "id": 3821119094, "name": "Squat University", "screen_name": "SquatUniversity", "lang": "en", "location": "null", "create_at": date("2015-10-07"), "description": "Lecture hall of Doctor @aaronhorschigPT. The blog to discover how to eliminate pain, optimize performance & find true strength potential all with the squat.", "followers_count": 2018, "friends_count": 1607, "statues_count": 1486 }, "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-01-02T10:00:31.000Z"), "id": 683347149199949824, "text": "Kristina making me grilled cheese ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 356435134, "name": "-A", "screen_name": "AliSkinner", "lang": "en", "location": "null", "create_at": date("2011-08-16"), "description": "I wanna travel. Instagram: _a_skinner_ Follow,❤️", "followers_count": 397, "friends_count": 359, "statues_count": 8100 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Caro, MI", "id": "5cd9dff3a66d3dce", "name": "Caro", "place_type": "city", "bounding_box": rectangle("-83.416958,43.473781 -83.348039,43.525193") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26157, "countyName": "Tuscola", "cityID": 2613420, "cityName": "Caro" } }
+{ "create_at": datetime("2016-01-02T10:00:31.000Z"), "id": 683347149397082116, "text": "@Hannahwhitlock_ @alexia_khalil @kelliemagda nothing but love����", "in_reply_to_status": 683341749125787649, "in_reply_to_user": 1187575106, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1187575106, 452792819, 345226517 }}, "user": { "id": 377646023, "name": "selma", "screen_name": "_selmicc", "lang": "en", "location": "null", "create_at": date("2011-09-21"), "description": "dont worry, stay happy ☪", "followers_count": 403, "friends_count": 331, "statues_count": 8452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mountain Park, GA", "id": "108811a523220f75", "name": "Mountain Park", "place_type": "city", "bounding_box": rectangle("-84.169688,33.822567 -84.101669,33.870428") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13135, "countyName": "Gwinnett", "cityID": 1353186, "cityName": "Mountain Park" } }
+{ "create_at": datetime("2016-01-02T10:00:31.000Z"), "id": 683347149724057600, "text": "This #BusinessMgmt #job might be a great fit for you: Shift Manager (US) - https://t.co/OspUG6fgZa #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.8570633,35.679337"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BusinessMgmt", "job", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 22363501, "name": "TMJ-TN Mgmt. Jobs", "screen_name": "tmj_tn_mgmt", "lang": "en", "location": "Tennessee", "create_at": date("2009-03-01"), "description": "Follow this account for geo-targeted Business/Mgmt. job tweets in Tennessee Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 362, "friends_count": 242, "statues_count": 109 }, "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-01-02T10:00:31.000Z"), "id": 683347149724237824, "text": "feel so gross ;/", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 607405534, "name": "camryn gallagher", "screen_name": "damcamm", "lang": "en", "location": "Florida, USA", "create_at": date("2012-06-13"), "description": "@robpiekarskie8 ❤️", "followers_count": 531, "friends_count": 172, "statues_count": 13016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ocean Breeze Park, FL", "id": "fe37c5064d19f7d0", "name": "Ocean Breeze Park", "place_type": "city", "bounding_box": rectangle("-80.234375,27.23679 -80.221325,27.243785") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12085, "countyName": "Martin", "cityID": 1250900, "cityName": "Ocean Breeze Park" } }
+{ "create_at": datetime("2016-01-02T10:00:31.000Z"), "id": 683347149770231808, "text": "This #Healthcare #job might be a great fit for you: ER Registered Nurse - Emergency Room RN - https://t.co/HPYhEhhLxq #Hackettstown, NJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.8290555,40.8539879"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "Hackettstown" }}, "user": { "id": 21629917, "name": "TMJ- NJN Health Jobs", "screen_name": "tmj_njn_health", "lang": "en", "location": "Northern NJ", "create_at": date("2009-02-22"), "description": "Follow this account for geo-targeted Healthcare job tweets in Northern NJ. Need help? Tweet us at @CareerArc!", "followers_count": 460, "friends_count": 307, "statues_count": 495 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hackettstown, NJ", "id": "db6b3052eea65096", "name": "Hackettstown", "place_type": "city", "bounding_box": rectangle("-74.872448,40.806831 -74.782595,40.910458") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34041, "countyName": "Warren", "cityID": 3428710, "cityName": "Hackettstown" } }
+{ "create_at": datetime("2016-01-02T10:00:31.000Z"), "id": 683347149879439361, "text": "If u believe you dumb af https://t.co/LMmGTQHbpj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 107554360, "name": "J-Q", "screen_name": "AKAJQ", "lang": "en", "location": "France ✈️Guadeloupe✈️ Florida", "create_at": date("2010-01-22"), "description": "#RIP Elyse Louis , Journey Continue...........", "followers_count": 5531, "friends_count": 192, "statues_count": 73555 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lely, FL", "id": "2e20c95fe4a08ad6", "name": "Lely", "place_type": "city", "bounding_box": rectangle("-81.745893,26.094681 -81.712478,26.109213") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12021, "countyName": "Collier", "cityID": 1239987, "cityName": "Lely" } }
+{ "create_at": datetime("2016-01-02T10:00:31.000Z"), "id": 683347149921202176, "text": "Thats love https://t.co/ATWgsaDyQR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 615692271, "name": "---", "screen_name": "ShredderEdgar", "lang": "en", "location": "Jasmine", "create_at": date("2012-06-22"), "description": "cokey shores posh gang", "followers_count": 414, "friends_count": 328, "statues_count": 38401 }, "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-01-02T10:00:31.000Z"), "id": 683347149984145408, "text": "���� special", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2567145685, "name": "imMe™", "screen_name": "JB_dgaf", "lang": "en", "location": "Texas, USA", "create_at": date("2014-06-14"), "description": "sc: sixtwobeast ig: jb.409 LU19 #freeZeno", "followers_count": 554, "friends_count": 493, "statues_count": 10516 }, "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-02T10:00:31.000Z"), "id": 683347150135103494, "text": "@christanVeliz ha fuck ��", "in_reply_to_status": 683224432949645312, "in_reply_to_user": 892777064, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 892777064 }}, "user": { "id": 828370280, "name": "chloe", "screen_name": "chloetramel", "lang": "en", "location": "victoria, tx", "create_at": date("2012-09-16"), "description": "all i want to do is create art & eat sushi. christan is my meme king", "followers_count": 424, "friends_count": 183, "statues_count": 24771 }, "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-02T10:00:31.000Z"), "id": 683347150223245312, "text": "this is how being sucked into a fandom feels like https://t.co/OuTH1M3GFC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1851866443, "name": "✧ C3PHOE ✧", "screen_name": "onibabe_", "lang": "en", "location": "@bigboysugoi", "create_at": date("2013-09-10"), "description": "✩ ║ paige • 19 years trash • texas║✩ ↵ padme is mom. I really like star wars, sailor moon and disney. stuck in collector hell. ☹", "followers_count": 513, "friends_count": 143, "statues_count": 9936 }, "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-02T10:00:31.000Z"), "id": 683347150432899072, "text": "����������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 588747155, "name": "GoofyGates", "screen_name": "LaDaishaBriAnte", "lang": "en", "location": "null", "create_at": date("2012-05-23"), "description": "Numb. Love is stronger than pride ❗️#LongLiveKushyyyy", "followers_count": 2302, "friends_count": 1850, "statues_count": 73373 }, "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-01-02T10:00:31.000Z"), "id": 683347150605021185, "text": "@RealCapnCrunch #crunchnationentry https://t.co/CeMxjqvwUq", "in_reply_to_status": -1, "in_reply_to_user": 247826993, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "crunchnationentry" }}, "user_mentions": {{ 247826993 }}, "user": { "id": 2426152362, "name": "Cindy Stacey", "screen_name": "CindyStacey1", "lang": "en", "location": "null", "create_at": date("2014-04-03"), "description": "Wife, Mother, Grandmother - Only have Rescued Pets!!! Love Life!! Love Smiles!! Love Laughter!!", "followers_count": 530, "friends_count": 2165, "statues_count": 66058 }, "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-02T10:00:31.000Z"), "id": 683347150642622464, "text": "We're #hiring! Read about our latest #job opening here: Security Lead Officer - https://t.co/L8RUJTopPA #security #Riverside, MO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.6243649,39.162493"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "security", "Riverside" }}, "user": { "id": 1725772980, "name": "Argosy Casino KC", "screen_name": "ArgosyKC_Jobs", "lang": "en", "location": "null", "create_at": date("2013-09-03"), "description": "A Penn National Gaming property, Argosy Casino Hotel & Spa offers gaming excitement, luxury hotel rooms, fine dining and a rejuvenating spa.", "followers_count": 179, "friends_count": 137, "statues_count": 77 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, MO", "id": "f8e9521355ca84c2", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-94.659895,39.154401 -94.601419,39.189295") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29165, "countyName": "Platte", "cityID": 2962156, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-01-02T10:00:31.000Z"), "id": 683347151259348992, "text": "When your dad goes to put up curtains and ends up cleaning your whole room", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1726819981, "name": "Barbie", "screen_name": "MelindaCassidy3", "lang": "en", "location": "null", "create_at": date("2013-09-03"), "description": "class of 2016 / Khs Cheer", "followers_count": 632, "friends_count": 803, "statues_count": 11808 }, "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-02T10:00:31.000Z"), "id": 683347151502442498, "text": "CVS Health: CA Staff Pharmacist FT (#LosGatos, CA) https://t.co/5RmIIaLUqU #Healthcare #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.9746797,37.226611"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LosGatos", "Healthcare", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 21712764, "name": "TMJ- SJC Health Jobs", "screen_name": "tmj_sjc_health", "lang": "en", "location": "San Jose, CA", "create_at": date("2009-02-23"), "description": "Follow this account for geo-targeted Healthcare job tweets in San Jose, CA. Need help? Tweet us at @CareerArc!", "followers_count": 427, "friends_count": 307, "statues_count": 398 }, "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-02T10:00:31.000Z"), "id": 683347151510880256, "text": "2016 is gonna be the year I stop using group messages. Because no one ever replies 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": 73934624, "name": "Kevin Pring", "screen_name": "itskeviinn", "lang": "en", "location": "Perris, CA", "create_at": date("2009-09-13"), "description": "UCR, 21", "followers_count": 223, "friends_count": 297, "statues_count": 10329 }, "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-02T10:00:31.000Z"), "id": 683347151611674624, "text": "@loganald @CuteEmergency me on christmas", "in_reply_to_status": 683178144749207553, "in_reply_to_user": 597751444, "favorite_count": 0, "retweet_count": 0, "lang": "et", "is_retweet": false, "user_mentions": {{ 597751444, 568825492 }}, "user": { "id": 2574642017, "name": "Libby Hodgkins", "screen_name": "elibbybeth_5", "lang": "en", "location": "Be Pure -Buddha ❥❥", "create_at": date("2014-05-31"), "description": "hi Logan☯ sc: elibbybeth_5", "followers_count": 159, "friends_count": 228, "statues_count": 3303 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cumberland Center, ME", "id": "015b57d96e07430c", "name": "Cumberland Center", "place_type": "city", "bounding_box": rectangle("-70.27555,43.787066 -70.229802,43.819684") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23005, "countyName": "Cumberland", "cityID": 2315500, "cityName": "Cumberland Center" } }
+{ "create_at": datetime("2016-01-02T10:00:31.000Z"), "id": 683347151628414976, "text": "Just thinking", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 212752235, "name": "Desiré Neshea", "screen_name": "DesiRaybans", "lang": "en", "location": "America", "create_at": date("2010-11-06"), "description": "kentucky girl ❤️ The University of Louisville Alumni", "followers_count": 819, "friends_count": 734, "statues_count": 25720 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elizabethtown, KY", "id": "998c5b6035f9b5a4", "name": "Elizabethtown", "place_type": "city", "bounding_box": rectangle("-85.952048,37.626084 -85.789831,37.771861") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21093, "countyName": "Hardin", "cityID": 2124274, "cityName": "Elizabethtown" } }
+{ "create_at": datetime("2016-01-02T10:00:31.000Z"), "id": 683347151720550402, "text": "Floral Associate - Harveys: (#BLAKELY, GA) https://t.co/vPH613ZOzX #Retail #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.9488702,31.3731614"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BLAKELY", "Retail", "Job", "Jobs", "Hiring" }}, "user": { "id": 2706847724, "name": "Harveys Jobs", "screen_name": "HarveysJobs", "lang": "en", "location": "null", "create_at": date("2014-08-04"), "description": "null", "followers_count": 1, "friends_count": 0, "statues_count": 3107 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Blakely, GA", "id": "4d3002f75fa15f0c", "name": "Blakely", "place_type": "city", "bounding_box": rectangle("-84.957827,31.346041 -84.913819,31.401293") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13099, "countyName": "Early", "cityID": 1308536, "cityName": "Blakely" } }
+{ "create_at": datetime("2016-01-02T10:00:31.000Z"), "id": 683347151838003201, "text": "@Getbreadlilgreg boy move at a grandma pace lol", "in_reply_to_status": 683338992285163521, "in_reply_to_user": 2866013467, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2866013467 }}, "user": { "id": 3838384993, "name": "Nope.", "screen_name": "Dbrooooo", "lang": "en", "location": "Detroit, MI | DFW,TX", "create_at": date("2015-10-09"), "description": "New Twitter, had to focus on other things for a while & Twitter was in the way. but I'm back, tryna be more than a statistic the grind never stops #RipTusan", "followers_count": 146, "friends_count": 112, "statues_count": 4635 }, "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-02T10:00:31.000Z"), "id": 683347151922053124, "text": "Don't get pissed over resolutioners at the gym. Help them out, you were in their shoes at one point!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2740469043, "name": "Patti Balliet", "screen_name": "patti_balliet", "lang": "en", "location": "Conyngham PA/Murrells Inlet SC", "create_at": date("2014-08-11"), "description": "building the perfect beast / Hebrews 12:1", "followers_count": 184, "friends_count": 97, "statues_count": 7182 }, "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-01-02T10:00:31.000Z"), "id": 683347152081403904, "text": "I'm at UFC Gym Winter Springs in Winter Springs, FL https://t.co/RjlN3s6ZvO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.26711369,28.64651326"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26022163, "name": "Julia Mitchell", "screen_name": "JulMitchell", "lang": "en", "location": "Maryland", "create_at": date("2009-03-23"), "description": "I am from Barbados but live in Maryland. I love code in Java, HTML, SQL,CSS. Have celiac disease 10 year now and Gluten-free for 8", "followers_count": 345, "friends_count": 409, "statues_count": 8874 }, "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" } }
+{ "create_at": datetime("2016-01-02T10:00:31.000Z"), "id": 683347152161128449, "text": "HEADING TO PROVIDENCE������ https://t.co/i9RRDIg6AX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3065905169, "name": "❄Samii Claus❄", "screen_name": "TWR_nation", "lang": "en", "location": "null", "create_at": date("2015-03-01"), "description": "My closest family may not be blood, but it definitely is the best ❤ #Riotsquad #Cliffer", "followers_count": 306, "friends_count": 314, "statues_count": 11903 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westport, CT", "id": "223bb92875fd221a", "name": "Westport", "place_type": "city", "bounding_box": rectangle("-73.388984,41.094742 -73.296065,41.194542") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 983570, "cityName": "Westport" } }
+{ "create_at": datetime("2016-01-02T10:00:31.000Z"), "id": 683347152181968897, "text": "@garrettmj97 bofa who?", "in_reply_to_status": 683346768206036992, "in_reply_to_user": 2873354201, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2873354201 }}, "user": { "id": 415315667, "name": "nate", "screen_name": "NastyN8Chattos", "lang": "en", "location": "Edmond, OK", "create_at": date("2011-11-17"), "description": "business in the front, party in the back", "followers_count": 398, "friends_count": 218, "statues_count": 7282 }, "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-02T10:00:31.000Z"), "id": 683347152240795649, "text": "Missing one day isn't gonna kill yaaaaa https://t.co/Zvhi80fQkJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 379302342, "name": "S.", "screen_name": "fanniediez", "lang": "en", "location": "Miami, FL", "create_at": date("2011-09-24"), "description": "”A lioness is not submissive. She merely let's you be a king for as long as it pleases her.” Leo.", "followers_count": 3564, "friends_count": 997, "statues_count": 85582 }, "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("2016-01-02T10:00:31.000Z"), "id": 683347152404365312, "text": "SAV SAV SAV SAV SAV SAV SAAAAAAAVVVVVVVVVVVV https://t.co/mzIyeu292Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "lv", "is_retweet": false, "user": { "id": 1334714786, "name": "Caesar", "screen_name": "Latinkidd305", "lang": "en", "location": "null", "create_at": date("2013-04-07"), "description": "Poetry is the food of the mind. Art is the food of the soul. Music is the food of love, so play on. Troupe 6328. VI.VI.MMXIV.", "followers_count": 480, "friends_count": 501, "statues_count": 7110 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Port, FL", "id": "81b55f54466d8ae4", "name": "North Port", "place_type": "city", "bounding_box": rectangle("-82.272379,27.026227 -82.067993,27.111839") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12115, "countyName": "Sarasota", "cityID": 1249675, "cityName": "North Port" } }
+{ "create_at": datetime("2016-01-02T10:00:31.000Z"), "id": 683347152416866304, "text": "Want to work at Covelli Enterprises? We're #hiring in #Chillicothe, OH! Click for details: https://t.co/ia71oH7ZrU #Hospitality #parttime", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.5930338,39.2039915"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Chillicothe", "Hospitality", "parttime" }}, "user": { "id": 2352066656, "name": "Covelli Enterprises", "screen_name": "CovelliJobs", "lang": "en", "location": "null", "create_at": date("2014-02-19"), "description": "At Covelli Enterprises, we pride ourselves on our unrivaled atmosphere, outstanding customer service, and talented people. Apply to our open #jobs here!", "followers_count": 61, "friends_count": 39, "statues_count": 281 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsboro, OH", "id": "b411bb59c0ada8f6", "name": "Hillsboro", "place_type": "city", "bounding_box": rectangle("-83.641444,39.176052 -83.589688,39.239334") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39071, "countyName": "Highland", "cityID": 3935560, "cityName": "Hillsboro" } }
+{ "create_at": datetime("2016-01-02T10:00:31.000Z"), "id": 683347152748163072, "text": "Wedding date set. Guest list estimated. Time to pick a venue!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 231667907, "name": "C", "screen_name": "cvrtnyal", "lang": "en", "location": "az", "create_at": date("2010-12-28"), "description": "AZ// RA // We're all equal in the eyes of the reaper.", "followers_count": 256, "friends_count": 198, "statues_count": 20032 }, "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-02T10:00:31.000Z"), "id": 683347152756715525, "text": "The @KYScience Center has made it to our Top 50 MoonShare candidates. Read about what they are doing here: https://t.co/t44ru0pPc4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20244687 }}, "user": { "id": 1447969429, "name": "SugarlandsDistilling", "screen_name": "SugarlandsShine", "lang": "en", "location": "Gatlinburg, TN", "create_at": date("2013-05-21"), "description": "5 Star @TripAdvisor distillery experience - free samples, tours, & live music venue. Producers of Sugarlands Shine & craft whiskey #SipsUp : 21+ to follow.", "followers_count": 3963, "friends_count": 2434, "statues_count": 5832 }, "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-02T10:00:31.000Z"), "id": 683347152815276032, "text": "Can you recommend anyone for this #job? Student Delivery Driver - https://t.co/IINcJ6w6PQ #Driver #Houston, TX https://t.co/fRdXKXqZMc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.4201377,29.7765065"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Driver", "Houston" }}, "user": { "id": 2557323745, "name": "Labatt Food Service", "screen_name": "LabattFoodJobs", "lang": "en", "location": "null", "create_at": date("2014-06-09"), "description": "Labatt Food Service is a privately owned food distribution company. Follow this page for information about our open #job opportunities.", "followers_count": 152, "friends_count": 317, "statues_count": 101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-02T10:00:31.000Z"), "id": 683347152937074688, "text": "These little STEAMSTERSSTOPPED by the booth to grab the #mobileapp do you have the Steam App.… https://t.co/gfouOFD5u8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.45819961,40.12380216"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "mobileapp" }}, "user": { "id": 3089647052, "name": "Project S.T.E.A.M.TV", "screen_name": "ProjectSTEAMtv", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2015-03-16"), "description": "Media-based non profit supporting children in grades 6-12, encouraging them to know the importance of Science, Technology, Engineering, Art, & Mathematics.", "followers_count": 142, "friends_count": 303, "statues_count": 512 }, "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": 42029, "countyName": "Chester" } }
+{ "create_at": datetime("2016-01-02T10:00:31.000Z"), "id": 683347153306058753, "text": "I don't fw none of you niggas!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3191362176, "name": "savage", "screen_name": "_babytank3", "lang": "en", "location": "SomeWhereShootingInnaGym!", "create_at": date("2015-05-10"), "description": "null", "followers_count": 566, "friends_count": 353, "statues_count": 9549 }, "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-02T10:00:32.000Z"), "id": 683347153427771396, "text": "@ilovejewlia ooooo reoccurring dreams are even more intriguing.. look for key symbols and feelings that u could compare to ur values������", "in_reply_to_status": 683346853413437440, "in_reply_to_user": 64482765, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 64482765 }}, "user": { "id": 60363113, "name": "Sॐ", "screen_name": "SummerKuhn", "lang": "en", "location": "Long Island, NY", "create_at": date("2009-07-26"), "description": "All riches have their beginning in an idea", "followers_count": 1102, "friends_count": 2023, "statues_count": 14765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakdale, NY", "id": "17ed72e98159e413", "name": "Oakdale", "place_type": "city", "bounding_box": rectangle("-73.157725,40.725255 -73.111969,40.755056") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3654144, "cityName": "Oakdale" } }
+{ "create_at": datetime("2016-01-02T10:00:32.000Z"), "id": 683347153448644610, "text": "Guys can we just take a moment... https://t.co/D1P4XvZuF0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28874945, "name": "s h e a", "screen_name": "shaywiener", "lang": "en", "location": "probably eating a cupcake", "create_at": date("2009-04-04"), "description": "future US soldier that's addicted to zumba with a hot Army boyfriend", "followers_count": 696, "friends_count": 493, "statues_count": 50691 }, "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-01-02T10:00:32.000Z"), "id": 683347153981431809, "text": "@william10225797 Nothing on the immediate horizon but I'd love to do more. Thanks!", "in_reply_to_status": 683322619677020160, "in_reply_to_user": 2171223427, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2171223427 }}, "user": { "id": 51360498, "name": "Michael Price", "screen_name": "mikepriceinla", "lang": "en", "location": "Los Angeles", "create_at": date("2009-06-26"), "description": "SIMPSONS writer, everything else watcher. Co-Creator and Showrunner of the Netflix animated series F IS FOR FAMILY, and EP/Writer on LEGO STAR WARS:DROID TALES.", "followers_count": 7655, "friends_count": 1529, "statues_count": 25745 }, "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-02T10:00:32.000Z"), "id": 683347154081980416, "text": "@adolforivera_ 4-0", "in_reply_to_status": 683328864286674946, "in_reply_to_user": 1948515084, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1948515084 }}, "user": { "id": 723480968, "name": "Alexander", "screen_name": "Dc5_Alexx", "lang": "en", "location": "Mount Haruna ", "create_at": date("2012-07-28"), "description": "Auto enthusiast.  I am Alpha and Omega, the beginning and the end.\nGarage bum.", "followers_count": 340, "friends_count": 391, "statues_count": 10600 }, "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-02T10:00:32.000Z"), "id": 683347154455269376, "text": "https://t.co/dnF2uNeo0K", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 598770957, "name": "Evy ⛳️", "screen_name": "KingEvy_", "lang": "en", "location": "47 Floors Up", "create_at": date("2012-06-03"), "description": "Dress it up and make it real for me", "followers_count": 1569, "friends_count": 937, "statues_count": 93068 }, "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-02T10:00:32.000Z"), "id": 683347154555908096, "text": "Interested in a #Nursing #job near #Valdosta, GA? This could be a great fit: https://t.co/yTwTwdT8BR #RN #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.2863,30.8625"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "Valdosta", "RN", "Hiring" }}, "user": { "id": 2842711225, "name": "SGMC Careers", "screen_name": "SGMCCareers", "lang": "en", "location": "Valdosta, Georgia", "create_at": date("2014-10-06"), "description": "SGMC offers employees a warm and welcoming environment in a rapidly growing, dynamic organization. Follow us to learn more about our openings!", "followers_count": 64, "friends_count": 33, "statues_count": 356 }, "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-01-02T10:00:32.000Z"), "id": 683347154639785984, "text": "@SheriLeigh7 @Salon Rates of uninsured falling across states: https://t.co/4TXobxfF3E so obviously *millions* can afford it.", "in_reply_to_status": 683343838325280768, "in_reply_to_user": 3318629362, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3318629362, 16955991 }}, "user": { "id": 59865242, "name": "Paul Rosenberg", "screen_name": "PaulHRosenberg", "lang": "en", "location": "Long Beach, CA", "create_at": date("2009-07-24"), "description": "null", "followers_count": 3115, "friends_count": 478, "statues_count": 34134 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-02T10:00:32.000Z"), "id": 683347154681765888, "text": "Should've won the Heisman https://t.co/WvhFzqwKcV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2406187321, "name": "chris murphy", "screen_name": "chrismurphy07", "lang": "en", "location": "null", "create_at": date("2014-03-22"), "description": "Varsity baseball. Ghchs. senior. 2015 area code participant. ⚾️ USD baseball commit #Toreros", "followers_count": 495, "friends_count": 635, "statues_count": 1029 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-02T10:00:32.000Z"), "id": 683347154757382146, "text": "Miss waking up to this mess :((( https://t.co/gNWTn2jEBB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 260558933, "name": "ℜatchet", "screen_name": "_rachelhowardd", "lang": "en", "location": "null", "create_at": date("2011-03-03"), "description": "K82& K83 ✟ 201 Davis ❂ Drake!¡ O'hara15; TU19 delco", "followers_count": 1016, "friends_count": 743, "statues_count": 39422 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Folsom, PA", "id": "c8d23c1f5d6736ea", "name": "Folsom", "place_type": "city", "bounding_box": rectangle("-75.341968,39.882641 -75.29943,39.916604") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42045, "countyName": "Delaware", "cityID": 4226432, "cityName": "Folsom" } }
+{ "create_at": datetime("2016-01-02T10:00:32.000Z"), "id": 683347155046633472, "text": "�� https://t.co/OvWL6z3onN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2345605688, "name": "Maya♛♡", "screen_name": "maayuhhh", "lang": "en", "location": "null", "create_at": date("2014-02-15"), "description": "Follow me on insta : Maayuhhh", "followers_count": 1027, "friends_count": 892, "statues_count": 21062 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-02T10:00:32.000Z"), "id": 683347155109588992, "text": "See our latest #Chicago, IL #job and click to apply: Account Representative - https://t.co/eseghsrx8N #sales #Sales #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", "sales", "Sales", "Hiring" }}, "user": { "id": 1425504374, "name": "CORT Careers", "screen_name": "CORTCareers", "lang": "en", "location": "Nationwide", "create_at": date("2013-05-13"), "description": "Join the CORT team where we help make a house a home, an office a great place to work, and an event a memorable celebration. @CORTFurniture", "followers_count": 662, "friends_count": 492, "statues_count": 1190 }, "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-02T10:00:32.000Z"), "id": 683347155189284864, "text": "New Signature #IT #Job: Microsoft Unified Communications Engineer (Toronto, or remote) (#Atlanta, GA) https://t.co/And5511075 #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.39,33.76"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "IT", "Job", "Atlanta", "Jobs", "Hiring" }}, "user": { "id": 4499586313, "name": "New Signature Jobs", "screen_name": "NewSigCareers", "lang": "en", "location": "null", "create_at": date("2015-12-08"), "description": "New Signature is hiring qualified professionals with a background in #information #technology, #computer #science, #web #development or #graphic #design.", "followers_count": 10, "friends_count": 83, "statues_count": 22 }, "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-02T10:00:32.000Z"), "id": 683347155227119616, "text": "Breakfast = �� (at @Arbys in Sun City West, AZ) https://t.co/vaqzMmwRpn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.35497696,33.65420455"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 155544257 }}, "user": { "id": 138243141, "name": "Freddie Velez", "screen_name": "NiceFreddie", "lang": "en", "location": "33.490274, -112.32406", "create_at": date("2010-04-28"), "description": "New York transplant in AZ, I'm a nice guy most of the time and occasionally I say something smart  Instagram: nicefreddie", "followers_count": 188, "friends_count": 391, "statues_count": 16723 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sun City West, AZ", "id": "859948b6ebcbbf25", "name": "Sun City West", "place_type": "city", "bounding_box": rectangle("-112.409262,33.638577 -112.311175,33.706973") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 470355, "cityName": "Sun City West" } }
+{ "create_at": datetime("2016-01-02T10:00:32.000Z"), "id": 683347155264778241, "text": "The Vitamin Shoppe #HealthWelfare #Job: Health Enthusiast Part-Time (#Baltimore, MD) https://t.co/w0U5iwhvJn #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.564368,39.2771025"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HealthWelfare", "Job", "Baltimore", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 21221744, "name": "Baltimore Health Job", "screen_name": "tmj_bal_health", "lang": "en", "location": "Baltimore, MD", "create_at": date("2009-02-18"), "description": "Follow this account for geo-targeted Healthcare job tweets in Baltimore, MD. Need help? Tweet us at @CareerArc!", "followers_count": 575, "friends_count": 305, "statues_count": 577 }, "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-02T10:00:32.000Z"), "id": 683347155600470016, "text": "https://t.co/glxzlecvuK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 35592477, "name": "Demonta B", "screen_name": "_TaeSays", "lang": "en", "location": "Small cities, Big dreams", "create_at": date("2009-04-26"), "description": "CMU Senior • AΦA 6-PΔ-Spring 15' • Instagram: Taesays06 810 & 989", "followers_count": 854, "friends_count": 735, "statues_count": 65292 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delray Beach, FL", "id": "b4a0e9082dcc8436", "name": "Delray Beach", "place_type": "city", "bounding_box": rectangle("-80.13094,26.420813 -80.05542,26.491482") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1217100, "cityName": "Delray Beach" } }
+{ "create_at": datetime("2016-01-02T10:00:32.000Z"), "id": 683347155872911360, "text": "now the pressure is on lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 503507892, "name": "dennis litt", "screen_name": "Br0keToke", "lang": "en", "location": "One Way, CT", "create_at": date("2012-02-25"), "description": "management.stray@gmail.com | @TheStraySociety | @45IVEMAG | #FreeMook #MookieBanks #OneWay #EarlWorld #StillSendSoup", "followers_count": 1267, "friends_count": 372, "statues_count": 37573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Half Moon Bay, CA", "id": "d0fc0f618c1eb790", "name": "Half Moon Bay", "place_type": "city", "bounding_box": rectangle("-122.477157,37.424863 -122.418544,37.524472") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 631708, "cityName": "Half Moon Bay" } }
+{ "create_at": datetime("2016-01-02T10:00:32.000Z"), "id": 683347155919200257, "text": "@_SupremeCraze oh naw I was at least in and out lol u made an appointment?", "in_reply_to_status": 683346988822360064, "in_reply_to_user": 229481270, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 229481270 }}, "user": { "id": 167456652, "name": "B.E.A.N.S✊", "screen_name": "Beansathon", "lang": "en", "location": "somewhere past u probably ", "create_at": date("2010-07-16"), "description": "12/8 ---- Being Everything A Nigga Should ┌П┐(•_•)┌П┐", "followers_count": 755, "friends_count": 667, "statues_count": 118100 }, "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-02T10:00:32.000Z"), "id": 683347156078587904, "text": "Blink 182 is on the radio����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3129494097, "name": "Clare Mazzei", "screen_name": "ClareMazzei", "lang": "en", "location": "null", "create_at": date("2015-03-30"), "description": "put that thing back where it came from or so help me", "followers_count": 181, "friends_count": 184, "statues_count": 539 }, "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-01-02T10:00:32.000Z"), "id": 683347156212699136, "text": "We're #hiring! Read about our latest #job opening here: Retail Cosmetics Sales - Counter Manager Fashion... - https://t.co/Dmdu3wQnvw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.6012561,39.4037717"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "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": 670, "friends_count": 0, "statues_count": 8387 }, "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-02T10:00:32.000Z"), "id": 683347156300906496, "text": "@lenaagaudio omg same .. I'm like okay I need at least 3 shots before I can carry on a conversation with a stranger ...", "in_reply_to_status": 683347001266814978, "in_reply_to_user": 280823613, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 280823613 }}, "user": { "id": 332459254, "name": "Sarah Cline", "screen_name": "scliney", "lang": "en", "location": "null", "create_at": date("2011-07-09"), "description": "null", "followers_count": 595, "friends_count": 579, "statues_count": 8682 }, "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-02T10:00:32.000Z"), "id": 683347156321746944, "text": "Can you recommend anyone for this #job? Retail Management Sales Lead - https://t.co/91k9NSXDaS #TwinFalls, ID #Retail #Veterans #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-114.4630615,42.5891895"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "TwinFalls", "Retail", "Veterans", "Hiring" }}, "user": { "id": 59921585, "name": "TMJ-ID Retail Jobs", "screen_name": "tmj_id_retail", "lang": "en", "location": "Idaho", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in Idaho Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 351, "friends_count": 302, "statues_count": 102 }, "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-01-02T10:00:32.000Z"), "id": 683347156489637888, "text": "Today is going by tooooo slow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 460693464, "name": "tori waters", "screen_name": "torireneaa", "lang": "en", "location": "sweet home alabama", "create_at": date("2012-01-10"), "description": "I can do all things through Christ who strenghtens me. Philippians 4:13", "followers_count": 505, "friends_count": 300, "statues_count": 21710 }, "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-02T10:00:32.000Z"), "id": 683347156489637889, "text": "Cuba Cuba!!!!!!!!!........�������� https://t.co/Y6Y26n7BPh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 4122137177, "name": "LadyLoquita", "screen_name": "damaris_bernia", "lang": "es", "location": "Cuba", "create_at": date("2015-11-05"), "description": "I ❤ my family and my friends\nI love Cuba. Amante de la musica\nI love singing and dancing.\n[Bautister] [Mahomie] [Bilieber]\nI love CNCO", "followers_count": 317, "friends_count": 669, "statues_count": 1399 }, "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-02T10:00:32.000Z"), "id": 683347156665667584, "text": "This book is stressing me out man ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 985195986, "name": "Des Lama :)", "screen_name": "DesireeLama", "lang": "en", "location": "Lost in TX ", "create_at": date("2012-12-02"), "description": "i like simple things, books, being alone, or with somebody who understands ; always and forever", "followers_count": 745, "friends_count": 351, "statues_count": 43299 }, "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-02T10:00:32.000Z"), "id": 683347156695003136, "text": "@_jessicagarcia1 eat healthy", "in_reply_to_status": 683346985521315840, "in_reply_to_user": 1355740674, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1355740674 }}, "user": { "id": 439288448, "name": "CARTER™", "screen_name": "Mr_Carter_78", "lang": "en", "location": "somewhere over the rainbow, TX", "create_at": date("2011-12-17"), "description": "#SLICC IG:@Mr_Carter_78 snapchat: daddycarter78", "followers_count": 498, "friends_count": 488, "statues_count": 40575 }, "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-02T10:00:32.000Z"), "id": 683347156753854469, "text": "@torisbrady NO YOUR HAIR IS PERFECT", "in_reply_to_status": 683342018634977281, "in_reply_to_user": 52773876, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 52773876 }}, "user": { "id": 1219679197, "name": "rachael❂lorene", "screen_name": "rachaellorene", "lang": "en", "location": "lost in the sauce", "create_at": date("2013-02-25"), "description": "If I can't have love I'll take sunshine.", "followers_count": 1063, "friends_count": 1125, "statues_count": 64128 }, "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-02T10:00:32.000Z"), "id": 683347156867112960, "text": "Im in a rush to lay on my couch lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 85074579, "name": "Free", "screen_name": "Bouji_one", "lang": "en", "location": "ÜT: 36.814247,-76.25721", "create_at": date("2009-10-25"), "description": "Yes I've made mistakes..... And I've corrected most and grew from them..... Questions? Im the writer and tweeter of my own life so sit back and enjoy :)", "followers_count": 331, "friends_count": 574, "statues_count": 11678 }, "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-02T10:00:32.000Z"), "id": 683347157030539264, "text": "#CareerArc #Retail #Job alert: Associate Manager (MIT) | Maurices | #OsageBeach, MO https://t.co/rwaSwJmGI1 #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.6179621,38.1503113"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "CareerArc", "Retail", "Job", "OsageBeach", "Jobs", "Hiring" }}, "user": { "id": 59967225, "name": "TMJ-MO Retail Jobs", "screen_name": "tmj_mo_retail", "lang": "en", "location": "Missouri", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in Missouri Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 469, "friends_count": 301, "statues_count": 655 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Osage Beach, MO", "id": "011c22c245e1c33b", "name": "Osage Beach", "place_type": "city", "bounding_box": rectangle("-92.643456,38.122685 -92.59596,38.175551") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29029, "countyName": "Camden", "cityID": 2955244, "cityName": "Osage Beach" } }
+{ "create_at": datetime("2016-01-02T10:00:32.000Z"), "id": 683347157383024640, "text": "I don't get how ppl didn't like spiderman 3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 345451684, "name": "gran santo", "screen_name": "agudze_", "lang": "en", "location": "null", "create_at": date("2011-07-30"), "description": "i don't forget or forgive.", "followers_count": 1294, "friends_count": 923, "statues_count": 72028 }, "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-02T10:00:32.000Z"), "id": 683347157521317888, "text": "@brycep24 or not", "in_reply_to_status": 683347060213465088, "in_reply_to_user": 551265397, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 551265397 }}, "user": { "id": 417612933, "name": "Desirae Lindow", "screen_name": "Desirae_Lindow", "lang": "en", "location": "i wish it would snow ", "create_at": date("2011-11-20"), "description": "I just really like the piano, reading things that rhyme, hot chocolate, and going places I've never been // I like elephants and politics", "followers_count": 514, "friends_count": 272, "statues_count": 5776 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buckhannon, WV", "id": "3cb2ec133f4934f3", "name": "Buckhannon", "place_type": "city", "bounding_box": rectangle("-80.27512,38.971888 -80.195065,39.009095") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54097, "countyName": "Upshur", "cityID": 5411188, "cityName": "Buckhannon" } }
+{ "create_at": datetime("2016-01-02T10:00:33.000Z"), "id": 683347157601153025, "text": "Support my sis Love Is Everything\"\nby @AntoniqueSmith\nAvailable Now on iTunes > https://t.co/VyXV9YlLsN Get it!\nhttps://t.co/wTxe77OFQo\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27787064 }}, "user": { "id": 89336239, "name": "Built2Win", "screen_name": "ladysmiles10", "lang": "en", "location": "Queen of Michigan ", "create_at": date("2009-11-11"), "description": "Hip Hop Artist & Songwriter! The voice of the society! #Bo$$Chick #Unsigned #MuskegonFirstLady inquiries contact: ladysmiles10@gmail.com", "followers_count": 5835, "friends_count": 2327, "statues_count": 15697 }, "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-02T10:00:33.000Z"), "id": 683347157647110144, "text": "I'm realistic, I sit down and draw out logical reasons why I feel the way I do and fix them depending on if I can 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": 54399294, "name": "another level", "screen_name": "TopherThangz", "lang": "en", "location": "Lafayette, LA", "create_at": date("2009-07-06"), "description": "i love tweeting my unpopular opinion", "followers_count": 1090, "friends_count": 531, "statues_count": 60579 }, "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-02T10:00:33.000Z"), "id": 683347157722611712, "text": "Bottomless mimosa brunch is the best way to start a day off. ☺️���� @ The Yard Market Square https://t.co/mEffOTzJ4g", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.0028687,40.4413986"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 513861562, "name": "Colleen Wilde", "screen_name": "ColleenWilde", "lang": "en", "location": "Pittsburgh, Pennsylvania", "create_at": date("2012-03-03"), "description": "Nature. Moon. Wisdom. Progress. Movement. Love. Wild. Woman.\nApostle of Beauty.\nInternational Bellydance & Movement Artist", "followers_count": 91, "friends_count": 201, "statues_count": 1496 }, "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-02T10:00:33.000Z"), "id": 683347157743734785, "text": "Lol seton hall. Really hope DePaul can tie it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 50658429, "name": "Showey D", "screen_name": "ShoweyD", "lang": "en", "location": "Cleveland, OH", "create_at": date("2009-06-25"), "description": "I enjoy Dayton Flyer basketball and marginally trained bulldogs.", "followers_count": 180, "friends_count": 96, "statues_count": 21479 }, "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-01-02T10:00:33.000Z"), "id": 683347157919793152, "text": "Can you recommend anyone for this #job? Pharmacy Technician - https://t.co/LwVuUXymc8 #pharmacy #Sacramento, CA #Healthcare #Veterans", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.4943996,38.5815719"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "pharmacy", "Sacramento", "Healthcare", "Veterans" }}, "user": { "id": 2786536903, "name": "Raley's Jobs", "screen_name": "raleysjobs", "lang": "en", "location": "null", "create_at": date("2014-09-02"), "description": "Feed your passion at Raley’s; join our team! Job opportunities you can grow your career with. Get news, recipes and more @raleys.", "followers_count": 73, "friends_count": 1, "statues_count": 1528 }, "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-02T10:00:33.000Z"), "id": 683347157957488640, "text": "Penn St receiving core is solid", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 235371680, "name": "Pesos Martinez", "screen_name": "SoleDreams", "lang": "en", "location": "405, Oklahoma ", "create_at": date("2011-01-07"), "description": "CFL Cornerback, #21", "followers_count": 2484, "friends_count": 1338, "statues_count": 144554 }, "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-02T10:00:33.000Z"), "id": 683347158259482625, "text": "@XenithFootball https://t.co/8oAEkq79yA", "in_reply_to_status": -1, "in_reply_to_user": 196177223, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 196177223 }}, "user": { "id": 2938679298, "name": "TexasRanger", "screen_name": "walkernunez37", "lang": "en", "location": "On the turf or the dirt ", "create_at": date("2014-12-21"), "description": "Hendrickson High School co'17 | God, Family, Base/foot Ball | Matthew 14:29-31 | E.M.K. ❤️", "followers_count": 350, "friends_count": 381, "statues_count": 1223 }, "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-02T10:00:33.000Z"), "id": 683347158662135808, "text": "We're #hiring! Read about our latest #job opening here: US Government Relations GLOBE Intern - https://t.co/u4P9YyKyuG #Arlington, VA #HR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.0881232,38.8880747"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Arlington", "HR" }}, "user": { "id": 472201672, "name": "Nature Conservancy", "screen_name": "nature_careers", "lang": "en", "location": "Worldwide", "create_at": date("2012-01-23"), "description": "Want to work for an organization you can believe in? The Nature Conservancy is a leading conservation organization working for people and nature.", "followers_count": 2063, "friends_count": 207, "statues_count": 194 }, "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-02T10:00:33.000Z"), "id": 683347158779703298, "text": "Can you spot me with all that snow around @ Hinsdale, Massachusetts https://t.co/ixCiVLLxMG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.1258,42.4386"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2209768815, "name": "liz™", "screen_name": "bat_mango", "lang": "en", "location": "16 | ♋️ | falls church", "create_at": date("2013-12-05"), "description": "Judgement Day's a runaway train, we just want off", "followers_count": 403, "friends_count": 347, "statues_count": 23679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hinsdale, MA", "id": "00d56ef23bbe93cc", "name": "Hinsdale", "place_type": "city", "bounding_box": rectangle("-73.132637,42.42231 -73.112852,42.452887") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25003, "countyName": "Berkshire" } }
+{ "create_at": datetime("2016-01-02T10:00:33.000Z"), "id": 683347158897147907, "text": "No thank you on Payton. I'd be OK with Saban. Smart money on McAdoo promotion. https://t.co/BUWyeUbJjs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 199544206, "name": "Nick Marino", "screen_name": "THE_NickMarino", "lang": "en", "location": "The Trap, ATL", "create_at": date("2010-10-06"), "description": "Follow me on Twitter, I'll be happy. Follow me in person, I'll have you arrested.", "followers_count": 295, "friends_count": 418, "statues_count": 10265 }, "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-02T10:00:33.000Z"), "id": 683347159157202948, "text": "@AriesSpears <--it's done!!", "in_reply_to_status": 683344945202229248, "in_reply_to_user": 30887601, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 30887601 }}, "user": { "id": 27852214, "name": "Akisha Cassermere", "screen_name": "AkishaComedy", "lang": "en", "location": "Connecticut", "create_at": date("2009-03-31"), "description": "I'm a new jack Comedienne that gives back 2 my community here -- @boffusa", "followers_count": 536, "friends_count": 826, "statues_count": 29637 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CT", "id": "7b62343e287fb7c0", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-73.085611,41.245045 -72.979996,41.31407") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 957670, "cityName": "Orange" } }
+{ "create_at": datetime("2016-01-02T10:00:33.000Z"), "id": 683347159203229696, "text": "George r r Martin might be the most frustrating author of this decade. 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": 930570510, "name": "Jeremiah", "screen_name": "lebontempkid", "lang": "en", "location": "null", "create_at": date("2012-11-06"), "description": "Actor,amateur photographer, Constantly falling down rabbit holes...come and join me.", "followers_count": 1054, "friends_count": 1494, "statues_count": 11451 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-02T10:00:33.000Z"), "id": 683347159241089024, "text": "#salad #season \nLol @ Cantina Laredo King of Prussia https://t.co/6pYqJlpcdo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.3851308,40.08946881"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "salad", "season" }}, "user": { "id": 36297137, "name": "Phillyrican", "screen_name": "Philly_Rican", "lang": "en", "location": "Philadelphia,South Philly", "create_at": date("2009-04-28"), "description": "null", "followers_count": 245, "friends_count": 580, "statues_count": 1507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "King of Prussia, PA", "id": "23b12afb6fe7507a", "name": "King of Prussia", "place_type": "city", "bounding_box": rectangle("-75.420016,40.071936 -75.343479,40.117633") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42091, "countyName": "Montgomery", "cityID": 4239736, "cityName": "King of Prussia" } }
+{ "create_at": datetime("2016-01-02T10:00:33.000Z"), "id": 683347159316447232, "text": "I know one of them�� https://t.co/VJo7eb00I3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3355914493, "name": "Te'Anna", "screen_name": "t_janks", "lang": "en", "location": "5 0 1 ❤️", "create_at": date("2015-08-26"), "description": "UCArk | Queen | P3130 | sc: @teejanks", "followers_count": 163, "friends_count": 276, "statues_count": 1606 }, "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-02T10:00:33.000Z"), "id": 683347159379382272, "text": "It's lowkey a lot of differences", "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": "3. January", "screen_name": "KourtneiCt", "lang": "en", "location": "null", "create_at": date("2013-12-13"), "description": "20. dalla$", "followers_count": 1265, "friends_count": 693, "statues_count": 30231 }, "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-02T10:00:33.000Z"), "id": 683347159547183105, "text": "@mhonividente #preguntaleaMhoni #preguntaleaMhoni Hola Mhoni como me ves de salud?11/29/1977", "in_reply_to_status": -1, "in_reply_to_user": 59094592, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "preguntaleaMhoni", "preguntaleaMhoni" }}, "user_mentions": {{ 59094592 }}, "user": { "id": 50579054, "name": "NANCY SALAS", "screen_name": "NANCEE1729", "lang": "en", "location": "null", "create_at": date("2009-06-25"), "description": "null", "followers_count": 4, "friends_count": 133, "statues_count": 25 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-02T10:00:33.000Z"), "id": 683347159731683328, "text": "@hunter_owens @pkedrosky Yes. Plenty of actuarial charts online", "in_reply_to_status": 683303229048930304, "in_reply_to_user": 18616963, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18616963, 1717291 }}, "user": { "id": 18034072, "name": "Don Surber", "screen_name": "donsurber", "lang": "en", "location": "Poca WV", "create_at": date("2008-12-10"), "description": "Retired. Author of EXCEPTIONAL AMERICANS 1 http://t.co/VVsXoqxSM9 EXCEPTIONAL AMERICANS 2: THE CAPITALISTS http://t.co/IQYwFlhQnO", "followers_count": 2805, "friends_count": 82, "statues_count": 30198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Poca, WV", "id": "001a9f0813d7a82f", "name": "Poca", "place_type": "city", "bounding_box": rectangle("-81.825749,38.43513 -81.799353,38.490068") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54079, "countyName": "Putnam", "cityID": 5464516, "cityName": "Poca" } }
+{ "create_at": datetime("2016-01-02T10:00:33.000Z"), "id": 683347160172216320, "text": "you scared na", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2997551977, "name": "yeaa it's me :))", "screen_name": "theybeonher", "lang": "en", "location": "null", "create_at": date("2015-01-26"), "description": "one life , two chances", "followers_count": 1066, "friends_count": 709, "statues_count": 33766 }, "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-02T10:00:33.000Z"), "id": 683347160235032576, "text": "awh man ������ he wan fight ���������� https://t.co/lDV5CHuAVR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 300013108, "name": "a$ia ayee .", "screen_name": "_asiar", "lang": "en", "location": "null", "create_at": date("2011-05-16"), "description": "I'm so Louisiana #UL@Lafayette❤️ I'll never give up on myself .| young rounds .", "followers_count": 2327, "friends_count": 1585, "statues_count": 51837 }, "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-02T10:00:33.000Z"), "id": 683347160272797696, "text": "@Spidey_Grips use the best to be the best visit https://t.co/AhDhSs2Y9p use the code joker for 10% off u wont regret it stay furious peeps", "in_reply_to_status": -1, "in_reply_to_user": 3198600293, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3198600293 }}, "user": { "id": 4168351522, "name": "MAK S4DISTIC JOKER", "screen_name": "S4DISTICJOKER", "lang": "en", "location": "Gotham City", "create_at": date("2015-11-11"), "description": "xb1 S4DISTIC JOKER MAK team member,sponsored by @Spidey_Grips, @brimanscustoms, and @westcoastchill. Gamer,streamer, father, http://youtube.com/channel/UCNW52…", "followers_count": 711, "friends_count": 681, "statues_count": 10285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenfield, WI", "id": "7f42154616cee8fa", "name": "Greenfield", "place_type": "city", "bounding_box": rectangle("-88.069295,42.930385 -87.948504,42.988186") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5531175, "cityName": "Greenfield" } }
+{ "create_at": datetime("2016-01-02T10:00:33.000Z"), "id": 683347160348266496, "text": "Director of Content Engineering - Avention, Inc.: (#Concord, MA) https://t.co/zKksdeYkgC #Marketing #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.349444,42.460278"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Concord", "Marketing", "Job", "Jobs", "Hiring" }}, "user": { "id": 2902955634, "name": "Avention, Inc.", "screen_name": "AventionCareers", "lang": "en", "location": "Global", "create_at": date("2014-11-17"), "description": "@AventionInc. is a global leader in business information and sales enablement. Follow us here for information about our open #jobs.", "followers_count": 9, "friends_count": 3, "statues_count": 5 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Concord, MA", "id": "00cd6f931dd01b13", "name": "Concord", "place_type": "city", "bounding_box": rectangle("-71.422775,42.41897 -71.317384,42.478626") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex" } }
+{ "create_at": datetime("2016-01-02T10:00:33.000Z"), "id": 683347160503455744, "text": "@andremarinpuig es de sabios reconocer sus errores.", "in_reply_to_status": 683339802725318657, "in_reply_to_user": 115558650, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 115558650 }}, "user": { "id": 424217204, "name": "arturo ortiz", "screen_name": "elcobritas", "lang": "en", "location": "null", "create_at": date("2011-11-29"), "description": "null", "followers_count": 107, "friends_count": 319, "statues_count": 10708 }, "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-02T10:00:33.000Z"), "id": 683347160545361920, "text": "Interested in a #Hospitality #job near #Pasadena, TX? This could be a great fit: https://t.co/YJe0otftt3 #IHOP #htx #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.1553944,29.666862"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Pasadena", "IHOP", "htx", "Hiring" }}, "user": { "id": 216428496, "name": "IHOP_ACG", "screen_name": "IHOP_ACGTexas", "lang": "en", "location": "Richardson, TX", "create_at": date("2010-11-16"), "description": "ACG Texas is proud to be Texas' largest #IHOP Franchisee. #Hospitality #jobs in 66 restaurants in Texas. Join our Team!", "followers_count": 311, "friends_count": 619, "statues_count": 713 }, "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-01-02T10:00:33.000Z"), "id": 683347160784605185, "text": "@mandyyypeterson I can't wait either!!! �� And that is SO exciting!! So happy you'll get to be with family! Are you telling them then?!", "in_reply_to_status": 683346573485543425, "in_reply_to_user": 2888661616, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2888661616 }}, "user": { "id": 30376612, "name": "Maddie Ray", "screen_name": "themaddieray", "lang": "en", "location": "null", "create_at": date("2009-04-10"), "description": "I like Jesus, sports, pictures, Cody Ray and singing loudly. I also do this college thing.", "followers_count": 870, "friends_count": 827, "statues_count": 8288 }, "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-02T10:00:33.000Z"), "id": 683347160893657089, "text": "It better get cold cause I didn't throw this big ass coat on 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": 595778791, "name": "CVS Thug", "screen_name": "PinasNPalmTrees", "lang": "en", "location": "null", "create_at": date("2012-05-31"), "description": "Im a legend dead or alive #CRC", "followers_count": 551, "friends_count": 351, "statues_count": 28179 }, "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-02T10:00:33.000Z"), "id": 683347161048809474, "text": "Preachhhhhhhhhh https://t.co/943Y3v2IYr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 42497437, "name": "izzy kabemba", "screen_name": "izzy_its_real", "lang": "en", "location": "west haven ct ", "create_at": date("2009-05-25"), "description": "first love is football #WCSU17 #NEWYORKGIANTS #BostonCeltics", "followers_count": 507, "friends_count": 561, "statues_count": 16006 }, "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-01-02T10:00:33.000Z"), "id": 683347161187131392, "text": "@piper_diaper I burnt my face, I'll sc you. It ain't cute����", "in_reply_to_status": 683347036544970752, "in_reply_to_user": 1928255353, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1928255353 }}, "user": { "id": 549472191, "name": "Sara Danielle", "screen_name": "SaraHoeddy", "lang": "en", "location": "null", "create_at": date("2012-04-09"), "description": "omg Hannah shut up ΣΣΣ⛵️", "followers_count": 387, "friends_count": 442, "statues_count": 5829 }, "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-01-02T10:00:33.000Z"), "id": 683347161208078336, "text": "Kindred At Home #Nursing #Job: Full Time Registered Nurse-Kindred at Home(Danville,... (#Indianapolis, Indiana) https://t.co/2QAB10HguQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.1580423,39.7683765"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "Job", "Indianapolis" }}, "user": { "id": 22158864, "name": "Indianapolis Nursing", "screen_name": "tmj_ind_nursing", "lang": "en", "location": "Indianapolis, IN", "create_at": date("2009-02-27"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Indianapolis, IN. Need help? Tweet us at @CareerArc!", "followers_count": 535, "friends_count": 302, "statues_count": 496 }, "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-01-02T10:00:33.000Z"), "id": 683347161422114821, "text": "@BleacherReport was it a soccer match?", "in_reply_to_status": 683341670595870720, "in_reply_to_user": 890891, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 890891 }}, "user": { "id": 74052575, "name": "Yatrix", "screen_name": "Yatrix7", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2009-09-13"), "description": "null", "followers_count": 19, "friends_count": 50, "statues_count": 2833 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin Park, PA", "id": "95151c2c72f5468c", "name": "Franklin Park", "place_type": "city", "bounding_box": rectangle("-80.14492,40.542221 -80.06405,40.616223") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4227552, "cityName": "Franklin Park" } }
+{ "create_at": datetime("2016-01-02T10:00:34.000Z"), "id": 683347161979994116, "text": "UD's Dancers are ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2763686651, "name": "Jordan.", "screen_name": "jor3__", "lang": "en", "location": "Delaware, USA", "create_at": date("2014-09-06"), "description": "null", "followers_count": 314, "friends_count": 541, "statues_count": 1064 }, "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-01-02T10:00:34.000Z"), "id": 683347162231607296, "text": "The South Florida Disciples 6th Grade Boys team with the 55-45 win over @browardsharks1 #FloridaBasketballBulletin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FloridaBasketballBulletin" }}, "user_mentions": {{ 3098540611 }}, "user": { "id": 236160906, "name": "Tony Perkins (ΚΑΨ)", "screen_name": "tperkinsbball23", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2011-01-09"), "description": "Owner & Director of Scouting for the Florida Basketball Bulletin & Las Vegas Greyhounds Basketball Club.", "followers_count": 2748, "friends_count": 2711, "statues_count": 6816 }, "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-01-02T10:00:34.000Z"), "id": 683347162252587009, "text": "\"why is there a bible in your bed\"\n\"I'm very religious\" @TheMBJR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 561683526 }}, "user": { "id": 635586203, "name": "ใіƒє оƒ яєіιιч", "screen_name": "_SmiLey_REiLLY", "lang": "en", "location": "215//609", "create_at": date("2012-07-14"), "description": "now you understand why Peter Pan didn't want to grow up | 19115//08260 | TU '19", "followers_count": 911, "friends_count": 851, "statues_count": 22739 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wildwood Crest, NJ", "id": "e4a79d0dc7c9b1c9", "name": "Wildwood Crest", "place_type": "city", "bounding_box": rectangle("-74.855015,38.957436 -74.821192,38.985403") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34009, "countyName": "Cape May", "cityID": 3481200, "cityName": "Wildwood Crest" } }
+{ "create_at": datetime("2016-01-02T10:00:34.000Z"), "id": 683347162445398016, "text": "Can you recommend anyone for this #Nursing #job? https://t.co/qimoF2E6wQ #SanAntonio, TX #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.429478,29.519092"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "SanAntonio", "Hiring" }}, "user": { "id": 259944587, "name": "BaptistHealthCareers", "screen_name": "BHSCareers", "lang": "en", "location": "San Antonio, TX", "create_at": date("2011-03-02"), "description": "Since 1903, Baptist Health System has been involved in the community, caring for the health and spiritual needs of people in San Antonio and beyond. Join us!", "followers_count": 258, "friends_count": 160, "statues_count": 2086 }, "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-02T10:00:34.000Z"), "id": 683347162483183617, "text": "I love you https://t.co/vcSVzPWsnB", "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, TX", "create_at": date("2012-01-10"), "description": "L.I.V.E ) CEO of @NoirBoyz | $ix Co| For Features and Bookings email BubbaTmusic@Gmail.Com |MGMT @NOIR_JVY", "followers_count": 1607, "friends_count": 897, "statues_count": 56665 }, "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-02T10:00:34.000Z"), "id": 683347162617540608, "text": "@ESPN_NHL oh don't act so surprised when you let fans vote.", "in_reply_to_status": 683345918939115520, "in_reply_to_user": 43165750, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 43165750 }}, "user": { "id": 814273556, "name": "Gabe LaNeve", "screen_name": "steelpengos42", "lang": "en", "location": "Clemson", "create_at": date("2012-09-09"), "description": "23. Penguins, Steelers and Pirates fan. Clemson University Class of 2015. Guitar and photographer as a hobby. Follow back if you're a Pittsburgh fan.", "followers_count": 121, "friends_count": 141, "statues_count": 8828 }, "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-01-02T10:00:34.000Z"), "id": 683347162634268672, "text": "Clarksville Fire Department had record year https://t.co/9x7LdWW40Y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 909171192, "name": "Regional News", "screen_name": "newsnet14895", "lang": "en", "location": "Wellsville, NY", "create_at": date("2012-10-27"), "description": "I write for Regional News Network. I also write, produce and anchor news for ten radio stations in NY/PA. Weather geek.", "followers_count": 408, "friends_count": 128, "statues_count": 8724 }, "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-02T10:00:34.000Z"), "id": 683347162852360196, "text": "@buddha2727 @katlivezey @tapatalk in the legal world they call it a disability could be any one of a dozen things", "in_reply_to_status": 683335109576122368, "in_reply_to_user": 1685187499, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1685187499, 426999530, 49560355 }}, "user": { "id": 1380313111, "name": "Joey Toledo", "screen_name": "joey_toledo", "lang": "en", "location": "Toledo Ohio ", "create_at": date("2013-04-25"), "description": "open-minded musician classically trained chef progressively minded activst unapologetically Black...", "followers_count": 1999, "friends_count": 1862, "statues_count": 19027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Findlay, OH", "id": "20a30e29568f7ddc", "name": "Findlay", "place_type": "city", "bounding_box": rectangle("-83.699081,40.993401 -83.568726,41.116626") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39063, "countyName": "Hancock", "cityID": 3927048, "cityName": "Findlay" } }
+{ "create_at": datetime("2016-01-02T10:00:34.000Z"), "id": 683347162931937280, "text": "I miss my rottie sm https://t.co/tmXfb2q7rf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3631906392, "name": "madeline ava chadie", "screen_name": "madelinechadie", "lang": "en", "location": "isaiah davilla", "create_at": date("2015-09-20"), "description": "☘⭐️✨☄⛈☃❄️", "followers_count": 341, "friends_count": 395, "statues_count": 2016 }, "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-02T10:00:34.000Z"), "id": 683347162986496000, "text": "Senior Financial Analyst Shared Mail Operations - Valassis: (#Livonia, MI) https://t.co/zZ1vzbcxAs #Finance #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.3527097,42.36837"), "retweet_count": 0, "lang": "da", "is_retweet": false, "hashtags": {{ "Livonia", "Finance", "Job", "Jobs", "Hiring" }}, "user": { "id": 106828847, "name": "Talent Acquisition", "screen_name": "ValassisCareers", "lang": "en", "location": "HQ - Livonia, MI", "create_at": date("2010-01-20"), "description": "Valassis is a leader in intelligent media delivery, providing proven & innovative media solutions to influence consumers wherever they plan, shop, buy & share.", "followers_count": 443, "friends_count": 80, "statues_count": 1593 }, "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-01-02T10:00:34.000Z"), "id": 683347163087147008, "text": "1. #CANvsFIN\n2. John Scott\n3. Finnish\n4. Strome\n5. Travis Konecny\n\n2016/1/2 11:57 CST #trndnl https://t.co/SHjd3tkINE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.309,56.9547"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CANvsFIN", "trndnl" }}, "user": { "id": 1270239397, "name": "Trendinalia Canada", "screen_name": "trendinaliaCA", "lang": "es", "location": "Canada", "create_at": date("2013-03-15"), "description": "Information about Twitter's Trending Topics in Canada", "followers_count": 1254, "friends_count": 7, "statues_count": 256052 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Division No. 23, Unorganized, Manitoba", "id": "654fd5617c7361d0", "name": "Division No. 23, Unorganized", "place_type": "city", "bounding_box": rectangle("-102.009382,55.158501 -88.989166,60.000283") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-02T10:00:34.000Z"), "id": 683347163095515136, "text": "Rt @marialrussell - Crafting in 2016 with #brentwoodowl @jlostamps. Here's a sneak peek. Happy… https://t.co/LNWQHRWMDe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.90809231,42.93791336"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "brentwoodowl" }}, "user_mentions": {{ 19963769, 35017283 }}, "user": { "id": 35017283, "name": "JessicaLynnOriginal", "screen_name": "JLOStamps", "lang": "en", "location": "Milwaukee, Wisconsin", "create_at": date("2009-04-24"), "description": "Custom #RubberStamps and Ready to Ship #ClearStamps for #CardMaking, #Scrapbooking, #Geocaching. In Oprah/Milwaukee Magazine in Dec we will be in a movie too!", "followers_count": 1196, "friends_count": 2023, "statues_count": 17555 }, "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-02T10:00:34.000Z"), "id": 683347163095658498, "text": "@Princespiffy ooooo ��lol", "in_reply_to_status": 683347015238062080, "in_reply_to_user": 1325135166, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1325135166 }}, "user": { "id": 92808043, "name": "Jane Dohh ?", "screen_name": "Nessa_Bxo", "lang": "en", "location": "null", "create_at": date("2009-11-26"), "description": "-Everytime Kanye Frowns, I Smile. . . ßlessed. UA 18' Dance . . . ✗O✗O", "followers_count": 936, "friends_count": 921, "statues_count": 4219 }, "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-02T10:00:34.000Z"), "id": 683347163334623232, "text": "Hey jacob how old are u #@jacobsartious https://t.co/IPPWS9yt1G", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2891248400, "name": "Kalyn Hernandez", "screen_name": "kalynh57", "lang": "en", "location": "null", "create_at": date("2014-11-05"), "description": "null", "followers_count": 4, "friends_count": 34, "statues_count": 84 }, "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-02T10:00:34.000Z"), "id": 683347163728998400, "text": "@niftynei the logic adds up", "in_reply_to_status": 683345640345210880, "in_reply_to_user": 22983338, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 22983338 }}, "user": { "id": 14254939, "name": "lagrunge multiplier", "screen_name": "jc4p", "lang": "en", "location": "Brooklyn", "create_at": date("2008-03-29"), "description": "Seller of propane and propane accessories at @StackOverflow (also, mobile lead). Looks best when viewed in 640x480.", "followers_count": 640, "friends_count": 259, "statues_count": 15779 }, "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-02T10:00:34.000Z"), "id": 683347164378968064, "text": "Yup! Gotta get moving! #tonyrobbins #success #addictedtosuccess #hustle #hustleallday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "tonyrobbins", "success", "addictedtosuccess", "hustle", "hustleallday" }}, "user": { "id": 425329730, "name": "Domenic Costa", "screen_name": "CostaDomenic", "lang": "en", "location": "San Pedro, CA", "create_at": date("2011-11-30"), "description": "Hustling for $s w/metal recycling, storage auctions, yard sales, Craigslist sales, hands-on landlord, YouTube vids, & teaching kids!", "followers_count": 1380, "friends_count": 1497, "statues_count": 17338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-02T10:00:34.000Z"), "id": 683347164500594688, "text": "Join the Accountable Healthcare Staffing team! See our latest #Healthcare #job opening here: https://t.co/X48klBnuAl #Stockton, CA #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.2907796,37.9577016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "Stockton", "Hiring" }}, "user": { "id": 22635038, "name": "Sacramento Health", "screen_name": "tmj_sac_health", "lang": "en", "location": "Sacramento, CA", "create_at": date("2009-03-03"), "description": "Follow this account for geo-targeted Healthcare job tweets in Sacramento, CA. Need help? Tweet us at @CareerArc!", "followers_count": 610, "friends_count": 304, "statues_count": 778 }, "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-02T10:00:34.000Z"), "id": 683347164542681089, "text": "RT �� https://t.co/46j9GopE3q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 447686784, "name": "bearald ⛽️", "screen_name": "And_E_93", "lang": "en", "location": "The United States of America", "create_at": date("2011-12-26"), "description": "cubs fan | beer league grinder | somewhere between Jack Daniels and Jesus | LB7 Duramax |LCC 5 |", "followers_count": 2125, "friends_count": 2320, "statues_count": 60311 }, "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-02T10:00:34.000Z"), "id": 683347164701917184, "text": "6. Quenneville\n7. Jake Virtanen\n8. Reed Deming\n9. #COYG\n10. #FilmFreeway\n\n2016/1/2 11:57 CST #trndnl https://t.co/SHjd3tkINE", "in_reply_to_status": 683347163087147008, "in_reply_to_user": 1270239397, "favorite_count": 0, "coordinate": point("-98.309,56.9547"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "COYG", "FilmFreeway", "trndnl" }}, "user": { "id": 1270239397, "name": "Trendinalia Canada", "screen_name": "trendinaliaCA", "lang": "es", "location": "Canada", "create_at": date("2013-03-15"), "description": "Information about Twitter's Trending Topics in Canada", "followers_count": 1254, "friends_count": 7, "statues_count": 256053 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Division No. 23, Unorganized, Manitoba", "id": "654fd5617c7361d0", "name": "Division No. 23, Unorganized", "place_type": "city", "bounding_box": rectangle("-102.009382,55.158501 -88.989166,60.000283") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-02T10:00:34.000Z"), "id": 683347164802740225, "text": "Super Sasadango Machine is my spirit animal.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 19901033, "name": "Fratboy Franny", "screen_name": "TheFrayMovement", "lang": "en", "location": "null", "create_at": date("2009-02-02"), "description": "Currently lacking the ability to handle life's banter. 2015 Sheety Award Winner for Best Twitter.", "followers_count": 1393, "friends_count": 190, "statues_count": 142662 }, "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-02T10:00:34.000Z"), "id": 683347164827938820, "text": "Why would you want to live a crack jack box aka tiny house? #confused", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "confused" }}, "user": { "id": 288470628, "name": "#RealRock'nRolla", "screen_name": "KrazyK12200", "lang": "en", "location": "Wisconsin", "create_at": date("2011-04-26"), "description": "Welcome to the world of random. I am a marketing major who loves my metal, pro wrestling, MMA, action sports, and mocking reality shows.", "followers_count": 275, "friends_count": 765, "statues_count": 15056 }, "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-02T10:00:34.000Z"), "id": 683347164961964033, "text": "@OaheCAPS PeeWee Cs lead Sioux Falls 13-4 after 2nd. Goals by Lyons 3, Starr 2, Hebb & Molseed. Assists by Hebb, Swartz, Starr, Hebb #GoCaps", "in_reply_to_status": -1, "in_reply_to_user": 1445015454, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GoCaps" }}, "user_mentions": {{ 1445015454 }}, "user": { "id": 125995754, "name": "Shawn Lyons", "screen_name": "lyonsden28", "lang": "en", "location": "null", "create_at": date("2010-03-24"), "description": "null", "followers_count": 260, "friends_count": 350, "statues_count": 578 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Pierre, SD", "id": "01e915a349d1ca8e", "name": "Fort Pierre", "place_type": "city", "bounding_box": rectangle("-100.40058,44.344364 -100.363738,44.385919") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46117, "countyName": "Stanley", "cityID": 4622260, "cityName": "Fort Pierre" } }
+{ "create_at": datetime("2016-01-02T10:00:34.000Z"), "id": 683347165142454274, "text": "����Im 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": 29822803, "name": "Darkskin Swaggin", "screen_name": "MudahPacino_973", "lang": "en", "location": "Paterson NJ", "create_at": date("2009-04-08"), "description": "Fuck You Too.", "followers_count": 1966, "friends_count": 739, "statues_count": 127914 }, "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-02T10:00:34.000Z"), "id": 683347165213782017, "text": "a rolling backpack just took out my knee. penn station is shit.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17913653, "name": "sabby", "screen_name": "gabbysully", "lang": "en", "location": "the garden state ", "create_at": date("2008-12-05"), "description": "☕️", "followers_count": 581, "friends_count": 270, "statues_count": 53493 }, "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-02T10:00:34.000Z"), "id": 683347165259952128, "text": "Green grass and palm trees! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1852473134, "name": "Kate Anthes", "screen_name": "KateAnthes", "lang": "en", "location": "The Mitten State", "create_at": date("2013-09-10"), "description": "First of her name, mother of cats, dealer (licensed) of drugs, lover of DMB/travel/wine/photography! #dmb #LoVE", "followers_count": 190, "friends_count": 219, "statues_count": 2675 }, "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-01-02T10:00:34.000Z"), "id": 683347165414985728, "text": "@yukisanwhoo @Muffinman_7 lmao ��������", "in_reply_to_status": 683346525741694976, "in_reply_to_user": 1002924200, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 1002924200, 2500007554 }}, "user": { "id": 2346187092, "name": "AlwysPolitclyIncrrct", "screen_name": "GonzoG_23", "lang": "en", "location": "Space Time Continuum ", "create_at": date("2014-02-15"), "description": "UCLA '19 Output is relative to input. You get what you give. IG: gonzo_wolfgang", "followers_count": 733, "friends_count": 701, "statues_count": 42756 }, "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-02T10:00:34.000Z"), "id": 683347165536714752, "text": "@lorijeansmith13 @KimmiOsborne @marylener9 morning lovely Lori https://t.co/c4pWlafSVD", "in_reply_to_status": 683335471225679872, "in_reply_to_user": 398881513, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 398881513, 2280736746, 721241231 }}, "user": { "id": 1271007283, "name": "bob lopez", "screen_name": "rplopez71511", "lang": "en", "location": "Chicago Ill.", "create_at": date("2013-03-15"), "description": "I enjoy viewing the works of the greatest artist of all the Lord our God .I am so lucky to have such great friends", "followers_count": 2801, "friends_count": 3000, "statues_count": 73297 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lombard, IL", "id": "9f3d755fe1a081a4", "name": "Lombard", "place_type": "city", "bounding_box": rectangle("-88.051015,41.831901 -87.9743,41.915092") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1744407, "cityName": "Lombard" } }
+{ "create_at": datetime("2016-01-02T10:00:34.000Z"), "id": 683347165545025537, "text": "Can You BELIEVE?It is handmade with tiny pieces of rice leaves https://t.co/8xvXhxzuka No color added to rice straw https://t.co/NZf1COZYsc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 84217658, "name": "Rajan Koshy", "screen_name": "rajankoshy", "lang": "en", "location": "Galveston,Texas", "create_at": date("2009-10-21"), "description": "Rice Straw artist: Create beautiful art out of dried leaf of rice plant and cottonwood tree. Texas tallship ELISSA, Birds, animals, people etc with leaf.", "followers_count": 144694, "friends_count": 132978, "statues_count": 60104 }, "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-01-02T10:00:34.000Z"), "id": 683347165666643968, "text": "Just posted a photo @ Milliken, Colorado https://t.co/83r7MDIMhj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.86,40.3311"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2751041485, "name": "Shannon M Johnson", "screen_name": "ShanMarie19", "lang": "en", "location": "Milliken, Colorado", "create_at": date("2014-08-20"), "description": "birthdate: 07/08/1976", "followers_count": 146, "friends_count": 460, "statues_count": 11296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milliken, CO", "id": "01600c1453c06999", "name": "Milliken", "place_type": "city", "bounding_box": rectangle("-104.886752,40.315188 -104.829683,40.335481") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8123, "countyName": "Weld", "cityID": 850480, "cityName": "Milliken" } }
+{ "create_at": datetime("2016-01-02T10:00:34.000Z"), "id": 683347165675036672, "text": "current weather in Wausau: fair, 21°F\n69% humidity, wind 8mph, visibility 10mi, pressure 29.89in falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.63031,44.95953"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 230061980, "name": "Wausau Weather", "screen_name": "WausauWI", "lang": "en", "location": "Wausau, WI", "create_at": date("2010-12-23"), "description": "Weather updates, forecast, warnings and information for Wausau, WI. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 87, "friends_count": 12, "statues_count": 19837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wausau, WI", "id": "35ecddb970148417", "name": "Wausau", "place_type": "city", "bounding_box": rectangle("-89.746623,44.919084 -89.590106,45.006833") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55073, "countyName": "Marathon", "cityID": 5584475, "cityName": "Wausau" } }
+{ "create_at": datetime("2016-01-02T10:00:34.000Z"), "id": 683347165733847040, "text": "How did I get this sick ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 372015608, "name": "Grriimy Chyy .", "screen_name": "Chyy_GotShmoney", "lang": "en", "location": "New York, NY", "create_at": date("2011-09-11"), "description": "We are not the same", "followers_count": 1145, "friends_count": 817, "statues_count": 17197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carbondale, PA", "id": "8c2b71e1a5d02706", "name": "Carbondale", "place_type": "city", "bounding_box": rectangle("-75.532224,41.556468 -75.480355,41.587352") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42069, "countyName": "Lackawanna", "cityID": 4211232, "cityName": "Carbondale" } }
+{ "create_at": datetime("2016-01-02T10:00:34.000Z"), "id": 683347165817745409, "text": "Berry and Britt have combined for five of #UNC's seven turnovers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "UNC" }}, "user": { "id": 19192748, "name": "Star-News ACC", "screen_name": "starnewsacc", "lang": "en", "location": "null", "create_at": date("2009-01-19"), "description": "null", "followers_count": 2959, "friends_count": 637, "statues_count": 56956 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chapel Hill, NC", "id": "23ebe811ea93fd41", "name": "Chapel Hill", "place_type": "city", "bounding_box": rectangle("-79.083153,35.869376 -78.968887,35.969507") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37135, "countyName": "Orange", "cityID": 3711800, "cityName": "Chapel Hill" } }
+{ "create_at": datetime("2016-01-02T10:00:35.000Z"), "id": 683347166098755584, "text": "#roseparade #sikh @ Rose Bowl Parade Floats https://t.co/CRVEOZLG0P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.0847816,34.16144032"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "roseparade", "sikh" }}, "user": { "id": 18791913, "name": "Robert_Valente", "screen_name": "Robert_Valente", "lang": "en", "location": "Northern Virginia", "create_at": date("2009-01-08"), "description": "(Home) Brewer, Patriot. Redskins - Nationals - Capitals", "followers_count": 461, "friends_count": 1875, "statues_count": 14558 }, "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-02T10:00:35.000Z"), "id": 683347166132187137, "text": "real shit https://t.co/wGtUh4FIq4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2307154844, "name": "Awesome", "screen_name": "iamnickwest_", "lang": "en", "location": "Louisiana ", "create_at": date("2014-01-23"), "description": "Prairie view A&M student athlete #8⃣8⃣ ✊ Rest easy el' chopo", "followers_count": 1123, "friends_count": 945, "statues_count": 5907 }, "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-02T10:00:35.000Z"), "id": 683347166199463936, "text": "Smh https://t.co/Xw6W481LQ9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1592282118, "name": "Ellie Taylor", "screen_name": "ellietaylor99", "lang": "en", "location": "DHS '17", "create_at": date("2013-07-13"), "description": "Chris Keller's work here is done", "followers_count": 558, "friends_count": 492, "statues_count": 1221 }, "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-02T10:00:35.000Z"), "id": 683347166266535936, "text": "@mahaliahtd I do ur usually not productive and we all know so shush it :'(", "in_reply_to_status": 683346863139979264, "in_reply_to_user": 449161618, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 449161618 }}, "user": { "id": 153912966, "name": "marie", "screen_name": "Toromarie23", "lang": "en", "location": "|music city| ", "create_at": date("2010-06-09"), "description": "fuck off .", "followers_count": 542, "friends_count": 553, "statues_count": 39121 }, "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-01-02T10:00:35.000Z"), "id": 683347166551773185, "text": "New year new me I'm done messing with people just worrying about the ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2873327961, "name": "Amadasimpson", "screen_name": "mandasimpson_4", "lang": "en", "location": "null", "create_at": date("2014-11-11"), "description": "null", "followers_count": 12, "friends_count": 26, "statues_count": 119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Essex, MD", "id": "8a8002ffef902f59", "name": "Essex", "place_type": "city", "bounding_box": rectangle("-76.489916,39.270065 -76.381559,39.330308") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2426600, "cityName": "Essex" } }
+{ "create_at": datetime("2016-01-02T10:00:35.000Z"), "id": 683347166895669248, "text": "Sits phone on couch\nLeaves for 5 minutes\nChecks it https://t.co/8F7rzBAtDz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 319415936, "name": "Derrick R L Garrett", "screen_name": "DerrickGarrettx", "lang": "en", "location": "Orlando, FL", "create_at": date("2011-06-17"), "description": "DPHS '18 | Wizard of Nas | Best singing 2013-2016 |''Be perfect.'' -Ryan Garr Louis Williams, MMXV", "followers_count": 90, "friends_count": 69, "statues_count": 1724 }, "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-02T10:00:35.000Z"), "id": 683347166903939072, "text": "Cancer Treatment Centers of America: Clinical Data Analyst (Philadelphia, PA) (#Philadelphia, PA) https://t.co/LH1BhiR6fC #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.0990766,40.0189574"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Philadelphia", "Job", "Jobs" }}, "user": { "id": 38035912, "name": "TMJ-PHL Jobs", "screen_name": "tmj_phl_jobs", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2009-05-05"), "description": "Follow this account for geo-targeted Other job tweets in Philadelphia, PA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 653, "friends_count": 539, "statues_count": 329 }, "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-02T10:00:35.000Z"), "id": 683347166904074241, "text": "Keep it tight in 2016. @PROCompression #keepittight #goals #runchat https://t.co/JBLdaJ5A6S", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "keepittight", "goals", "runchat" }}, "user_mentions": {{ 267851074 }}, "user": { "id": 2576924275, "name": "HappyRunningCo", "screen_name": "HappyRunningCo", "lang": "en", "location": "Philadelphia ", "create_at": date("2014-06-19"), "description": "All you need is shoes and Happy Running. Pro Compression Socks #keepittight #cityfitgirlsambassador #sweatpinkambassador\n#girlsgonesportyambassador", "followers_count": 435, "friends_count": 429, "statues_count": 2523 }, "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-02T10:00:35.000Z"), "id": 683347167184990208, "text": "@LindaSuhler Anyone check his papers recently? Can we deport him back to where he'll need to fear for his life daily? One can wish.", "in_reply_to_status": 683346893003296768, "in_reply_to_user": 347627434, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 347627434 }}, "user": { "id": 9953462, "name": "Michael Murdock", "screen_name": "docmurdock", "lang": "en", "location": "Scottsdale, AZ", "create_at": date("2007-11-04"), "description": "US NAVY VETERAN, PATRIOT, Pixar Mac Sys Eng in Toy Story 1995., Photog - NO DM'S SELLING STUFF PLEASE\nIG: murdockme", "followers_count": 15293, "friends_count": 13278, "statues_count": 66470 }, "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-02T10:00:35.000Z"), "id": 683347167327600640, "text": "Miss my ombré hair :-(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 583999350, "name": "rubes", "screen_name": "Ruubygomez", "lang": "en", "location": "null", "create_at": date("2012-05-18"), "description": "Okay?", "followers_count": 555, "friends_count": 358, "statues_count": 34704 }, "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-02T10:00:35.000Z"), "id": 683347167579340800, "text": "my mood determines on how you 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": 1007828214, "name": "Sam burden", "screen_name": "Sammietkd", "lang": "en", "location": "buffalo", "create_at": date("2012-12-12"), "description": "angelo paul eckert ♡", "followers_count": 710, "friends_count": 196, "statues_count": 16039 }, "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-01-02T10:00:35.000Z"), "id": 683347167952658432, "text": "@LobotC2DFW @tim031845 @SinceHeWillWin I think this could be the last year in the AAC", "in_reply_to_status": 683346497329442816, "in_reply_to_user": 23125289, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23125289, 971316044, 2190506484 }}, "user": { "id": 346954251, "name": "Bearcat Ed", "screen_name": "BearcatEd_30", "lang": "en", "location": "Cincinnati, OH", "create_at": date("2011-08-01"), "description": "Just living the Dream! Proud Cincinnati and NKU Alum #Reds #Bearcats", "followers_count": 627, "friends_count": 1990, "statues_count": 33606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indian Harbour Beach, FL", "id": "ceb66dcdfed0c950", "name": "Indian Harbour Beach", "place_type": "city", "bounding_box": rectangle("-80.612737,28.114562 -80.574803,28.172355") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12009, "countyName": "Brevard", "cityID": 1233450, "cityName": "Indian Harbour Beach" } }
+{ "create_at": datetime("2016-01-02T10:00:35.000Z"), "id": 683347168141393920, "text": "�� https://t.co/sYsIQ08cBf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2666005156, "name": "BandzupBlanco", "screen_name": "juanferxxv", "lang": "en", "location": "null", "create_at": date("2014-07-02"), "description": "954//813 Barranquilla&Cali Colombia//RipRodri// RipJulZ//RipAndresakamiko//RIPOgGustavo//FreeDaGuys//954We show u how to stunt.", "followers_count": 659, "friends_count": 510, "statues_count": 18021 }, "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("2016-01-02T10:00:35.000Z"), "id": 683347168195907585, "text": "Having a blast in city...#loveDetroit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.04636,42.332315"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "loveDetroit" }}, "user": { "id": 471301948, "name": "HisHwy", "screen_name": "HisHwy", "lang": "en", "location": "MI", "create_at": date("2012-01-22"), "description": "His Hwy is a great reminder of your faith and a lifestyle you're trying to lead. Please visit our website to order a wristband/shirt.", "followers_count": 1582, "friends_count": 1220, "statues_count": 18938 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hard Rock Cafe Detroit", "id": "07d9d21337487003", "name": "Hard Rock Cafe Detroit", "place_type": "poi", "bounding_box": rectangle("-83.0463601,42.3323149 -83.04636,42.332315") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-01-02T10:00:35.000Z"), "id": 683347168216760320, "text": "Interested in a #Healthcare #job near #Wellman, IA? This could be a great fit: https://t.co/wYnppg1syA #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-91.8382273,41.4641806"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "Wellman", "Hiring", "CareerArc" }}, "user": { "id": 57694228, "name": "TMJ-IA Health Jobs", "screen_name": "tmj_ia_health", "lang": "en", "location": "Iowa Non-Metro", "create_at": date("2009-07-17"), "description": "Follow this account for geo-targeted Healthcare job tweets in Iowa Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 345, "friends_count": 295, "statues_count": 429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Iowa, USA", "id": "3cd4c18d3615bbc9", "name": "Iowa", "place_type": "admin", "bounding_box": rectangle("-96.639706,40.375437 -90.140061,43.50102") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19183, "countyName": "Washington", "cityID": 1983280, "cityName": "Wellman" } }
+{ "create_at": datetime("2016-01-02T10:00:35.000Z"), "id": 683347168254685185, "text": "@realDonaldTrump https://t.co/eFBl3rZrTM", "in_reply_to_status": -1, "in_reply_to_user": 25073877, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 25073877 }}, "user": { "id": 2281435852, "name": "johny glo", "screen_name": "Paul_cghornet11", "lang": "en", "location": "Coal Grove, OH", "create_at": date("2014-01-14"), "description": "null", "followers_count": 259, "friends_count": 287, "statues_count": 3609 }, "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-01-02T10:00:35.000Z"), "id": 683347168342589441, "text": "Citizens Financial Group #Banking #Job: Teller (#Albany, NY) https://t.co/5vFIt1pOSU #cfgjobs #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9712488,42.6284995"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Banking", "Job", "Albany", "cfgjobs", "Jobs", "Hiring" }}, "user": { "id": 379141823, "name": "CitizensBankITJobs", "screen_name": "RBSCFG_Tech", "lang": "en", "location": "Rhode Island", "create_at": date("2011-09-24"), "description": "We are hiring! Go to http://CFGCareers.com for more information about working at Citizens Bank.", "followers_count": 111, "friends_count": 50, "statues_count": 273 }, "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": 36001, "countyName": "Albany" } }
+{ "create_at": datetime("2016-01-02T10:00:35.000Z"), "id": 683347168577490946, "text": "I just want to DIE or at least not have any feelings anymore lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1905791832, "name": "jaz", "screen_name": "jasmynfshaw", "lang": "en", "location": "null", "create_at": date("2013-09-25"), "description": "null", "followers_count": 546, "friends_count": 696, "statues_count": 1257 }, "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-02T10:00:35.000Z"), "id": 683347168682377216, "text": "Specialist, Digital Creative Production - Boston Proper: (#BocaRaton, FL) https://t.co/VZhmqbY1e3 #Retail #holidayjobs #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.0830984,26.3586885"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BocaRaton", "Retail", "holidayjobs", "Job", "Jobs", "Hiring" }}, "user": { "id": 2798260664, "name": "Boston Proper Jobs", "screen_name": "BostonProperJob", "lang": "en", "location": "null", "create_at": date("2014-09-08"), "description": "Boston Proper provides fashions for women who are fearlessly feminine, enviably chic and who possess the poise and confidence to “wear it like no one else“.", "followers_count": 30, "friends_count": 0, "statues_count": 461 }, "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-02T10:00:35.000Z"), "id": 683347168699133952, "text": "@GroverNorquist Shit, when Obama has finished his 2nd term, you'll still be claiming he wants to take yer guns.\n✊✊��✊✊", "in_reply_to_status": 683282313078259712, "in_reply_to_user": 16045956, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16045956 }}, "user": { "id": 279683174, "name": "Classless Skip", "screen_name": "MenaceSocietyUT", "lang": "en", "location": "SL,UT", "create_at": date("2011-04-09"), "description": "Classless Ute / Utah '95 / BS Accounting / I understand sample size - I passed Stats 301 / Not a Proven Christian - Can I Stay? / Fell into the Tolerance Trap", "followers_count": 410, "friends_count": 494, "statues_count": 30698 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Draper, UT", "id": "4c3c7782398bd438", "name": "Draper", "place_type": "city", "bounding_box": rectangle("-111.921988,40.45236 -111.823749,40.544422") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4920120, "cityName": "Draper" } }
+{ "create_at": datetime("2016-01-02T10:00:35.000Z"), "id": 683347168736993282, "text": "@001Mightymike you call it. We can bet $, gift card, date you tell me. I know the Steelers R gonna whoop ass so be kind to yourself haha", "in_reply_to_status": 683346631262011392, "in_reply_to_user": 3232430712, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3232430712 }}, "user": { "id": 29744790, "name": "Sexy Shay", "screen_name": "Sexyshay69", "lang": "en", "location": "Toledo,Oh", "create_at": date("2009-04-08"), "description": "I'm a Adult Entertainer based out of Toledo Oh. I ♡ 2 travel . I call it how I see it. Please don't exspect nothing 4 free and I wont. ADULTS ONLY/21 +", "followers_count": 1486, "friends_count": 698, "statues_count": 2264 }, "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-02T10:00:35.000Z"), "id": 683347168963371009, "text": "Can you recommend anyone for this #Underwriting #job? https://t.co/I4jTnJbXrZ #Scottsdale, Arizona #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.9260519,33.4941704"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Underwriting", "job", "Scottsdale", "Hiring" }}, "user": { "id": 113408527, "name": "Nationwide", "screen_name": "Nationwide_jobs", "lang": "en", "location": "Columbus, Ohio", "create_at": date("2010-02-11"), "description": "Official Careers Page. We believe our talent is the key to success. This is how we continue to differentiate ourselves from the competition. #GoNationwide", "followers_count": 831, "friends_count": 496, "statues_count": 1920 }, "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-02T10:00:35.000Z"), "id": 683347168997015552, "text": "bitches really be full blown rats wtf ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 605117281, "name": "alaya", "screen_name": "alayadivine", "lang": "en", "location": "null", "create_at": date("2012-06-10"), "description": "null", "followers_count": 2061, "friends_count": 859, "statues_count": 51458 }, "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-02T10:00:35.000Z"), "id": 683347169030483968, "text": "Can you recommend anyone for this #EmergencyMedicine #job? https://t.co/nFMR578b8v #Baltimore, MD #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.622542,39.28658"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "EmergencyMedicine", "job", "Baltimore", "Hiring" }}, "user": { "id": 3330042873, "name": "UMMS Careers", "screen_name": "JobsAtUMMS", "lang": "en", "location": "null", "create_at": date("2015-06-16"), "description": "Discover the clinical variety and world-class resources that make University of Maryland Medical System the place for providers who want a more dynamic career", "followers_count": 1, "friends_count": 0, "statues_count": 133 }, "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-02T10:00:35.000Z"), "id": 683347169231781888, "text": "Royal Tire: Auto Tire/Lube Technician - FT $300 BONUS! (#NewUlm, MN) https://t.co/9jefzhcFBe #Automotive #mechanic #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.461114,44.312225"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "NewUlm", "Automotive", "mechanic", "Job", "Jobs", "Hiring" }}, "user": { "id": 2963298858, "name": "Royal Tire Jobs", "screen_name": "RoyalTireJobs", "lang": "en", "location": "St. Cloud, MN", "create_at": date("2015-01-07"), "description": "Since 1948, Royal Tire has been providing customers with quality transportation care, products, and services.", "followers_count": 69, "friends_count": 279, "statues_count": 31 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Ulm, MN", "id": "40aef2dae5f2e9cd", "name": "New Ulm", "place_type": "city", "bounding_box": rectangle("-94.507217,44.280315 -94.414288,44.344475") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27015, "countyName": "Brown", "cityID": 2746042, "cityName": "New Ulm" } }
+{ "create_at": datetime("2016-01-02T10:00:35.000Z"), "id": 683347169273749505, "text": "all I ever needed was the squad so that's what'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": 443738211, "name": "sydglass❥", "screen_name": "sydney672", "lang": "en", "location": "cedar rapids", "create_at": date("2011-12-22"), "description": "fear is stupid, so are regrets ❊ roll tide roll", "followers_count": 1149, "friends_count": 801, "statues_count": 20564 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walnut Hill, TN", "id": "6e8074185d300420", "name": "Walnut Hill", "place_type": "city", "bounding_box": rectangle("-82.277485,36.558253 -82.231833,36.590232") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47163, "countyName": "Sullivan", "cityID": 4777900, "cityName": "Walnut Hill" } }
+{ "create_at": datetime("2016-01-02T10:00:35.000Z"), "id": 683347169466712064, "text": "@cchauvet it's really cool. I'm watching a 4-D dinosaur movie later.", "in_reply_to_status": 683346799826882560, "in_reply_to_user": 15416277, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15416277 }}, "user": { "id": 2327642106, "name": "Nick Packard", "screen_name": "Meowzalot_", "lang": "en", "location": "Massachusetts, USA", "create_at": date("2014-02-04"), "description": "Aspiring MTG pro, wrestling nerd, and I take pictures of my cats. Pop punk is life.", "followers_count": 327, "friends_count": 837, "statues_count": 22502 }, "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-01-02T10:00:35.000Z"), "id": 683347169550577664, "text": "Ladies first! @WestlakeWBB takes on Lake Travis today @ 2pm at Westlake. #GettotheGym #GoChaps", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GettotheGym", "GoChaps" }}, "user_mentions": {{ 1897643071 }}, "user": { "id": 50703494, "name": "Joe Taylor", "screen_name": "Westlake_Nation", "lang": "en", "location": "Austin, TX", "create_at": date("2009-06-25"), "description": "null", "followers_count": 1212, "friends_count": 101, "statues_count": 4511 }, "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-02T10:00:35.000Z"), "id": 683347169575833600, "text": "@JeffGohogs Reagan taught me \"Trust, but verify\" ☺ Don't know what happened to the rest of the GOP ��", "in_reply_to_status": 683345872805990400, "in_reply_to_user": 1895983010, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1895983010 }}, "user": { "id": 3193073724, "name": "Dennis Reichel", "screen_name": "dennis_reichel", "lang": "en", "location": "Riviera Beach, FL", "create_at": date("2015-05-12"), "description": "#FF #MountainBiker #UrbanGardener\n#Sailor #FOSSDeveloper\n#Writer #Blogger #MetaInfluencer\n#PracticingAbsurdist #Sanders2016", "followers_count": 2704, "friends_count": 3186, "statues_count": 3833 }, "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-01-02T10:00:35.000Z"), "id": 683347169764610053, "text": "@Ray_Coleman10 https://t.co/c1s7CUrpac", "in_reply_to_status": -1, "in_reply_to_user": 335549374, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 335549374 }}, "user": { "id": 3384137229, "name": "AJ's Father", "screen_name": "imafather_1st", "lang": "en", "location": "null", "create_at": date("2015-07-19"), "description": "I'm a father first!\r\nFuture Business Owner\r\nDiesel Mechanic", "followers_count": 306, "friends_count": 208, "statues_count": 4912 }, "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-01-02T10:00:35.000Z"), "id": 683347169911402496, "text": "#Nursing #Job alert: CLINICAL NURSE; Registered Nurse | Yale-New Haven Hospital | #NewHaven, CT https://t.co/bbphAz6CoV #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.9268626,41.3052226"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "Job", "NewHaven", "Jobs", "Hiring" }}, "user": { "id": 25288024, "name": "TMJ-CTM Nursing Jobs", "screen_name": "tmj_ctm_nursing", "lang": "en", "location": "Middletown, CT", "create_at": date("2009-03-19"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Middletown, CT. Need help? Tweet us at @CareerArc!", "followers_count": 375, "friends_count": 302, "statues_count": 149 }, "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-01-02T10:00:35.000Z"), "id": 683347169995169792, "text": "There's a praise on the inside that I can't keep 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": 2244677288, "name": "3rdWorldBaby❤️❗️", "screen_name": "_guttaaaa", "lang": "en", "location": "LOOKING FOR LOVE", "create_at": date("2013-12-13"), "description": "D'Shayla Janiell Gutter ❗️❤️ fly high Cobe , Mone & WoeWoe ❤️", "followers_count": 1176, "friends_count": 968, "statues_count": 54845 }, "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-02T10:00:36.000Z"), "id": 683347170443964418, "text": "#Reston, VA #BusinessMgmt #Job: Senior Principal Consultant at Oracle https://t.co/MZ9uDZVJKe #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.3570028,38.9586307"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Reston", "BusinessMgmt", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22871752, "name": "TMJ-VA Mgmt. Jobs", "screen_name": "tmj_va_mgmt", "lang": "en", "location": "Virginia", "create_at": date("2009-03-04"), "description": "Follow this account for geo-targeted Business/Mgmt. job tweets in Virginia Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 346, "friends_count": 246, "statues_count": 494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reston, VA", "id": "ec212eb1116b92e2", "name": "Reston", "place_type": "city", "bounding_box": rectangle("-77.393246,38.90861 -77.304868,39.002993") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5166672, "cityName": "Reston" } }
+{ "create_at": datetime("2016-01-02T10:00:36.000Z"), "id": 683347170565730305, "text": "all I ever ask for is patience .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35295042, "name": "#GreeceHots", "screen_name": "OdieMaGee", "lang": "en", "location": "GreeceSide ✌️", "create_at": date("2009-04-25"), "description": "artist-comedian-burgerflipper-landshark-motivational speaker-chiropractor-loud whistler-weird nigga with a cool side #WeirdGang✌️ IG|Snapchat: @odiemagee", "followers_count": 742, "friends_count": 513, "statues_count": 20445 }, "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-02T10:00:36.000Z"), "id": 683347170628513793, "text": "@AchaFlocka listen here Nigerian jollof is OG jollof", "in_reply_to_status": 683344926189342721, "in_reply_to_user": 174372330, "favorite_count": 0, "retweet_count": 0, "lang": "nl", "is_retweet": false, "user_mentions": {{ 174372330 }}, "user": { "id": 472538042, "name": "Riheoma Fenty", "screen_name": "IAMLEBADDEST", "lang": "en", "location": "Hurst, TX -- Washington, DC.", "create_at": date("2012-01-23"), "description": "I love Rihanna, future world renowned journalist. I AM the media. Howard University @the2400", "followers_count": 723, "friends_count": 567, "statues_count": 42865 }, "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-01-02T10:00:36.000Z"), "id": 683347170674642944, "text": "#SocialSciences #Job in #Denver, CO: Vocational... at Mental Health Center of Denver https://t.co/LpeAqj6KHN #Jobs https://t.co/XpYdSDB863", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.9355793,39.7350359"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SocialSciences", "Job", "Denver", "Jobs" }}, "user": { "id": 4036025892, "name": "MHCD Careers", "screen_name": "MHCD_Careers", "lang": "en", "location": "Denver, CO", "create_at": date("2015-10-27"), "description": "We believe that people can and do recover from mental illness and should have that chance. We hire people dedicated to working toward that goal. Join us!", "followers_count": 10, "friends_count": 2, "statues_count": 48 }, "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-02T10:00:36.000Z"), "id": 683347170779529216, "text": "Mood. Amy Winehouse", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 336115445, "name": "Velvet.", "screen_name": "Lord_Lana", "lang": "en", "location": "DTX| PV", "create_at": date("2011-07-15"), "description": "you don't even know me, you think you know me. #HighlifePromos", "followers_count": 969, "friends_count": 333, "statues_count": 36166 }, "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-02T10:00:36.000Z"), "id": 683347170825641984, "text": "Today’s daily concentrate special at 2nd Step Dispensary in Salem, Oregon is Dab Society! All Dab Society https://t.co/da1afUZ6Pt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2750476154, "name": "Nature Quest LLC", "screen_name": "Nature_Questllc", "lang": "en", "location": "Salem, Oregon", "create_at": date("2014-08-20"), "description": "Nature Quest is a medical marijuana clinic and dispensary located in Salem, Oregon.", "followers_count": 219, "friends_count": 58, "statues_count": 2652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Four Corners, OR", "id": "8271f56611245451", "name": "Four Corners", "place_type": "city", "bounding_box": rectangle("-122.990493,44.909874 -122.955303,44.962335") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41047, "countyName": "Marion", "cityID": 4126750, "cityName": "Four Corners" } }
+{ "create_at": datetime("2016-01-02T10:00:36.000Z"), "id": 683347171106762752, "text": "Started off 2016 adventuring and loving life �� @ Mount Yonah https://t.co/Z9NOaND8g8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.72527455,34.63728506"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 88566547, "name": "taylor vance", "screen_name": "taylorrrvance", "lang": "en", "location": "null", "create_at": date("2009-11-08"), "description": "null", "followers_count": 163, "friends_count": 115, "statues_count": 2858 }, "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": 13311, "countyName": "White", "cityID": 1384832, "cityName": "Yonah" } }
+{ "create_at": datetime("2016-01-02T10:00:36.000Z"), "id": 683347171156963328, "text": "See our latest #LongBeach, CA #job and click to apply: Facility Services Assistant (Work Order... - https://t.co/CJTAvimIfd #FacilitiesMgmt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.1580556,33.8041667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LongBeach", "job", "FacilitiesMgmt" }}, "user": { "id": 172568244, "name": "CA Facility Mgmt.", "screen_name": "tmj_CA_facmgmt", "lang": "en", "location": "California", "create_at": date("2010-07-29"), "description": "Follow this account for geo-targeted Facilities Management job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 204, "friends_count": 135, "statues_count": 325 }, "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-02T10:00:36.000Z"), "id": 683347171308113920, "text": "we jus tried \"flashbang hot sauce\" bruh... niggas almost died", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 67112338, "name": "RidiculousNdisclosed", "screen_name": "BizzareMarley", "lang": "en", "location": "Htrae", "create_at": date("2009-08-19"), "description": "ഞാന് ചെയ്തുകൊണ്ടിരിക്കുന്നത് അറിയുന്നു . . laugh hard smile loud", "followers_count": 284, "friends_count": 159, "statues_count": 32166 }, "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-01-02T10:00:36.000Z"), "id": 683347171605921792, "text": "Maybe tonight I'll call you after my blood, turns into alcohol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1033418419, "name": "kearrrrrr", "screen_name": "FosheeKearstin", "lang": "en", "location": "null", "create_at": date("2012-12-24"), "description": "Soccer, Doctor Who, Twenty One Pilots", "followers_count": 424, "friends_count": 620, "statues_count": 8893 }, "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-01-02T10:00:36.000Z"), "id": 683347171761106944, "text": "@cwarzel yep.", "in_reply_to_status": 683347065544511494, "in_reply_to_user": 20178419, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 20178419 }}, "user": { "id": 12263542, "name": "Matthew Zeitlin", "screen_name": "MattZeitlin", "lang": "en", "location": "New York", "create_at": date("2008-01-15"), "description": "reporter @BuzzFeedNews matt.zeitlin@buzzfeed.com", "followers_count": 12843, "friends_count": 2864, "statues_count": 2839 }, "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-02T10:00:36.000Z"), "id": 683347171790471168, "text": "Finished quilt and pillow. #penguinquilt https://t.co/Q06fyTcAo0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "penguinquilt" }}, "user": { "id": 124169356, "name": "Shelly Karstens", "screen_name": "Shellygirl88", "lang": "en", "location": "Wisconsin", "create_at": date("2010-03-18"), "description": "Library Media Specialist in the Green Bay Public Schools. Loves books and technology.", "followers_count": 169, "friends_count": 225, "statues_count": 1731 }, "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-01-02T10:00:36.000Z"), "id": 683347172490895362, "text": "That's my QB dammit\nNow we're playing\n@J_Allison4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 784872852 }}, "user": { "id": 2673202871, "name": "keeper of 3 kings", "screen_name": "_QueenLioness_", "lang": "en", "location": "null", "create_at": date("2014-07-04"), "description": "queen of 3 kings #DemMullinsBoys and princess Suga.\nfake people sicken me", "followers_count": 546, "friends_count": 95, "statues_count": 26362 }, "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-02T10:00:36.000Z"), "id": 683347172809641984, "text": "We ❤️ our customer #selfies \n\nThe beautiful londynnicole1988 in our Peruvian body wave hair.… https://t.co/j9y9y5Qzud", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.2573318,36.9168396"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "selfies" }}, "user": { "id": 620447488, "name": "Weave Envy", "screen_name": "Weave_Envy", "lang": "en", "location": "Norfolk, VA", "create_at": date("2012-06-27"), "description": "WeaveEnvy Beauty Shoppe Women's Retail shop specializing in Hair Extensions, Skin care, Makeup services and exclusive apparel all under one roof.\n757-454-6338", "followers_count": 464, "friends_count": 897, "statues_count": 1567 }, "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-02T10:00:36.000Z"), "id": 683347172956372994, "text": "Start the new year right off with some New Years money DM me! https://t.co/mvlbiRijLe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 127158479, "name": "K.Ro$$", "screen_name": "RawsyFlawsy", "lang": "en", "location": "Los Angeles", "create_at": date("2010-03-28"), "description": "A Los Angeles Legend", "followers_count": 2324, "friends_count": 382, "statues_count": 7516 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-02T10:00:36.000Z"), "id": 683347173262536704, "text": "The fuck do them entertainment/sports drama blogger bitches do really? Just lurk every rapper/athlete/paid thottie social pages for dirt?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20023186, "name": "Chaun Bandelay", "screen_name": "RooseveltChaun", "lang": "en", "location": "Astro Blue, On dese Heaux", "create_at": date("2009-02-03"), "description": "its a gift, don't get shot for kicks. #ygrh", "followers_count": 1681, "friends_count": 671, "statues_count": 259345 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-02T10:00:36.000Z"), "id": 683347173354921984, "text": "Click on your tweet and click the 3 dots, and then press pin to your profile https://t.co/ZurM9UPk4X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3040961495, "name": "Destiny", "screen_name": "_baabydes", "lang": "en", "location": "dreamville, USA ", "create_at": date("2015-02-16"), "description": "Do you stand tall & be bold, or do you fold? | #MPR", "followers_count": 1214, "friends_count": 897, "statues_count": 27838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hope Mills, NC", "id": "014cce082fca34f9", "name": "Hope Mills", "place_type": "city", "bounding_box": rectangle("-79.033596,34.894886 -78.834267,35.009696") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37051, "countyName": "Cumberland", "cityID": 3732640, "cityName": "Hope Mills" } }
+{ "create_at": datetime("2016-01-02T10:00:36.000Z"), "id": 683347173816205313, "text": "We're #hiring! Click to apply: CDL-A Driver Team Drivers Conover, NC - https://t.co/IF6S6M38lx #Job #Conover, NC https://t.co/3NCqAgLS4B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2186933,35.7065217"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Job", "Conover" }}, "user": { "id": 2841515944, "name": "WKSH Jobs", "screen_name": "JoinWKSH", "lang": "en", "location": "null", "create_at": date("2014-10-22"), "description": "Great jobs for great drivers We are looking for talented Owner Operators and company drivers. Get the facts and get on the road with Watkins Shepard.", "followers_count": 74, "friends_count": 1, "statues_count": 276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conover, NC", "id": "4380b04f6eef3d09", "name": "Conover", "place_type": "city", "bounding_box": rectangle("-81.267049,35.682082 -81.159804,35.747116") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37035, "countyName": "Catawba", "cityID": 3714340, "cityName": "Conover" } }
+{ "create_at": datetime("2016-01-02T10:00:36.000Z"), "id": 683347173937819648, "text": "https://t.co/Ss3usCxkov", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2907759283, "name": "William Woods", "screen_name": "willwoodssr1", "lang": "en", "location": "null", "create_at": date("2014-11-22"), "description": "null", "followers_count": 672, "friends_count": 1230, "statues_count": 3479 }, "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-02T10:00:36.000Z"), "id": 683347174160248832, "text": "Drinking a Miller Lite by @MillerCoors at @jaxevents — https://t.co/veyGiaKNtM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.6386,30.3246"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 441998843, 14687803 }}, "user": { "id": 456711123, "name": "Chris Ziemer", "screen_name": "BeerManZ", "lang": "en", "location": "Kissimmee, FL USA", "create_at": date("2012-01-06"), "description": "Disney Fanatic, Beer Enthusiast and overall great guy", "followers_count": 91, "friends_count": 369, "statues_count": 967 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midtown, Jacksonville", "id": "74d045c1a394f792", "name": "Midtown", "place_type": "neighborhood", "bounding_box": rectangle("-81.647881,30.318083 -81.62476,30.333091") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12031, "countyName": "Duval", "cityID": 1235000, "cityName": "Jacksonville" } }
+{ "create_at": datetime("2016-01-02T10:00:36.000Z"), "id": 683347174252507136, "text": "Back to the grind tomorrow ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 299357926, "name": "Ramsey", "screen_name": "DeCarlo_15", "lang": "en", "location": "null", "create_at": date("2011-05-15"), "description": "Be 1% better everyday. \nU.S Army 91B", "followers_count": 433, "friends_count": 344, "statues_count": 20834 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenville, RI", "id": "c0c921eebc73be01", "name": "Greenville", "place_type": "city", "bounding_box": rectangle("-71.579975,41.855076 -71.524402,41.921347") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4431600, "cityName": "Greenville" } }
+{ "create_at": datetime("2016-01-02T10:00:37.000Z"), "id": 683347174432878593, "text": "@OBJ_3 @OBJ_3 do the right thing https://t.co/t7nQigWxOr", "in_reply_to_status": -1, "in_reply_to_user": 380730306, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 380730306, 380730306 }}, "user": { "id": 450737591, "name": "malik", "screen_name": "B1GLEEK", "lang": "en", "location": "null", "create_at": date("2011-12-30"), "description": "Philippians 4:13 I can do anything i put my mind to! it's all love and nobody ever hated.", "followers_count": 1132, "friends_count": 1100, "statues_count": 42565 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mechanicstown, NY", "id": "00afb5d51a9334f6", "name": "Mechanicstown", "place_type": "city", "bounding_box": rectangle("-74.419087,41.428134 -74.340666,41.472077") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36071, "countyName": "Orange", "cityID": 3646349, "cityName": "Mechanicstown" } }
+{ "create_at": datetime("2016-01-02T10:00:37.000Z"), "id": 683347174453714944, "text": "We're #hiring! Click to apply: Senior Global IT Infrastructure Auditor - https://t.co/HGXF9kgSJb #BusinessMgmt #Merrimack, NH #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.4948355,42.8678651"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "BusinessMgmt", "Merrimack", "Job", "Jobs" }}, "user": { "id": 22028695, "name": "TMJ-NH Mgmt. Jobs", "screen_name": "tmj_nh_mgmt", "lang": "en", "location": "New Hampshire", "create_at": date("2009-02-26"), "description": "Follow this account for geo-targeted Business/Mgmt. job tweets in New Hampshire Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 289, "friends_count": 244, "statues_count": 104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merrimack, NH", "id": "000a400a1aba18e8", "name": "Merrimack", "place_type": "city", "bounding_box": rectangle("-71.589882,42.790953 -71.453424,42.910922") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33011, "countyName": "Hillsborough", "cityID": 3321780, "cityName": "East Merrimack" } }
+{ "create_at": datetime("2016-01-02T10:00:37.000Z"), "id": 683347174747275264, "text": "Fucking Super Donuts���� https://t.co/16Sht7InWK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 848636726, "name": "Killa Kev™", "screen_name": "kevin_kwas", "lang": "en", "location": "null", "create_at": date("2012-09-26"), "description": "Huron 17'", "followers_count": 421, "friends_count": 305, "statues_count": 32540 }, "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-02T10:00:37.000Z"), "id": 683347174810206208, "text": "I can't wait to call this place home in 6 months and spend every weekend on the beach.… https://t.co/Sp5LFBkAFh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.05450461,26.89364835"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 99440361, "name": "Kells", "screen_name": "David_Keller_", "lang": "en", "location": "Madison, WI ✈️ Nashville, TN", "create_at": date("2009-12-25"), "description": "OT student graduating in the spring of '16 and heading South for good. Packers. Bucks. Brewers. Badgers. Home Improvement Enthusiast.", "followers_count": 271, "friends_count": 358, "statues_count": 16784 }, "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": 12099, "countyName": "Palm Beach", "cityID": 1235850, "cityName": "Juno Beach" } }
+{ "create_at": datetime("2016-01-02T10:00:37.000Z"), "id": 683347174835404800, "text": "I absolutely love bare minerals foundation. It took me like 5 years to find a foundation that actually compliments my skin. I'm stoked", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1059071186, "name": "cass", "screen_name": "cassadelics", "lang": "en", "location": "null", "create_at": date("2013-01-03"), "description": "TB❤️ Monterey Peninsula, CA", "followers_count": 131, "friends_count": 89, "statues_count": 1433 }, "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-02T10:00:37.000Z"), "id": 683347174910898177, "text": "De cualquier manera es rico", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2635203026, "name": "_milanno", "screen_name": "__mota_milano", "lang": "es", "location": "California, USA", "create_at": date("2014-07-12"), "description": "La liberad no existe. La lógica es solo una palabra. Eres libre de alma. La gente es un asco. Piensa diferente. ACUARIANO, mente activa. Aire.", "followers_count": 338, "friends_count": 246, "statues_count": 5176 }, "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-02T10:00:37.000Z"), "id": 683347175028428801, "text": "Black https://t.co/O0TiHON5gr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1383162211, "name": "Myaaaa", "screen_name": "Mya_Darling_x3", "lang": "en", "location": "860", "create_at": date("2013-04-26"), "description": "aa| zr | The Ville | Views From the 6| Aries | princess af | Gay 4 Rihanna |", "followers_count": 514, "friends_count": 531, "statues_count": 7927 }, "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-02T10:00:37.000Z"), "id": 683347175036686337, "text": "Demi Lovato- father>>>>> ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 428434005, "name": "Raii ✮", "screen_name": "_raiyon", "lang": "en", "location": "I❤️Kae", "create_at": date("2011-12-04"), "description": "sparkle and shine ✨", "followers_count": 445, "friends_count": 392, "statues_count": 21158 }, "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-02T10:00:37.000Z"), "id": 683347175091355649, "text": "@apiergustavo14 https://t.co/kqqU2DfL5o", "in_reply_to_status": -1, "in_reply_to_user": 495554123, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 495554123 }}, "user": { "id": 438722706, "name": "Taylor", "screen_name": "TLeBrun_4", "lang": "en", "location": "null", "create_at": date("2011-12-16"), "description": "Simmons College PT '19 ⚾️", "followers_count": 493, "friends_count": 896, "statues_count": 12621 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milford, MA", "id": "002564726a89af1d", "name": "Milford", "place_type": "city", "bounding_box": rectangle("-71.57581,42.116846 -71.47777,42.191514") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25027, "countyName": "Worcester", "cityID": 2541200, "cityName": "Milford" } }
+{ "create_at": datetime("2016-01-02T10:00:37.000Z"), "id": 683347175703736320, "text": "Just been to Acrylic Aquarium's aka Tanked and Wayde King was in, nice truck�� https://t.co/lKHA7BSb8G", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 51721692, "name": "iVan Sestan", "screen_name": "ivansestan", "lang": "en", "location": "Liverpool", "create_at": date("2009-06-28"), "description": "UK Tech Futurologist, likes interesting people, music, films, travel, art, philanthropy and much more", "followers_count": 117, "friends_count": 47, "statues_count": 4452 }, "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-02T10:00:37.000Z"), "id": 683347175875735552, "text": "That bit in Comedians in Cars Getting Coffee with Obama where in his bombproof limo he shows Seinfeld his drone strike console. Dark.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 193753, "name": "Damien Mulley ", "screen_name": "damienmulley", "lang": "en", "location": "Cork, Ireland", "create_at": date("2006-12-23"), "description": "I like words that start with C. Reculer pour mieux sauter. You can DM me without me following you. But DM for RT = block :) #fuckoffier ™ 734,300", "followers_count": 16913, "friends_count": 975, "statues_count": 189777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gaylord, MI", "id": "0e48e78d08d90c6c", "name": "Gaylord", "place_type": "city", "bounding_box": rectangle("-84.72699,44.981869 -84.646793,45.045994") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26137, "countyName": "Otsego", "cityID": 2631720, "cityName": "Gaylord" } }
+{ "create_at": datetime("2016-01-02T10:00:37.000Z"), "id": 683347175934267392, "text": "This #Clerical #job might be a great fit for you: Supervisor Clinic Practice Job - https://t.co/VePVSznkl4 #Littleton, CO #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.0166498,39.613321"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Clerical", "job", "Littleton", "Hiring" }}, "user": { "id": 3290306581, "name": "Centura Careers", "screen_name": "CenturaCareers", "lang": "en", "location": "null", "create_at": date("2015-07-24"), "description": "We are the region’s leading health care leader, focused on connecting Colorado and Kansas to affordable, world-class care. Follow to learn about our jobs!", "followers_count": 163, "friends_count": 752, "statues_count": 657 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Littleton, CO", "id": "910868722e1e4825", "name": "Littleton", "place_type": "city", "bounding_box": rectangle("-105.062023,39.563868 -104.973551,39.62983") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8005, "countyName": "Arapahoe", "cityID": 845255, "cityName": "Littleton" } }
+{ "create_at": datetime("2016-01-02T10:00:37.000Z"), "id": 683347176127377408, "text": "P1190855.JPG https://t.co/U3ioxvjInY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-123.01151207,37.69918948"), "retweet_count": 0, "lang": "lt", "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": 4, "friends_count": 0, "statues_count": 9737 }, "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-02T10:00:37.000Z"), "id": 683347176232099841, "text": "@ImaniNaeem_ @BacksliderDee well that's pretty much first church right now...", "in_reply_to_status": 683346948657680385, "in_reply_to_user": 1373607878, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1373607878, 30986817 }}, "user": { "id": 268515028, "name": "Art Hoë", "screen_name": "Vixnnixn", "lang": "en", "location": "OU Twitter, What's Good?", "create_at": date("2011-03-18"), "description": "Unfriendly Black Hottie // The Panther Movement // I Support the SlutWalk //God Fearing // Art Hoe Movement // Feminist //#OU18", "followers_count": 438, "friends_count": 312, "statues_count": 17868 }, "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-02T10:00:37.000Z"), "id": 683347176265793538, "text": "Soooo are you supposed to ask for head or is the girl just supposed to already 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": 2724830492, "name": "ひCaptainSaveAHoe☄✨", "screen_name": "HoesLoveJoeDirt", "lang": "en", "location": "SAV912✈️Memphis901✈️SAV912", "create_at": date("2014-08-11"), "description": "#SouthernNotState #UglyGangCEO✨ SC: joedirty_22 RestEasyFatQuail", "followers_count": 1973, "friends_count": 2080, "statues_count": 116202 }, "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-02T10:00:37.000Z"), "id": 683347176320282624, "text": "@carrieffisher = awesome. Everything she said/did in recent months was cool. Writers and Producers should have her in mind as they create.", "in_reply_to_status": -1, "in_reply_to_user": 75641903, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 75641903 }}, "user": { "id": 18469879, "name": "jasonstaten", "screen_name": "jasonstaten", "lang": "en", "location": "Atlanta GA", "create_at": date("2008-12-29"), "description": "Write, budget, Shoot, Edit, Repeat", "followers_count": 896, "friends_count": 1717, "statues_count": 2949 }, "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-02T10:00:37.000Z"), "id": 683347176387428356, "text": "@i_am_fabs @Bareburger New York ����", "in_reply_to_status": 683345180771090432, "in_reply_to_user": 5476332, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 5476332, 99144994 }}, "user": { "id": 155188625, "name": "Daniela Holzer", "screen_name": "matscha", "lang": "en", "location": "Vienna", "create_at": date("2010-06-13"), "description": "working as digital team leader @schonbrunn #contentstrategy student w/ @auscarmen @jakobi_claudia. RTs are not endorsements, opinions are my own", "followers_count": 530, "friends_count": 573, "statues_count": 2702 }, "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-02T10:00:37.000Z"), "id": 683347176613920769, "text": "Wen I come up Ima leave the ���� n the grass and only take the niggas THT was there since day 1 and THT did shit 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": 1864520148, "name": "primetime", "screen_name": "Donovanfrom900", "lang": "en", "location": "null", "create_at": date("2013-09-14"), "description": "uptown 900blk till the death bed R.I.P BREWSIE AVA", "followers_count": 864, "friends_count": 759, "statues_count": 12108 }, "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-02T10:00:37.000Z"), "id": 683347177133871108, "text": "See our latest #Richmond, VA #job and click to apply: Weekend Verifier, Part-time (Brandermill area) - https://t.co/8M5zZKexPy #nowhiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.469092,37.542979"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Richmond", "job", "nowhiring" }}, "user": { "id": 2690479897, "name": "BH Media Jobs", "screen_name": "BHMediaJobs", "lang": "en", "location": "Nationwide", "create_at": date("2014-07-29"), "description": "null", "followers_count": 246, "friends_count": 0, "statues_count": 1797 }, "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-02T10:00:37.000Z"), "id": 683347177515659266, "text": "lol im piercing my ears rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 513695960, "name": "jessoni ann", "screen_name": "soitsjess", "lang": "en", "location": "Minooka, IL", "create_at": date("2012-03-03"), "description": "Shel Bell is my prison wife and weigi is my boyfriend", "followers_count": 302, "friends_count": 545, "statues_count": 3694 }, "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-02T10:00:37.000Z"), "id": 683347177532309504, "text": "2nd�� https://t.co/xXfiDLSg2q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2951384420, "name": "Kristèn .", "screen_name": "Whitegalkris", "lang": "en", "location": "null", "create_at": date("2014-12-29"), "description": "Follow Me On Instagram @Whitegalkris__✈️ AMOSC Whitegalkris⌛️", "followers_count": 1750, "friends_count": 986, "statues_count": 15911 }, "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-02T10:00:37.000Z"), "id": 683347177620533249, "text": "@_OverImpossible this nigga", "in_reply_to_status": 683345930850975744, "in_reply_to_user": 499932561, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 499932561 }}, "user": { "id": 601222922, "name": "Lucid", "screen_name": "WuTang_Mang", "lang": "en", "location": "Ma", "create_at": date("2012-06-06"), "description": "THE BEST MF IN THE FIELD #Bipolar #BMG It's all just a lucid dream!", "followers_count": 454, "friends_count": 526, "statues_count": 18764 }, "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("2016-01-02T10:00:37.000Z"), "id": 683347178136285184, "text": "@Powerful @jaayk_", "in_reply_to_status": 683089372833538048, "in_reply_to_user": 351951741, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 351951741, 612280204 }}, "user": { "id": 421948613, "name": "ღ sASSy ღ", "screen_name": "ShonticDavis", "lang": "en", "location": "null", "create_at": date("2011-11-26"), "description": "@DonKrez_\n6.22 & 11.6 ❤", "followers_count": 932, "friends_count": 1981, "statues_count": 31346 }, "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-02T10:00:37.000Z"), "id": 683347178505564160, "text": "Big scary black man https://t.co/oqW9f9skcd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 241562484, "name": "Yves St.Lauren", "screen_name": "Kolour_mePINK", "lang": "en", "location": "KING", "create_at": date("2011-01-22"), "description": "I just write beautiful love stories I don't think i'll ever live one", "followers_count": 1134, "friends_count": 988, "statues_count": 77442 }, "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-02T10:00:38.000Z"), "id": 683347178593492992, "text": "BNP Paribas Open- Championship Match Indian Wells \n\nTennis Garden - Stadium 1\nIndian Wells, CA Sun, Mar 20 2016\n\nhttps://t.co/7aiYlr4OKc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 117912012, "name": "eticketsin", "screen_name": "eticketsin", "lang": "en", "location": "Boston, United States", "create_at": date("2010-02-26"), "description": "eticketsin gives you access to great tickets.Find and Buy tickets for sports events, concert events, theater events.", "followers_count": 2469, "friends_count": 2278, "statues_count": 6280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Foxborough, MA", "id": "a204fee6e70c68a6", "name": "Foxborough", "place_type": "city", "bounding_box": rectangle("-71.289373,42.014947 -71.178008,42.106307") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25021, "countyName": "Norfolk", "cityID": 2524855, "cityName": "Foxborough" } }
+{ "create_at": datetime("2016-01-02T10:00:38.000Z"), "id": 683347178983559169, "text": "Forever #thuggin it ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thuggin" }}, "user": { "id": 77068504, "name": "Sir Flex-ALOT #Rx3", "screen_name": "papitodvnny", "lang": "en", "location": "New Warleans ", "create_at": date("2009-09-24"), "description": "R'eal | R'ecognize | R'eal Ent. #Rx3 #GotMyOwnWave #LoveNLoyaly #NeverChangedUp #YUMad #ARTIST. PapitoDvnny504@gmail.com", "followers_count": 1329, "friends_count": 908, "statues_count": 82899 }, "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-02T10:00:38.000Z"), "id": 683347179298250754, "text": "https://t.co/jHreB8ugBS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1546465909, "name": "Kat", "screen_name": "PINENIPPLESS", "lang": "en", "location": "South Florida", "create_at": date("2013-06-25"), "description": "Miami sports & wine fanatic", "followers_count": 4616, "friends_count": 597, "statues_count": 178357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pine Manor, FL", "id": "bb3c878553134384", "name": "Pine Manor", "place_type": "city", "bounding_box": rectangle("-81.884278,26.567963 -81.871876,26.581244") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12071, "countyName": "Lee", "cityID": 1257025, "cityName": "Pine Manor" } }
+{ "create_at": datetime("2016-01-02T10:00:38.000Z"), "id": 683347179340050432, "text": "tonight ���� https://t.co/tEkCJ76kyN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 628113239, "name": "SC: todopejayy", "screen_name": "todopejay", "lang": "en", "location": "Toronto, Ontario", "create_at": date("2012-07-05"), "description": "null", "followers_count": 303, "friends_count": 127, "statues_count": 11611 }, "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-02T10:00:38.000Z"), "id": 683347179440717824, "text": "Helping my sister out. I wouldn't trade her 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": 1010572285, "name": "Tina Massey", "screen_name": "oneluckywoman2u", "lang": "en", "location": "null", "create_at": date("2012-12-13"), "description": "null", "followers_count": 130, "friends_count": 332, "statues_count": 731 }, "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-01-02T10:00:38.000Z"), "id": 683347179457646592, "text": "�� https://t.co/KcdZbzlbTX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2202054037, "name": "Luis Angel", "screen_name": "LuiggyD_", "lang": "en", "location": "Bronx NYC", "create_at": date("2013-11-18"), "description": "IG & SC kingluiggy", "followers_count": 316, "friends_count": 287, "statues_count": 37545 }, "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-02T10:00:38.000Z"), "id": 683347179872890880, "text": "Black and privileged = Tre Smith", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46587084, "name": "Tre S", "screen_name": "TreSmithers", "lang": "en", "location": "Cleveland, OH", "create_at": date("2009-06-11"), "description": "open format dj. producer. tracking engineer. blogger via @westaychill. founder of #BIXBFest. Paypal: http://www.paypal.me/TreSmith | 1 DM could change our lives", "followers_count": 2131, "friends_count": 956, "statues_count": 38308 }, "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-01-02T10:00:38.000Z"), "id": 683347180124549120, "text": "@SCompton72 he's never touched a woman", "in_reply_to_status": 683346638077693952, "in_reply_to_user": 1029865488, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1029865488 }}, "user": { "id": 2957515706, "name": "HeisenVol", "screen_name": "MrSmokeyMcVol", "lang": "en", "location": "Fountain City TN. ", "create_at": date("2015-01-03"), "description": "TheNorthKnox. Former standout at Wade Houston's basketball camp. I eat Honeycrisp apples because all other apples are ratchet.", "followers_count": 542, "friends_count": 722, "statues_count": 4264 }, "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-02T10:00:38.000Z"), "id": 683347180199911426, "text": "Want to work in #Indianapolis, IN? View our latest opening: https://t.co/pF9qqrikz8 #Nursing #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.1580423,39.7683765"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Indianapolis", "Nursing", "Job", "Jobs", "Hiring" }}, "user": { "id": 3300641331, "name": "Community Health", "screen_name": "WorkAtCHNw", "lang": "en", "location": "null", "create_at": date("2015-05-27"), "description": "Explore current job opportunities at @CHNw, Central Indiana’s leader in providing exceptional health services at over 200 sites of care.", "followers_count": 75, "friends_count": 31, "statues_count": 827 }, "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-01-02T10:00:38.000Z"), "id": 683347180426530816, "text": "YO FUCK YOU. YOU FUCKING PIECE OF SHIT. YOURE A PIECE OF TRASH.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2287320530, "name": "antisocial pessimist", "screen_name": "_upthebutt_", "lang": "en", "location": "Newington, CT", "create_at": date("2014-01-11"), "description": "- I'm too busy working on my own grass to notice if yours is greener -", "followers_count": 246, "friends_count": 239, "statues_count": 2551 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newington, CT", "id": "89df89b3b7ac521b", "name": "Newington", "place_type": "city", "bounding_box": rectangle("-72.762431,41.647296 -72.700621,41.724482") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 952210, "cityName": "Newington" } }
+{ "create_at": datetime("2016-01-02T10:00:38.000Z"), "id": 683347180531380225, "text": "@WainscottKyle my teams playing yours. My money's on mine ��", "in_reply_to_status": -1, "in_reply_to_user": 3328075845, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3328075845 }}, "user": { "id": 441085273, "name": "Courtney McPheters", "screen_name": "ballinondacourt", "lang": "en", "location": "null", "create_at": date("2011-12-19"), "description": "Trine University Women's Basketball #44 ⚡️ Homestead GBB State runner up. in love with cities i've never been to and people i've never met", "followers_count": 545, "friends_count": 530, "statues_count": 15235 }, "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-02T10:00:38.000Z"), "id": 683347180841648128, "text": "@RoseBityRae @JoeneenH she need a 2x nigga in her life lol", "in_reply_to_status": 683346704771321856, "in_reply_to_user": 125873905, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 125873905, 3619465213 }}, "user": { "id": 2740223767, "name": "PUGBUBS❌", "screen_name": "BIGMONEYBUBS", "lang": "en", "location": "Ontario, CA", "create_at": date("2014-08-17"), "description": "IN THE 9 LIKE THREE 3's ! ❌PULLUPGANG", "followers_count": 301, "friends_count": 343, "statues_count": 8668 }, "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-02T10:00:38.000Z"), "id": 683347180971671552, "text": "We're #hiring! Click to apply: REGISTRAR - https://t.co/QZLhrymgf5 #Clerical #CentegraJobs #Woodstock, IL #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.400195,42.275973"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Clerical", "CentegraJobs", "Woodstock", "Job", "Jobs" }}, "user": { "id": 607451943, "name": "Centegra Careers", "screen_name": "centegracareers", "lang": "en", "location": "McHenry, IL ", "create_at": date("2012-06-13"), "description": "This is the official #careers & #jobs social site for Centegra Health System. #nowhiring in #McHenry County. Excellence is a way of life!", "followers_count": 693, "friends_count": 512, "statues_count": 2862 }, "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": 17111, "countyName": "McHenry", "cityID": 1717887, "cityName": "Crystal Lake" } }
+{ "create_at": datetime("2016-01-02T10:00:38.000Z"), "id": 683347181202477056, "text": "@Zach_Dorsey we're patient. Don't worry", "in_reply_to_status": -1, "in_reply_to_user": 251796499, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 251796499 }}, "user": { "id": 799176576, "name": "Stephanie ♡", "screen_name": "AMsBigClark", "lang": "en", "location": "New York, NY", "create_at": date("2012-09-02"), "description": "| | 4/4 | | 12.09.14 | 08.11.15 | 08.16.15 | 11.11.15 | 11.12.15 | @AustinMahone everyday", "followers_count": 1889, "friends_count": 988, "statues_count": 20639 }, "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-02T10:00:38.000Z"), "id": 683347181395439620, "text": "I want kush", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 327178366, "name": "JK", "screen_name": "JACKYLE_96", "lang": "en", "location": "my room ", "create_at": date("2011-06-30"), "description": "Izdrāzt Visiem", "followers_count": 151, "friends_count": 36, "statues_count": 1753 }, "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-02T10:00:38.000Z"), "id": 683347181412089858, "text": "See our latest #Irving, TX #job and click to apply: CDL Delivery Driver - https://t.co/eQrbHrMk2n #Retail #Hiring", "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": {{ "Irving", "job", "Retail", "Hiring" }}, "user": { "id": 28443787, "name": "TMJ-DFW Retail Jobs", "screen_name": "tmj_dfw_retail", "lang": "en", "location": "DFW, Texas", "create_at": date("2009-04-02"), "description": "Follow this account for geo-targeted Retail job tweets in DFW, Texas. Need help? Tweet us at @CareerArc!", "followers_count": 473, "friends_count": 313, "statues_count": 1530 }, "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-03T10:00:00.000Z"), "id": 683709407075647488, "text": "Coffee is actually pretty gross.✌��️ https://t.co/C7DRU5RBm7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.276591,37.740074"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 81759362, "name": "Meagan Elizabitch", "screen_name": "awhhKitty", "lang": "en", "location": "Rivendell", "create_at": date("2009-10-11"), "description": "Meagan. Senior at ECHS. Gilbert is pretty great. My cats are my children. And no, I don't play flute.✌️ ▪️2015 KMEA SMBC AA State Champion▪️", "followers_count": 250, "friends_count": 230, "statues_count": 1155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Starbucks", "id": "07d9f05211883000", "name": "Starbucks", "place_type": "poi", "bounding_box": rectangle("-84.27659109999999,37.7400739 -84.276591,37.740074") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21151, "countyName": "Madison", "cityID": 2165226, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-01-03T10:00:00.000Z"), "id": 683709407281037312, "text": "Like some, love some, trust none.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 316599472, "name": "Jaden", "screen_name": "JadenSmittyCox", "lang": "en", "location": "Cedar City, Utah", "create_at": date("2011-06-13"), "description": "20... SUU T-Bird SC:jadencox1", "followers_count": 770, "friends_count": 1163, "statues_count": 7666 }, "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-01-03T10:00:00.000Z"), "id": 683709407331528705, "text": "I'm at Starbucks in Des Moines, IA https://t.co/cIL0qjtyST", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.69817,41.63118"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18100387, "name": "RyanKolder", "screen_name": "RyanKolder", "lang": "en", "location": "Des Moines Iowa", "create_at": date("2008-12-13"), "description": "null", "followers_count": 564, "friends_count": 1314, "statues_count": 2229 }, "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-03T10:00:00.000Z"), "id": 683709407780278277, "text": "CHRIS IS BACK IN MARYLAND YAYAYAYAYAYAAYYAYAAYAY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 91861753, "name": "Kaysha", "screen_name": "KaylaShrewsbury", "lang": "en", "location": "null", "create_at": date("2009-11-22"), "description": "South River '16", "followers_count": 446, "friends_count": 226, "statues_count": 10021 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crofton, MD", "id": "63e2540eaa633704", "name": "Crofton", "place_type": "city", "bounding_box": rectangle("-76.707926,38.980007 -76.63862,39.029866") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24003, "countyName": "Anne Arundel", "cityID": 2420875, "cityName": "Crofton" } }
+{ "create_at": datetime("2016-01-03T10:00:00.000Z"), "id": 683709407801155584, "text": "Yay another person got engaged...so happy 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": 368118981, "name": "Amanda Wood", "screen_name": "Amanda3693", "lang": "en", "location": "Chico, CA", "create_at": date("2011-09-04"), "description": "this is the story of a girl", "followers_count": 264, "friends_count": 243, "statues_count": 4837 }, "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.714333") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 624638, "cityName": "Folsom" } }
+{ "create_at": datetime("2016-01-03T10:00:00.000Z"), "id": 683709408275124224, "text": "Leaving Meme and Papaw's house in Zephyrhills, Florida. On our way to spend our Anniversary at a… https://t.co/7MaEkCfX6J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.20703158,28.22424655"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2539034672, "name": "The Megan Ellis Band", "screen_name": "MeganEllisBand", "lang": "en", "location": "null", "create_at": date("2014-06-01"), "description": "A happy newly wed, singing my heart out in Nashville for my rich style of living \n\nhttp://favstar.fm/users/meganellisband ❤️❤️❤️", "followers_count": 274, "friends_count": 446, "statues_count": 470 }, "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-01-03T10:00:00.000Z"), "id": 683709408325570560, "text": "@_melanyge thank you gorgeous��������", "in_reply_to_status": 683709100031488001, "in_reply_to_user": 365107528, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 365107528 }}, "user": { "id": 4055983035, "name": "slimthickkkk❣", "screen_name": "leesacehoe", "lang": "en", "location": "null", "create_at": date("2015-10-27"), "description": "@Beyoncediaz", "followers_count": 381, "friends_count": 383, "statues_count": 1407 }, "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-01-03T10:00:00.000Z"), "id": 683709408405135360, "text": "you know you slept crazy when .... https://t.co/NFR0LE6jvi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4340089880, "name": "m(i)(c)a(n't) benge", "screen_name": "miabengie", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2015-11-23"), "description": "volleyball is kinda my thing :)", "followers_count": 180, "friends_count": 295, "statues_count": 1232 }, "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-03T10:00:00.000Z"), "id": 683709408682053632, "text": "Their answer is that is closer to the turnpike thus closer to BG. Alright.", "in_reply_to_status": 683709251852828672, "in_reply_to_user": 235709935, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 235709935, "name": "Selena Glomez", "screen_name": "SJ_lovesyou", "lang": "xx-lc", "location": "in between joy and pain", "create_at": date("2011-01-08"), "description": "BGSU journalism major. Twinsburg Alum '14.", "followers_count": 891, "friends_count": 419, "statues_count": 88092 }, "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-01-03T10:00:00.000Z"), "id": 683709408711323648, "text": "#STONERSCIRCLE\n─▀██▀▀▀█\n──██▄█\n──██▀█ OLLOW�� @imnicetho \n─▄██", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "STONERSCIRCLE" }}, "user_mentions": {{ 495618652 }}, "user": { "id": 3226449510, "name": "jfΣծ", "screen_name": "JFed_", "lang": "en", "location": "vapιng on a cloυd", "create_at": date("2015-05-25"), "description": "CEO/PREZ:#ѕтonerѕcιrcle✊|| \n вooĸιng ιnғo: jғed816@gмaιhttp://l.coм", "followers_count": 18455, "friends_count": 15753, "statues_count": 16508 }, "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-03T10:00:00.000Z"), "id": 683709408862273536, "text": "girl wait ������������ https://t.co/fc1S7EuRdp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 251400261, "name": "rah smoove", "screen_name": "_laslimm", "lang": "en", "location": "Laffy, La", "create_at": date("2011-02-12"), "description": "aux cord queen ✨", "followers_count": 2730, "friends_count": 991, "statues_count": 86513 }, "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-03T10:00:00.000Z"), "id": 683709409072029696, "text": "#GoCowboys", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "GoCowboys" }}, "user": { "id": 38446224, "name": "Freddy", "screen_name": "elbigfred", "lang": "en", "location": "Socorro, TX", "create_at": date("2009-05-07"), "description": "Assistant Football Coach and Special Education Teacher at Socorro High School! #BulldogFootball #Chatos I Corinthians 9:24-27", "followers_count": 581, "friends_count": 1116, "statues_count": 9625 }, "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-03T10:00:00.000Z"), "id": 683709409072037892, "text": "https://t.co/brESbfAID9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 281822817, "name": "That Valley Life", "screen_name": "Showroom_77", "lang": "en", "location": "West Hills, CA", "create_at": date("2011-04-13"), "description": "strangers til we meet...", "followers_count": 14738, "friends_count": 4484, "statues_count": 35414 }, "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-03T10:00:00.000Z"), "id": 683709409474809859, "text": "Lmao �������� https://t.co/suXEnyzSsZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 598486121, "name": "kyndall", "screen_name": "kyntayllor", "lang": "en", "location": "San Antonio, TX ", "create_at": date("2012-06-03"), "description": "19. ✝. Angelo State. sc: kyndaalll", "followers_count": 819, "friends_count": 599, "statues_count": 26957 }, "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-03T10:00:00.000Z"), "id": 683709409650814980, "text": "Do I have \"treat me like shit \" written on my forehead? Cause damn dude.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3382588828, "name": "jumanji", "screen_name": "myanji", "lang": "en", "location": "null", "create_at": date("2015-07-18"), "description": "null", "followers_count": 39, "friends_count": 31, "statues_count": 68 }, "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-03T10:00:00.000Z"), "id": 683709409684512772, "text": "sorry i poured alcohol in your eyes @ kelsey", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 395028542, "name": "sophie", "screen_name": "sophieeborga", "lang": "en", "location": "naptown", "create_at": date("2011-10-20"), "description": "Dirty Bury '19", "followers_count": 800, "friends_count": 675, "statues_count": 12482 }, "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-03T10:00:00.000Z"), "id": 683709410145898496, "text": "Ripley SW Limestone Co. Temp: 48.7°F Wind:2.2mph Pressure: 999.3mb 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": 46985 }, "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-03T10:00:00.000Z"), "id": 683709410175250432, "text": "@allankellynet Happy to hear any feedback you have on the product, we just rolled out the new design Friday, still improving the mkting docs", "in_reply_to_status": 683708985913044992, "in_reply_to_user": 33743565, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28073411 }}, "user": { "id": 33743565, "name": "Scott Feinberg", "screen_name": "scottefein", "lang": "en", "location": "New York, NY", "create_at": date("2009-04-20"), "description": "Doing API Things at the NYTimes. I also publish @TheWeeklyCFP and help conferences through Papercall.io.", "followers_count": 630, "friends_count": 157, "statues_count": 5964 }, "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-03T10:00:00.000Z"), "id": 683709410208649217, "text": "Last night was pretty damn good", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 532980255, "name": "J.Cuts", "screen_name": "TheBabeBandit", "lang": "en", "location": "Inland Empire. ", "create_at": date("2012-03-21"), "description": "20 summers. Paintball.", "followers_count": 508, "friends_count": 549, "statues_count": 8870 }, "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-03T10:00:00.000Z"), "id": 683709410221277184, "text": "@dean_demoss https://t.co/0S2F14BATJ", "in_reply_to_status": -1, "in_reply_to_user": 1172955720, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1172955720 }}, "user": { "id": 1113462602, "name": "єʏєẓѧċҡ", "screen_name": "iezaicungpaguyo", "lang": "en", "location": "null", "create_at": date("2013-01-22"), "description": "VCHS 2016", "followers_count": 205, "friends_count": 131, "statues_count": 7904 }, "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-03T10:00:00.000Z"), "id": 683709410250743810, "text": "@archbernard Um... Because Germans have a great track record on that.", "in_reply_to_status": 683705579999723521, "in_reply_to_user": 82766330, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 82766330 }}, "user": { "id": 39887061, "name": "Todd", "screen_name": "MTB_TC", "lang": "en", "location": "In yo face", "create_at": date("2009-05-13"), "description": "Visionary Thought Leader. Yeah, that's a thing.", "followers_count": 188, "friends_count": 175, "statues_count": 17196 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sugar Hill, GA", "id": "1d1d97727174539e", "name": "Sugar Hill", "place_type": "city", "bounding_box": rectangle("-84.10171,34.035008 -84.008326,34.151864") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13135, "countyName": "Gwinnett", "cityID": 1374180, "cityName": "Sugar Hill" } }
+{ "create_at": datetime("2016-01-03T10:00:00.000Z"), "id": 683709410376577024, "text": "I think I'm 3 minutes pregnant https://t.co/CIryhhCDSP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2768422365, "name": "reilly", "screen_name": "reillyccc", "lang": "en", "location": "Denver, CO", "create_at": date("2014-09-12"), "description": "why is there no oreo emoji", "followers_count": 283, "friends_count": 215, "statues_count": 3597 }, "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-01-03T10:00:00.000Z"), "id": 683709410393387009, "text": "https://t.co/83ItIX7nLf is this better or worse", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "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 @smolhinata @keleticism I Ra's Maraq - Hyperion", "followers_count": 399, "friends_count": 522, "statues_count": 88736 }, "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-03T10:00:00.000Z"), "id": 683709410519035904, "text": "What a beautiful day", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 313544910, "name": "Jaybreezy", "screen_name": "JesseJay_Hoops3", "lang": "en", "location": "null", "create_at": date("2011-06-08"), "description": "Basketball Is My Life | Instagram: FuckkYooPicuress | These are the only places you'll catch me following. #Dyna #Htown", "followers_count": 1638, "friends_count": 1616, "statues_count": 31802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-03T10:00:00.000Z"), "id": 683709410720530432, "text": "Wind 0 mph --. Barometer 30.16 in, Falling. Temperature 51.3 °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": 19944 }, "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-03T10:00:00.000Z"), "id": 683709410762469376, "text": "WEIRD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2194967788, "name": "5/22", "screen_name": "yannaeeee", "lang": "en", "location": "null", "create_at": date("2013-11-24"), "description": "Yae bitchhhh!", "followers_count": 644, "friends_count": 559, "statues_count": 7306 }, "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-03T10:00:00.000Z"), "id": 683709410959466496, "text": "Simon Danczuk blames 'drink problem' for lewd texts to teenager https://t.co/ccJEZZdOhG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.0887807,33.5380057"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16224698, "name": "Brad Hubert", "screen_name": "BradHubert", "lang": "en", "location": "Arizona ", "create_at": date("2008-09-10"), "description": "ASU MBA, Hillsdale College BA, Run, Yoga, Bike, Hike, Camp. Born in Ann Arbor, Raised in Tucson, Live in Phoenix, Work in Scottsdale", "followers_count": 3974, "friends_count": 3225, "statues_count": 24243 }, "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-03T10:00:00.000Z"), "id": 683709411039166464, "text": "Me all the time https://t.co/gl1SNGDhHY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 256617738, "name": "Alexisss", "screen_name": "Sasha_f15", "lang": "en", "location": "HTX", "create_at": date("2011-02-23"), "description": "Snapchat : sasha_f15 ✨", "followers_count": 1383, "friends_count": 1400, "statues_count": 18502 }, "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-03T10:00:00.000Z"), "id": 683709411093823489, "text": "Hiking in a 33° blizzard today ��������������������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 59658374, "name": "Emily Parham", "screen_name": "empar11", "lang": "en", "location": "Portland, OR", "create_at": date("2009-07-23"), "description": "I like fun", "followers_count": 326, "friends_count": 243, "statues_count": 14808 }, "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-01-03T10:00:00.000Z"), "id": 683709411177582592, "text": "We get 5, but there is only 1 FFP for me, and it is the most wonderful @MsAbigailMac https://t.co/ItwRoFSrdD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 526114597 }}, "user": { "id": 64609690, "name": "chris", "screen_name": "NoPantsChris", "lang": "en", "location": "Sweet Home Chicago", "create_at": date("2009-08-10"), "description": "null", "followers_count": 716, "friends_count": 2044, "statues_count": 6392 }, "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-03T10:00:01.000Z"), "id": 683709411299168256, "text": "I'm about to watch 2 hours of Justin Bieber , ������ https://t.co/XHTwkp7QuD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2220711648, "name": "Gabriella Alexis", "screen_name": "alexissspena", "lang": "en", "location": "Conroe, TX", "create_at": date("2013-11-28"), "description": "null", "followers_count": 549, "friends_count": 486, "statues_count": 15406 }, "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-01-03T10:00:01.000Z"), "id": 683709411689394176, "text": "Amen �������� https://t.co/xZIKNxlW2b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 353477839, "name": "Zaddie", "screen_name": "_Padillaaa_", "lang": "en", "location": "null", "create_at": date("2011-08-11"), "description": "- Ig - Zpadillaaa. ❄️", "followers_count": 480, "friends_count": 693, "statues_count": 16868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cranston, RI", "id": "4015afc69a8a8506", "name": "Cranston", "place_type": "city", "bounding_box": rectangle("-71.550463,41.730422 -71.377473,41.807454") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4419180, "cityName": "Cranston" } }
+{ "create_at": datetime("2016-01-03T10:00:01.000Z"), "id": 683709411764879360, "text": "streets done said I went Hollywood �� bitch I'm OG Bobby Johnson good", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 614851692, "name": "$OLO DOLO.", "screen_name": "SanaeDestiniee", "lang": "en", "location": "killadelphia..", "create_at": date("2012-06-21"), "description": "on my own..... RIP GEE ❤ #7️⃣1️⃣7️⃣", "followers_count": 1724, "friends_count": 615, "statues_count": 47452 }, "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-03T10:00:01.000Z"), "id": 683709411882328064, "text": "@kingbradxx hahaha https://t.co/PMjq9Fiqvn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 4685290591, "name": "Doug Vorhees", "screen_name": "DougVorhees", "lang": "en", "location": "null", "create_at": date("2015-12-31"), "description": "null", "followers_count": 16, "friends_count": 81, "statues_count": 11 }, "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-01-03T10:00:01.000Z"), "id": 683709411940945921, "text": "Just saw Raiders fans stranded on the highway lol #ChiefsNation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ChiefsNation" }}, "user": { "id": 338034865, "name": "Scottie2Hottie", "screen_name": "Cliff_Scottt", "lang": "en", "location": "KCMO/Maryville", "create_at": date("2011-07-18"), "description": "Sprinter for #NWMSUniversity, #AllStateTrackStar, #TeamNIKE, #TrackNation, #MizzouNation, #CHIEFSKINGDOM, #GrandviewAlum", "followers_count": 1056, "friends_count": 1233, "statues_count": 19235 }, "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-03T10:00:01.000Z"), "id": 683709412100292608, "text": "4 https://t.co/ZHP3KIAiKi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 439679632, "name": "Benji", "screen_name": "CottonJordan", "lang": "en", "location": "null", "create_at": date("2011-12-17"), "description": "blah blah blah blah....Big boy 18", "followers_count": 497, "friends_count": 590, "statues_count": 11675 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jennings, MO", "id": "0a58f44e2d264da4", "name": "Jennings", "place_type": "city", "bounding_box": rectangle("-90.292312,38.70187 -90.239333,38.741342") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2937178, "cityName": "Jennings" } }
+{ "create_at": datetime("2016-01-03T10:00:01.000Z"), "id": 683709412591185920, "text": "And it brings tears to my eyes ��☺️��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 51489305, "name": "❤️кιиgg✨иιиαα ❣", "screen_name": "keyys_0pendoors", "lang": "en", "location": "in yo granddaddy pockets ! ", "create_at": date("2009-06-27"), "description": "[ Proud Mommy of 2wo ] _princessKYMIYAH;kingKYEI_ | ❤️ Team : LESBIAN ™ | •Queen*i*Am• SnapChat : ninaa_dinero", "followers_count": 1365, "friends_count": 1425, "statues_count": 45148 }, "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-03T10:00:01.000Z"), "id": 683709412821889024, "text": "RT https://t.co/awsXGTk9yU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1656106550, "name": "Kånåri", "screen_name": "KanariLJ", "lang": "en", "location": "null", "create_at": date("2013-08-08"), "description": "⠀⠀⠀⠀⠀@tyson11_ ❤️|SC: kanari_littlej", "followers_count": 540, "friends_count": 246, "statues_count": 6983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Landrum, SC", "id": "faec461ddef27c06", "name": "Landrum", "place_type": "city", "bounding_box": rectangle("-82.208035,35.156448 -82.160388,35.18536") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45083, "countyName": "Spartanburg", "cityID": 4540075, "cityName": "Landrum" } }
+{ "create_at": datetime("2016-01-03T10:00:01.000Z"), "id": 683709413106958336, "text": "Taeyeon \n#TeamSNSD https://t.co/oOmuWVc7Ca", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "TeamSNSD" }}, "user": { "id": 2193588649, "name": "♎ImSorryILikeKpop", "screen_name": "Shayla_Bear15", "lang": "en", "location": "my room", "create_at": date("2013-11-13"), "description": "I have an unhealthy obsession with Yura of Girls Day, ASMR completes me", "followers_count": 275, "friends_count": 270, "statues_count": 7254 }, "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-03T10:00:01.000Z"), "id": 683709413144850432, "text": "01/03@13:00 - Temp 49.4F, WC 46.8F. Wind 6.3mph WSW, Gust 15.0mph. Bar 29.858in, Falling. Rain 0.22in. Hum 42%. UV 1.8. SolarRad 454.", "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": 46002 }, "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-03T10:00:01.000Z"), "id": 683709413308289024, "text": "SNOW!!! View from Dad and Mom's in Portland (Milwaukie) @ Willamette View https://t.co/fUnxFIaAvS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.6450424,45.4300385"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 13145552, "name": "Dave Weidlich", "screen_name": "revdavew", "lang": "en", "location": "Petaluma, CA", "create_at": date("2008-02-05"), "description": "Bi-Vocational Business Owner & Pastor of the Vine Church. Enjoy mountain biking & naps.", "followers_count": 204, "friends_count": 295, "statues_count": 1418 }, "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-03T10:00:01.000Z"), "id": 683709413639589889, "text": "Can you recommend anyone for this #IT #job? https://t.co/P3qkNNQsLX #NettempsJobs #Cincinnati, OH #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.5363226,39.1066354"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "IT", "job", "NettempsJobs", "Cincinnati", "Hiring", "CareerArc" }}, "user": { "id": 21964064, "name": "TMJ-CIN IT Jobs", "screen_name": "tmj_cin_it", "lang": "en", "location": "Cincinnati, OH", "create_at": date("2009-02-25"), "description": "Follow this account for geo-targeted Software Dev. - General/IT job tweets in Cincinnati, OH. Need help? Tweet us at @CareerArc!", "followers_count": 474, "friends_count": 325, "statues_count": 192 }, "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-03T10:00:01.000Z"), "id": 683709413702537216, "text": "Had to leave these hoes alone to get my mind right", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2419494114, "name": "Dezz Loaf", "screen_name": "thatcuteshychic", "lang": "en", "location": "Texas, USA", "create_at": date("2014-03-30"), "description": "feel like I'm in too deep #TJC18", "followers_count": 2793, "friends_count": 1943, "statues_count": 29326 }, "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-03T10:00:01.000Z"), "id": 683709414063259648, "text": "yikes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1494454478, "name": "Stephanie Pearson", "screen_name": "stephpearson27", "lang": "en", "location": "zach cissell's crib", "create_at": date("2013-06-08"), "description": "null", "followers_count": 469, "friends_count": 281, "statues_count": 463 }, "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-03T10:00:01.000Z"), "id": 683709414117928960, "text": "Movie time! #daydate - Drinking a Snowdrift Vanilla Porter by @Leinenkugels at @marcus_theatres — https://t.co/gdWyKVxYBI #photo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.6289,40.816"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "daydate", "photo" }}, "user_mentions": {{ 2281112444, 27805320 }}, "user": { "id": 146679382, "name": "Daine Patton", "screen_name": "DaineP", "lang": "en", "location": "Lincoln, Ne", "create_at": date("2010-05-21"), "description": "Son Brother Father Husband Husker Fantasy Football Legend and probably a person your friends and family warn you about", "followers_count": 117, "friends_count": 262, "statues_count": 4314 }, "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-03T10:00:01.000Z"), "id": 683709414486839296, "text": "@BlastinKulurz The metaphilosophy is only 99 cents today, please share. #metaphysics #indigo #meditation https://t.co/fq3QqWh336", "in_reply_to_status": 682255337165074432, "in_reply_to_user": 3950524515, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "metaphysics", "indigo", "meditation" }}, "user_mentions": {{ 3950524515 }}, "user": { "id": 3420825929, "name": "Ethan Indigo", "screen_name": "EthanIndigo", "lang": "en", "location": "USA, NorCal", "create_at": date("2015-08-13"), "description": "Writer/philosopher. Ethan Penned Geometry of Energy, 108 Steps To Be In The Zone, The Matrix of Four, The Terraist Letters, Complete Patriot's Guide, and more.", "followers_count": 484, "friends_count": 819, "statues_count": 3424 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orangevale, CA", "id": "d950fc7dc2cd14f9", "name": "Orangevale", "place_type": "city", "bounding_box": rectangle("-121.262486,38.637436 -121.187521,38.718208") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 654092, "cityName": "Orangevale" } }
+{ "create_at": datetime("2016-01-03T10:00:01.000Z"), "id": 683709414616858624, "text": "OK ...it GAME TIME..\nColts vs Titans.. Can we win? doubt it, but I turned on my lucky COLTS SNOWMAN anyways! Lol�� https://t.co/mscfLxC0iB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 829734386, "name": "Julie Ann Pequignot", "screen_name": "Julie9320", "lang": "en", "location": "Fort Wayne, IN", "create_at": date("2012-09-17"), "description": "True friends are like diamonds, precious and rare....false friends are like leaves, found everywhere.", "followers_count": 215, "friends_count": 49, "statues_count": 10 }, "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-03T10:00:01.000Z"), "id": 683709414969315329, "text": "Wind 4.0 mph NW. Barometer 30.235 in, Rising. Temperature 34.5 °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": 110058 }, "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-03T10:00:02.000Z"), "id": 683709415850115072, "text": "Only way to get rich is too act broke", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 260935045, "name": "PR❣NCESS", "screen_name": "_TiRAW", "lang": "en", "location": "Ethiopia ", "create_at": date("2011-03-04"), "description": "null", "followers_count": 1834, "friends_count": 672, "statues_count": 122419 }, "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-01-03T10:00:02.000Z"), "id": 683709415883718657, "text": "I need to go to the mall asapppp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2562033459, "name": "EℓℓaKayy", "screen_name": "Ellakayyy", "lang": "en", "location": "413 livin ", "create_at": date("2014-05-24"), "description": "♱ Be so good that they cant ignore you ♱", "followers_count": 675, "friends_count": 353, "statues_count": 11227 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Watertown, NY", "id": "d4930fd3a1388ef1", "name": "Watertown", "place_type": "city", "bounding_box": rectangle("-75.958806,43.917677 -75.855647,44.014324") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36045, "countyName": "Jefferson", "cityID": 3678608, "cityName": "Watertown" } }
+{ "create_at": datetime("2016-01-03T10:00:02.000Z"), "id": 683709415892062208, "text": "Work (at @MarylandLive! Casino in Hanover, MD) https://t.co/JFpv34mqrV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.72626257,39.15715317"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 402980192 }}, "user": { "id": 33292000, "name": "Marcus Noble", "screen_name": "marcusnoble", "lang": "en", "location": "Bowie, MD", "create_at": date("2009-04-19"), "description": "the hockey, MLB, WWE, and poker lovin' anthropomorphic skunk. NSFW account is @NobleNSFW", "followers_count": 347, "friends_count": 558, "statues_count": 32260 }, "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-03T10:00:02.000Z"), "id": 683709416252784642, "text": "Wind 4.0 mph NW. Barometer 30.098 in, Falling slowly. Temperature 39.6 °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": 40, "friends_count": 25, "statues_count": 16602 }, "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-03T10:00:02.000Z"), "id": 683709416785313792, "text": "Can you recommend anyone for this #job? Meat Cutter - Apprentice - https://t.co/LYl2PdTGU7 #Pittsfield, Massachusetts #Hospitality #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.2453824,42.4500845"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Pittsfield", "Hospitality", "Hiring" }}, "user": { "id": 88182028, "name": "TMJ-MA HRTA Jobs", "screen_name": "tmj_ma_hrta", "lang": "en", "location": "Massachusetts", "create_at": date("2009-11-07"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Massachusetts Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 370, "friends_count": 291, "statues_count": 378 }, "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-03T10:00:02.000Z"), "id": 683709416886108160, "text": "@nyjets The Game has officially started. I'm praying that we win with no injuries. JETS JETS JETS. #JetsvsBills #NewYorkJets #football #NFL", "in_reply_to_status": -1, "in_reply_to_user": 17076218, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "JetsvsBills", "NewYorkJets", "football", "NFL" }}, "user_mentions": {{ 17076218 }}, "user": { "id": 312871680, "name": "adam baehr", "screen_name": "adambaehr", "lang": "en", "location": "new jersey", "create_at": date("2011-06-07"), "description": "I love games!!!!! I also love Syfy and television shows Galore lmao. I also enjoy the outdoors and sports. I'm Obsessed with everything Horror!!!!!", "followers_count": 159, "friends_count": 587, "statues_count": 2378 }, "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-03T10:00:02.000Z"), "id": 683709417028763648, "text": "Wind 5.6 mph WNW. Barometer 30.00 in, Falling. Temperature 47.3 °F. Rain today 0.00 in. Humidity 43%", "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": 306038 }, "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-03T10:00:02.000Z"), "id": 683709417058091008, "text": "You knew this stuff was going to start pouring out https://t.co/AKhqf9ZuZD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25227273, "name": "Nick Pasacreta", "screen_name": "npasacreta", "lang": "en", "location": "Newtown, CT", "create_at": date("2009-03-18"), "description": "Twitter? I just met her! Production Assistant at MLB Advanced Media. Quinnipiac alum. Water connoisseur.", "followers_count": 222, "friends_count": 1744, "statues_count": 4837 }, "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-03T10:00:02.000Z"), "id": 683709417267671040, "text": "https://t.co/YvmaBuuZXZ porn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 490927050, "name": "-______________-", "screen_name": "joshuadunigan", "lang": "en", "location": "null", "create_at": date("2012-02-12"), "description": "Look at my beautiful girl friend ^", "followers_count": 143, "friends_count": 134, "statues_count": 18894 }, "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-03T10:00:02.000Z"), "id": 683709417267818497, "text": "Damn. Went on IG and this girl literally has 78 days left to live and all she wants is her bf back ...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3251086300, "name": "losiram", "screen_name": "gymsocks191", "lang": "en", "location": "null", "create_at": date("2015-05-13"), "description": "011914 JR;*", "followers_count": 96, "friends_count": 71, "statues_count": 5661 }, "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-03T10:00:02.000Z"), "id": 683709417553027073, "text": "Got to go to work in 2 hrs! HELP!!!!! YUCK!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.863892,34.22419"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1120257332, "name": "Chandler", "screen_name": "cmlfpu", "lang": "en", "location": "Georgia", "create_at": date("2013-01-25"), "description": "cmlfpu@yahoo.com", "followers_count": 719, "friends_count": 2085, "statues_count": 21364 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Krystal", "id": "07d9c948e1080000", "name": "Krystal", "place_type": "poi", "bounding_box": rectangle("-83.8638921,34.2241899 -83.863892,34.22419") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13139, "countyName": "Hall", "cityID": 1357260, "cityName": "Oakwood" } }
+{ "create_at": datetime("2016-01-03T10:00:02.000Z"), "id": 683709417628372993, "text": "@2kewlabs @RosieRoseLA @Skat_Cat @Shananay_G Hi Pippa!! You so purrty, nice to meet you!! ��XOX", "in_reply_to_status": 683458083389980673, "in_reply_to_user": 232844798, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 232844798, 1611949034, 346497732, 568719800 }}, "user": { "id": 1676941554, "name": "RoxyRoxLA", "screen_name": "RoxyRoxLA", "lang": "en", "location": "California", "create_at": date("2013-08-16"), "description": "I'm a striped tabby...street urchin, love muffin, one smart cookie. I know what I want and I know how to get it. I live in LA and rule the household. #ZSHQ", "followers_count": 6651, "friends_count": 4625, "statues_count": 51043 }, "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-03T10:00:02.000Z"), "id": 683709417703903232, "text": "49.2F (Feels: 49.2F) - Humidity: 62% - Wind: 6.9mph NW - Gust: 9.2mph - 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": 220853 }, "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-03T10:00:02.000Z"), "id": 683709417834020865, "text": "This is how January 3rd looks like���� @NikeRunning @NikeTraining #getouthere #nikechiberia @NikeNYC #the48 https://t.co/4jHa9QF5jq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "getouthere", "nikechiberia", "the48" }}, "user_mentions": {{ 337726224, 610450858, 444913649 }}, "user": { "id": 1045839800, "name": "eve may", "screen_name": "JeniaMay", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2012-12-29"), "description": "Live to run.. Nah, run to live ! #THE48", "followers_count": 108, "friends_count": 165, "statues_count": 367 }, "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-03T10:00:02.000Z"), "id": 683709417880064000, "text": "@Muzayn so true you should write a book", "in_reply_to_status": 683708634900004864, "in_reply_to_user": 551565300, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 551565300 }}, "user": { "id": 635726349, "name": "MçMüffįñ", "screen_name": "kennarik", "lang": "en", "location": "Cedar Rapids, IA", "create_at": date("2012-07-14"), "description": "Ayyyyyyyyyyy God bless", "followers_count": 497, "friends_count": 380, "statues_count": 5961 }, "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-01-03T10:00:02.000Z"), "id": 683709418161098752, "text": "Either my hair is done nice or my makeup, it never seems to be both", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1918156262, "name": "Amanda", "screen_name": "ammanndahh", "lang": "en", "location": "Chicago, IL", "create_at": date("2013-09-29"), "description": "I'm basically Cher Horowitz", "followers_count": 218, "friends_count": 180, "statues_count": 1898 }, "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-03T10:00:02.000Z"), "id": 683709418223972352, "text": "My @LodgeCastIron collection is making this low carb diet seem easy. Steak, steak, steak, chicken, steak, eggs, steak. . .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 35782573 }}, "user": { "id": 389016319, "name": "Jeff Denney", "screen_name": "grubber87", "lang": "en", "location": "at Auntie Mae's", "create_at": date("2011-10-11"), "description": "Daddy's water burns.", "followers_count": 283, "friends_count": 493, "statues_count": 1569 }, "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-03T10:00:02.000Z"), "id": 683709418224013313, "text": "#FelizDomingo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "FelizDomingo" }}, "user": { "id": 76183147, "name": "Onesimo A.De Gante®", "screen_name": "degante_rayos16", "lang": "en", "location": "Mexico City, Los Angeles Ca.", "create_at": date("2009-09-21"), "description": "The Official Twitter Account Page ➡️Vivir la vida a lo que da!⬅️ http://www.instagram.com/degante_rayos16", "followers_count": 282, "friends_count": 714, "statues_count": 23781 }, "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-03T10:00:02.000Z"), "id": 683709418320470016, "text": "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": 3244099173, "name": "ahunnitahunnit!", "screen_name": "MojOneHunnid", "lang": "en", "location": "null", "create_at": date("2015-05-09"), "description": "rest up shayla. 19 . Chicago.", "followers_count": 551, "friends_count": 337, "statues_count": 34089 }, "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-03T10:00:02.000Z"), "id": 683709418475753472, "text": "Temp: 49.0°F Wind:0.4mph Pressure: 30.197hpa 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": 55753 }, "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-03T10:00:02.000Z"), "id": 683709418488360960, "text": "@BernieSanders fix the @Browns first please. We need help:(", "in_reply_to_status": -1, "in_reply_to_user": 216776631, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 216776631, 40358743 }}, "user": { "id": 42314331, "name": "Scotty Michael", "screen_name": "HelloImScotty23", "lang": "en", "location": "Cleveland/Columbus ", "create_at": date("2009-05-24"), "description": "Hakuna Matata\n\n\n⚪THE Ohio State University '17⚪", "followers_count": 508, "friends_count": 504, "statues_count": 25916 }, "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-01-03T10:00:02.000Z"), "id": 683709418655973376, "text": "#Honolulu, Hawaii #Labor #Job: New Store Opening - Cleaning Support at Nordstrom https://t.co/0wzOjt0hRI #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Honolulu", "Labor", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22777294, "name": "TMJ-HON Labor Jobs", "screen_name": "tmj_hon_labor", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-04"), "description": "Follow this account for geo-targeted General Labor job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 298, "friends_count": 291, "statues_count": 2 }, "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-03T10:00:02.000Z"), "id": 683709418849091589, "text": "Transformation and Execution ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 227600522, "name": "кαуѕнαиαє", "screen_name": "_KekeShanae", "lang": "en", "location": "null", "create_at": date("2010-12-16"), "description": "null", "followers_count": 121, "friends_count": 101, "statues_count": 6747 }, "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-03T10:00:02.000Z"), "id": 683709418899402752, "text": "1️⃣��- I hate u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1303280563, "name": "kodak Miah", "screen_name": "_knicolem_", "lang": "en", "location": "Highland City, FL", "create_at": date("2013-03-26"), "description": "$enior16 . Famu20.", "followers_count": 3436, "friends_count": 3537, "statues_count": 30727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland City, FL", "id": "00b90fe055d9951c", "name": "Highland City", "place_type": "city", "bounding_box": rectangle("-81.907237,27.937965 -81.850951,27.976255") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12105, "countyName": "Polk", "cityID": 1230225, "cityName": "Highland City" } }
+{ "create_at": datetime("2016-01-03T10:00:02.000Z"), "id": 683709418928758784, "text": "A good laugh to end the break, @BookofMormon at @DPAC https://t.co/8Td0iR4EAZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 132646498, 14718975 }}, "user": { "id": 112491081, "name": "Matthew Caggia", "screen_name": "caggiasocstud", "lang": "en", "location": "null", "create_at": date("2010-02-08"), "description": "Droppin' knowledge on a daily, I smite the ignorant with my Mighty Hammer of Knowledge! #pickupthepickle", "followers_count": 134, "friends_count": 44, "statues_count": 2345 }, "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-03T10:00:02.000Z"), "id": 683709418970689536, "text": "It was 2am and I was a little drunk and his room was so warm and cozy and he was good at cuddling ����������", "in_reply_to_status": 683702993095622656, "in_reply_to_user": 59615041, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 59615041, "name": "Tofu Hoe", "screen_name": "Feministfists", "lang": "en", "location": "she/her, cis, queer, chubby", "create_at": date("2009-07-23"), "description": "Tofu is my life source. General TW for content. Mixed bae. Award-winning activist. Do no harm, take no shit. Abortion Doula. #BlackVegansRock Snapchat: TofuHoe", "followers_count": 3460, "friends_count": 1260, "statues_count": 60792 }, "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-03T10:00:02.000Z"), "id": 683709418987323392, "text": "We're #hiring! Click to apply: Barista (US) - https://t.co/05wEZIsbgp #Hospitality #Veterans #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.7473035,30.4021802"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Hospitality", "Veterans", "Job", "Jobs", "CareerArc" }}, "user": { "id": 22486616, "name": "TMJ-AUS HRTA Jobs", "screen_name": "tmj_aus_hrta", "lang": "en", "location": "Austin, TX", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Austin, TX. Need help? Tweet us at @CareerArc!", "followers_count": 433, "friends_count": 294, "statues_count": 731 }, "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-03T10:00:03.000Z"), "id": 683709419696320512, "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": 320440400, "name": "Courtney Pangallo ❤️", "screen_name": "courpang", "lang": "en", "location": "IG/snapchat: courpang", "create_at": date("2011-06-19"), "description": "null", "followers_count": 896, "friends_count": 291, "statues_count": 9140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newport, KY", "id": "25e7ede60b304d89", "name": "Newport", "place_type": "city", "bounding_box": rectangle("-84.504476,39.064988 -84.4611,39.102427") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21037, "countyName": "Campbell", "cityID": 2155884, "cityName": "Newport" } }
+{ "create_at": datetime("2016-01-03T10:00:03.000Z"), "id": 683709420010778625, "text": "Want to work at King's Creek Plantation? We're #hiring in #Williamsburg, VA! Click for details: https://t.co/VQZvreDRS0 #Construction #va", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.6350285,37.2668356"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Williamsburg", "Construction", "va" }}, "user": { "id": 3015648286, "name": "King's Creek Jobs", "screen_name": "KingsCreekJobs", "lang": "en", "location": "null", "create_at": date("2015-02-03"), "description": "Check here for current job opportunities with King's Creek Plantation, in beautiful Williamsburg, VA.", "followers_count": 21, "friends_count": 46, "statues_count": 5 }, "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": 51199, "countyName": "York" } }
+{ "create_at": datetime("2016-01-03T10:00:03.000Z"), "id": 683709420077842433, "text": "Made 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": 3993479958, "name": "BreSmoove✨", "screen_name": "bregotthajuice1", "lang": "en", "location": "Cedar Hill, TX", "create_at": date("2015-10-23"), "description": "Hacked at 2K so FOLLOW ME!", "followers_count": 746, "friends_count": 617, "statues_count": 4023 }, "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-03T10:00:03.000Z"), "id": 683709420140892160, "text": "Anyone selling a Xbox 360 ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2232175965, "name": "ralph lauren", "screen_name": "tim_whitty", "lang": "en", "location": "null", "create_at": date("2013-12-18"), "description": "Saint Leo University, '19", "followers_count": 570, "friends_count": 597, "statues_count": 6931 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Salerno, FL", "id": "97e2346184350535", "name": "Port Salerno", "place_type": "city", "bounding_box": rectangle("-80.237333,27.111576 -80.151866,27.172504") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12085, "countyName": "Martin", "cityID": 1258727, "cityName": "Port Salerno" } }
+{ "create_at": datetime("2016-01-03T10:00:03.000Z"), "id": 683709420237385728, "text": "I'm at Harvest - A Church of New Beginnings in Oswego, IL https://t.co/kxIueTET2n", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.30881291,41.64544637"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25393325, "name": "Cortney Kinzler", "screen_name": "_BlackAngel", "lang": "en", "location": "Chicago", "create_at": date("2009-03-19"), "description": "Cortney Black Kinzler - Christian. Mom. Wife. \nOwner of @BlackAngelSauce & @AGrandPlan4U. Co-founder of @ForeverKidsFDN.", "followers_count": 265, "friends_count": 573, "statues_count": 1083 }, "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": 17093, "countyName": "Kendall", "cityID": 1756887, "cityName": "Oswego" } }
+{ "create_at": datetime("2016-01-03T10:00:03.000Z"), "id": 683709420321153024, "text": "@jacobsartorius how old are you?", "in_reply_to_status": -1, "in_reply_to_user": 3579012568, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3579012568 }}, "user": { "id": 2790514979, "name": "sofia is happy af", "screen_name": "idgafConnors", "lang": "en", "location": "Arkansas || 3/5Quad", "create_at": date("2014-09-28"), "description": "trc:)", "followers_count": 1093, "friends_count": 312, "statues_count": 26582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Farmington, AR", "id": "da25141980d997de", "name": "Farmington", "place_type": "city", "bounding_box": rectangle("-94.285526,36.000016 -94.222787,36.05422") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5143, "countyName": "Washington", "cityID": 523170, "cityName": "Farmington" } }
+{ "create_at": datetime("2016-01-03T10:00:03.000Z"), "id": 683709420560363521, "text": "Archers girls #fender #bass #yingyang #rosewood #precisionbass #dimarzio https://t.co/eMaQE5HVA0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "fender", "bass", "yingyang", "rosewood", "precisionbass", "dimarzio" }}, "user": { "id": 2959785148, "name": "Calven", "screen_name": "calvenband", "lang": "en", "location": "Akron, OH", "create_at": date("2015-01-03"), "description": "SOUNDS LIKE: @ChevelleInc, @Deftones, @Breakingbenj, @ToolMusic, @10years, with a 90's influence!", "followers_count": 1095, "friends_count": 1208, "statues_count": 228 }, "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-01-03T10:00:03.000Z"), "id": 683709420644253696, "text": "LETS GOOOOOOOO @nyjets", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17076218 }}, "user": { "id": 382813706, "name": "Just Gareld.", "screen_name": "Chaoticisms", "lang": "en", "location": "null", "create_at": date("2011-09-30"), "description": "I ain't here to save the fucking children", "followers_count": 262, "friends_count": 269, "statues_count": 10732 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Farmingdale, NY", "id": "c55c9fed765e35a1", "name": "South Farmingdale", "place_type": "city", "bounding_box": rectangle("-73.468517,40.704497 -73.429787,40.730502") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3669001, "cityName": "South Farmingdale" } }
+{ "create_at": datetime("2016-01-03T10:00:03.000Z"), "id": 683709420702838784, "text": "KH6EF Passing Ko Olina, Hawaii, Kapolei, HI https://t.co/P5PsuSEQJV #APRS 13b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-158.1205,21.34467"), "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "APRS" }}, "user": { "id": 13608552, "name": "Bryan", "screen_name": "KH6EF", "lang": "en", "location": "Hawaii, USA", "create_at": date("2008-02-17"), "description": "Radio and technology enthusiast. Technical Specialist for Walt Disney Parks and Resorts. Tweets are my own and do not reflect the opinion of The Walt Disney Co.", "followers_count": 305, "friends_count": 848, "statues_count": 1951 }, "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-01-03T10:00:03.000Z"), "id": 683709421264842753, "text": "@stiIlxborn hey happy birthday luke stay rad", "in_reply_to_status": -1, "in_reply_to_user": 840417438, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 840417438 }}, "user": { "id": 2992624141, "name": "alexa :(", "screen_name": "mdrnbsebal", "lang": "en-GB", "location": "pnw", "create_at": date("2015-01-22"), "description": "a walking vagina w feelings", "followers_count": 1354, "friends_count": 274, "statues_count": 8270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hood River, OR", "id": "ba6d1507b88d9604", "name": "Hood River", "place_type": "city", "bounding_box": rectangle("-121.557201,45.692953 -121.491657,45.717307") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41027, "countyName": "Hood River", "cityID": 4134900, "cityName": "Hood River" } }
+{ "create_at": datetime("2016-01-03T10:00:03.000Z"), "id": 683709421277478913, "text": "Yup these morons are literally forcing me to want to vote for 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": 263642964, "name": "Joseph", "screen_name": "thomaschaske93", "lang": "en", "location": "Milwaukee, WI", "create_at": date("2011-03-10"), "description": "Software developer at Penrod Software. The only reason I have a Twitter is to troll @andrewbudziszek", "followers_count": 325, "friends_count": 293, "statues_count": 3564 }, "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-03T10:00:03.000Z"), "id": 683709421315338241, "text": "West Virginia getting two dubs yesterday��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 622428923, "name": "Scratchin' Killah", "screen_name": "KingMichaelV", "lang": "en", "location": "null", "create_at": date("2012-06-29"), "description": "chill out", "followers_count": 463, "friends_count": 212, "statues_count": 4501 }, "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-01-03T10:00:03.000Z"), "id": 683709421562638336, "text": "Fuck little boys, I want 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": 1858778556, "name": "HXRMONI", "screen_name": "harmoni_garbade", "lang": "en", "location": "843 ", "create_at": date("2013-09-12"), "description": "Going through a crisis at 17\n\nIG》@ harms2", "followers_count": 445, "friends_count": 316, "statues_count": 3608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden City, SC", "id": "596fb7b1e3f22a38", "name": "Garden City", "place_type": "city", "bounding_box": rectangle("-79.069664,33.535979 -78.972953,33.637127") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4528455, "cityName": "Garden City" } }
+{ "create_at": datetime("2016-01-03T10:00:03.000Z"), "id": 683709421692796930, "text": "Wind 1.6 mph WSW. Barometer 29.73 in, Falling. Temperature 38.5 °F. Rain today 0.00 in. Humidity 69%", "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": 156394 }, "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-03T10:00:03.000Z"), "id": 683709422024192000, "text": "Awesome veggie burger! Nice surprise or I'm just that hungry lol. (@ Smashburger - @hmshost in Salt Lake City, UT) https://t.co/OBZxpr1WXh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.98486567,40.78816136"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 29050885 }}, "user": { "id": 139845037, "name": "Keita Hall", "screen_name": "keitahall", "lang": "en", "location": "NC", "create_at": date("2010-05-03"), "description": "Sexy, intelligent, unique woman & that's just the way I like it...Full time Promotions/Manager/Actress/Model. Overall a business woman Instagram-Keitahall", "followers_count": 451, "friends_count": 594, "statues_count": 8158 }, "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-03T10:00:03.000Z"), "id": 683709422032531457, "text": "You only fail when you stop trying....... #sundayfunday #Sundayze @ Snapchat: Teamabraham https://t.co/QjhRZwcoPj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-66.1803,18.3797"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sundayfunday", "Sundayze" }}, "user": { "id": 37054155, "name": "TEAM ABRAHAM", "screen_name": "ambientalshirts", "lang": "en", "location": "Bayamon P.R.", "create_at": date("2009-05-01"), "description": "«BE FRESH»", "followers_count": 72, "friends_count": 134, "statues_count": 1265 }, "place": { "country": "Puerto Rico", "country_code": "Puerto Rico", "full_name": "Bayamón, Puerto Rico", "id": "ac6fea0297ea027f", "name": "Bayamón", "place_type": "city", "bounding_box": rectangle("-66.200774,18.328316 -66.11836,18.430998") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72021, "countyName": "Bayam?n", "cityID": 7206593, "cityName": "Bayamn" } }
+{ "create_at": datetime("2016-01-03T10:00:03.000Z"), "id": 683709422070140934, "text": "Austin just called me mom�� I think 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": 380110237, "name": "B", "screen_name": "BROOKEMATHER", "lang": "en", "location": "☀️Bakersfield California☀", "create_at": date("2011-09-25"), "description": "| @WelteAustin♡ |", "followers_count": 2562, "friends_count": 2486, "statues_count": 76789 }, "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-03T10:00:03.000Z"), "id": 683709422129012737, "text": "First indoor Tri #committotri #newyear #2016 #garminfitness #garmin920xt #exhausted @ Life Time… https://t.co/7r0egNA71f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.05652887,42.04162503"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "committotri", "newyear", "garminfitness", "garmin920xt", "exhausted" }}, "user": { "id": 32519462, "name": "Jessica Taylor", "screen_name": "JessMelindaTay", "lang": "en", "location": "South Loop Chicago, IL", "create_at": date("2009-04-17"), "description": "Travel is fatal to prejudice, bigotry, and narrow-mindedness, and many of our people need it sorely on these accounts. -Mark Twain", "followers_count": 581, "friends_count": 1873, "statues_count": 5055 }, "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-03T10:00:03.000Z"), "id": 683709422200303616, "text": "I mean look how precious https://t.co/PhDYwKKFFT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25847721, "name": "cas", "screen_name": "hymengoddess", "lang": "en", "location": "Massachusetts, USA", "create_at": date("2009-03-22"), "description": "null", "followers_count": 160, "friends_count": 88, "statues_count": 13325 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Athol, MA", "id": "c3999f07c0fc284d", "name": "Athol", "place_type": "city", "bounding_box": rectangle("-72.282481,42.564444 -72.192035,42.616557") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25027, "countyName": "Worcester", "cityID": 2502515, "cityName": "Athol" } }
+{ "create_at": datetime("2016-01-03T10:00:03.000Z"), "id": 683709422221176832, "text": "@ShawtyyRe with an occasional bud light lime or so", "in_reply_to_status": 683709015780651009, "in_reply_to_user": 263841925, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 263841925 }}, "user": { "id": 516011935, "name": "haley.", "screen_name": "H_Hanselmanx3", "lang": "en", "location": "Miamisburg Ohio", "create_at": date("2012-03-05"), "description": "|| TWENTY1 || || 937 || || MOMMY ||", "followers_count": 499, "friends_count": 533, "statues_count": 12969 }, "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-01-03T10:00:03.000Z"), "id": 683709422502305792, "text": "Watching dreamgirls ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 735093703, "name": "she poppin'", "screen_name": "WhatDAENeed_", "lang": "en", "location": "w| Envision", "create_at": date("2012-08-03"), "description": "You wanna glo up, baby Dae got you.✨", "followers_count": 1877, "friends_count": 1498, "statues_count": 56972 }, "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-03T10:00:03.000Z"), "id": 683709422518992897, "text": "Sunny this afternoon, high 46 (8 C). Low 23 (-5 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15953346, "name": "Amarillo Weather", "screen_name": "AmarilloWeather", "lang": "en", "location": "Amarillo, Texas", "create_at": date("2008-08-22"), "description": "Forecasts and advisories for Amarillo, Texas", "followers_count": 4042, "friends_count": 395, "statues_count": 8705 }, "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-01-03T10:00:03.000Z"), "id": 683709422653300736, "text": "My nose looks good today ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26174402, "name": "B$", "screen_name": "Half_Ah_Virgin", "lang": "en", "location": "P H I L L Y ", "create_at": date("2009-03-23"), "description": "My DMs are open Mon-Fri 11am-11pm and I charge $20 for a follow back!", "followers_count": 9567, "friends_count": 968, "statues_count": 315877 }, "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-03T10:00:03.000Z"), "id": 683709422779133952, "text": "If the gov of Oregon/feds don't step in & emasculate these morons,& do it soon, I'm gonna have a hard time believing anymore tough on 1/2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 129202560, "name": "Aku's bane", "screen_name": "MADattheworld2", "lang": "en", "location": "Free Range Human", "create_at": date("2010-04-03"), "description": "...and we're back.", "followers_count": 151, "friends_count": 205, "statues_count": 12114 }, "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-01-03T10:00:03.000Z"), "id": 683709422804320256, "text": "I'm really goin to buy a house on Broadway and freak 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": 158069651, "name": "❄️WorLd", "screen_name": "King_B_E_E", "lang": "en", "location": "null", "create_at": date("2010-06-21"), "description": "BrittBoo♒ to be continued... #ravensNATION", "followers_count": 929, "friends_count": 730, "statues_count": 30908 }, "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-03T10:00:03.000Z"), "id": 683709422816772096, "text": "This is why the bills suck dick every year https://t.co/rDHwXQO5n0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 579214508, "name": "CWright", "screen_name": "ConnorWright81", "lang": "en", "location": "Miami, FL - Dallas, Tx", "create_at": date("2012-05-13"), "description": "Accused for every crime known through the equator..", "followers_count": 269, "friends_count": 338, "statues_count": 10287 }, "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-03T10:00:03.000Z"), "id": 683709422841892864, "text": "Join the dressbarn team! See our latest #Retail #job opening here: https://t.co/Istm70GSmN #OverlandPark, KS #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.6760395,38.8630091"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "OverlandPark", "Hiring", "CareerArc" }}, "user": { "id": 100240048, "name": "TMJ-KSK Retail Jobs", "screen_name": "tmj_ksk_retail", "lang": "en", "location": "Kansas City, KS", "create_at": date("2009-12-29"), "description": "Follow this account for geo-targeted Retail job tweets in Kansas City, KS from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 313, "friends_count": 293, "statues_count": 161 }, "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-03T10:00:03.000Z"), "id": 683709423047577600, "text": "Bday Gworl �� @CcNeiko https://t.co/uDQyjSpgjy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2792348933 }}, "user": { "id": 3280743140, "name": "Jelly Vee", "screen_name": "jellycanfly", "lang": "en", "location": "Detroitisiana", "create_at": date("2015-07-15"), "description": "the internet junkie. the Leo. hard in the paint goer. fashion ethusiast. makeup insider.", "followers_count": 94, "friends_count": 69, "statues_count": 5105 }, "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-03T10:00:03.000Z"), "id": 683709423051751424, "text": "I pray 2016 is great 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": 3061023868, "name": "Mar 'Teezy☃", "screen_name": "Theofficalmar", "lang": "en", "location": "null", "create_at": date("2015-02-24"), "description": "Young Getting To The Money Rest In Paradise Grandma I Love And Miss You So Much❤️", "followers_count": 773, "friends_count": 499, "statues_count": 4533 }, "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-03T10:00:03.000Z"), "id": 683709423206936576, "text": "@_xKissME fb ��", "in_reply_to_status": -1, "in_reply_to_user": 78427706, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 78427706 }}, "user": { "id": 788494285, "name": "HumbleBEAST ☝", "screen_name": "theofficial_key", "lang": "en", "location": "Norfolk, VA", "create_at": date("2012-08-28"), "description": "Philippians 4:13\nI can do all things through him who gives me strength.☝", "followers_count": 1282, "friends_count": 1278, "statues_count": 10928 }, "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-03T10:00:03.000Z"), "id": 683709423349547008, "text": "@Knickfan_4life That's a part of coaching too.", "in_reply_to_status": 683707346351165440, "in_reply_to_user": 161974845, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 161974845 }}, "user": { "id": 363316899, "name": "M.J. Rico", "screen_name": "MichaelJRico", "lang": "en", "location": " New York, New York", "create_at": date("2011-08-27"), "description": "#Giants/Knicks/Yankees. Science teacher. Dad. Seeker of knowledge. Lover of life's simple pleasures. Tolerate no B.S. Just having fun. Ya dig?", "followers_count": 1580, "friends_count": 1414, "statues_count": 27076 }, "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-03T10:00:03.000Z"), "id": 683709423446061056, "text": "@BossManRoss_ damn so you just out here tryna give hoes rings nshit? ������", "in_reply_to_status": 683708119512317952, "in_reply_to_user": 266310501, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 266310501 }}, "user": { "id": 108196139, "name": "⭕️⛽️", "screen_name": "Matt_iAm_", "lang": "en", "location": "null", "create_at": date("2010-01-24"), "description": "9O1 #LongLiveZQ", "followers_count": 1537, "friends_count": 1936, "statues_count": 47465 }, "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-03T10:00:03.000Z"), "id": 683709423500570624, "text": "Not if you blocked https://t.co/7CaYDwh9GQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29619102, "name": "1212", "screen_name": "shanbonita", "lang": "en", "location": "in the sky", "create_at": date("2009-04-07"), "description": "lost something close to me.... R.I.P. Grandmommy Doris Thompson I love u forever ❤️ 0415", "followers_count": 40690, "friends_count": 6, "statues_count": 87564 }, "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-03T10:00:03.000Z"), "id": 683709423798325252, "text": "Just posted a photo @ Cabo Rojo Lighthouse https://t.co/gxx6M7SzMt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.19251387,17.9333821"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 12277662, "name": "Arne Heggestad", "screen_name": "ArneHeggestad", "lang": "en", "location": "Chattanooga, TN", "create_at": date("2008-01-15"), "description": "Died Tragically Rescuing His Family from the Remains of a Destroyed Sinking Battleship", "followers_count": 251, "friends_count": 191, "statues_count": 17483 }, "place": { "country": "Puerto Rico", "country_code": "Puerto Rico", "full_name": "Llanos Costa, Puerto Rico", "id": "97c4d94c590718b3", "name": "Llanos Costa", "place_type": "city", "bounding_box": rectangle("-67.200812,17.930892 -67.102278,18.029243") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-01-03T10:00:04.000Z"), "id": 683709424083582976, "text": "Next up....\nMeet us there #J5 #Divine9 #NUPES #clt #cltnightlife @ Sports ONE CLT https://t.co/OzNjoNIiJo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.8372879,35.2300415"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "J5", "Divine9", "NUPES", "clt", "cltnightlife" }}, "user": { "id": 473923117, "name": "Antwoin Bryant", "screen_name": "TwoinB1911", "lang": "en", "location": "Charlotte, NC", "create_at": date("2012-01-25"), "description": "Moncks Corner's Finest", "followers_count": 224, "friends_count": 676, "statues_count": 1752 }, "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-03T10:00:04.000Z"), "id": 683709424121155584, "text": "@KaylinEvergreen I was busy and feeling sick!!", "in_reply_to_status": 683709318739431424, "in_reply_to_user": 1369528592, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1369528592 }}, "user": { "id": 1115343726, "name": "Tye", "screen_name": "TyeHusky", "lang": "en", "location": "Whatever, USA Catalina Island", "create_at": date("2013-01-23"), "description": "Yo listen up, here's a story about a lil husky that lives in a blue world #CrankThatCurl", "followers_count": 1685, "friends_count": 498, "statues_count": 48147 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Avalon, CA", "id": "7a9d1044bce5cd3b", "name": "Avalon", "place_type": "city", "bounding_box": rectangle("-118.342496,33.330327 -118.309367,33.357104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 603274, "cityName": "Avalon" } }
+{ "create_at": datetime("2016-01-03T10:00:04.000Z"), "id": 683709424175714304, "text": "Wind 2.0 mph SSE. Barometer 29.960 in, Steady. Temperature 48.9 °F. Rain today 0.01 in. Humidity 44%", "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": 191, "friends_count": 58, "statues_count": 239704 }, "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-03T10:00:04.000Z"), "id": 683709424242835456, "text": "@davidcoverdale The EPIC Dry Humor.. That we share... Makes the Sun Shine...Ty David..You \"Rock\"", "in_reply_to_status": 683704308668354560, "in_reply_to_user": 166665318, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 166665318 }}, "user": { "id": 2302692242, "name": "Dan Klabunde Jr", "screen_name": "DkjDan", "lang": "en", "location": "Nebraska, USA", "create_at": date("2014-01-20"), "description": "Music is my Medicine,1st love.&My Bully SAMI, Keep Faith,& RocknRolln on this Journey Life which is too short!!!Enjoy the Simple things. lol...GODBLESS.....", "followers_count": 277, "friends_count": 392, "statues_count": 319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, NE", "id": "5ab200939c0b29b0", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-96.573254,41.413036 -96.444738,41.476099") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31053, "countyName": "Dodge", "cityID": 3117670, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-01-03T10:00:04.000Z"), "id": 683709424536387584, "text": "Every progressive person who scoffs at the idea of @BernieSanders getting the nom: YOU are the problem.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 216776631 }}, "user": { "id": 176968760, "name": "Haus of Rad", "screen_name": "retrotrash", "lang": "en", "location": "the trash", "create_at": date("2010-08-10"), "description": "Probably angry about something.", "followers_count": 319, "friends_count": 288, "statues_count": 18073 }, "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-03T10:00:04.000Z"), "id": 683709424557506560, "text": "Reasons why I don't give a fuck https://t.co/3jUFAB4foD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2832101143, "name": "anthony", "screen_name": "anthxnyperez", "lang": "en", "location": "null", "create_at": date("2014-09-25"), "description": "Pride too big", "followers_count": 355, "friends_count": 249, "statues_count": 4874 }, "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-03T10:00:04.000Z"), "id": 683709424570101760, "text": "@Gerg48 true gerg true", "in_reply_to_status": 683705383706296320, "in_reply_to_user": 635877528, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 635877528 }}, "user": { "id": 1063453579, "name": "Athena Dogantzis", "screen_name": "adogantzis", "lang": "en", "location": "null", "create_at": date("2013-01-05"), "description": "null", "followers_count": 307, "friends_count": 280, "statues_count": 5363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Short Pump, VA", "id": "018325d17666f4e3", "name": "Short Pump", "place_type": "city", "bounding_box": rectangle("-77.65493,37.624458 -77.582259,37.689361") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51087, "countyName": "Henrico", "cityID": 5172272, "cityName": "Short Pump" } }
+{ "create_at": datetime("2016-01-03T10:00:04.000Z"), "id": 683709424674975745, "text": "Chicago to Kuala Lumpur on American Airlines for $886 #CheapFlights #ORD https://t.co/Y534tgtvsF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.904404,41.979401"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CheapFlights", "ORD" }}, "user": { "id": 756129644, "name": "Flight Fishing", "screen_name": "FlightFishing", "lang": "en", "location": "Orlando, FL", "create_at": date("2012-08-13"), "description": "Collecting the internet's best airfares for flexible explorers. Sign up for free at http://www.flightfishing.com", "followers_count": 1132, "friends_count": 1505, "statues_count": 7967 }, "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-03T10:00:04.000Z"), "id": 683709424976834560, "text": "Join the Aurora Health Care team! See our latest #Nursing #job opening here: https://t.co/h6owUJ0ujn #KIEL, WI #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.023353,43.917963"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "KIEL", "Hiring", "CareerArc" }}, "user": { "id": 22025674, "name": "TMJ-WI Nursing Jobs", "screen_name": "tmj_wi_nursing", "lang": "en", "location": "Wisconsin", "create_at": date("2009-02-26"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Wisconsin Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 354, "friends_count": 301, "statues_count": 460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kiel, WI", "id": "fa60acd6499e19e2", "name": "Kiel", "place_type": "city", "bounding_box": rectangle("-88.054096,43.905273 -87.996048,43.928203") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55071, "countyName": "Manitowoc", "cityID": 5539525, "cityName": "Kiel" } }
+{ "create_at": datetime("2016-01-03T10:00:04.000Z"), "id": 683709425186545664, "text": "@kyliebking @fallon_holliday I'm sitting in my room alone laughing OUT LOUD remembering this", "in_reply_to_status": 683709253547261952, "in_reply_to_user": 4323040948, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4323040948, 2522761422 }}, "user": { "id": 48505303, "name": "lex.", "screen_name": "AlexiaKeann", "lang": "en", "location": "null", "create_at": date("2009-06-18"), "description": "flexin in texas", "followers_count": 833, "friends_count": 372, "statues_count": 31164 }, "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-03T10:00:04.000Z"), "id": 683709425224388608, "text": "tbh I'm really excited for 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": 421398134, "name": "sad sap", "screen_name": "sp00kmaster", "lang": "en", "location": "chicago", "create_at": date("2011-11-25"), "description": "I am a major disappointment and I like to play music", "followers_count": 254, "friends_count": 470, "statues_count": 15070 }, "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.887803 -88.081435,41.942768") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1711332, "cityName": "Carol Stream" } }
+{ "create_at": datetime("2016-01-03T10:00:04.000Z"), "id": 683709425488625665, "text": "Our first NFL game #riseup @ The Georgia Dome https://t.co/xFwqQTe7G9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.40067772,33.75775899"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "riseup" }}, "user": { "id": 264245273, "name": "Kelly", "screen_name": "Kelly_B_Roney", "lang": "en", "location": "null", "create_at": date("2011-03-11"), "description": "#LWYRUP", "followers_count": 52, "friends_count": 273, "statues_count": 399 }, "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-03T10:00:04.000Z"), "id": 683709425522204672, "text": "NYC's take on Western Chinese food, hand pulled noodle with tofu/seitan @ Xi'an Famous Foods https://t.co/x7EuwHr2uW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9853973,40.7279015"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 185910976, "name": "Holly Bik", "screen_name": "hollybik", "lang": "en", "location": "New York City", "create_at": date("2010-09-01"), "description": "Microbial eukaryotes & marine genomics - sequencing the ocean one handful of mud at a time. Project Scientist at @nyuniversity & blog at http://deepseanews.com", "followers_count": 6307, "friends_count": 1253, "statues_count": 13018 }, "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-03T10:00:04.000Z"), "id": 683709425928933376, "text": "Sunny this afternoon, high 47 (8 C). Low 22 (-6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712333, "name": "LubbockWeather", "screen_name": "LubbockWeather", "lang": "en", "location": "Lubbock, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Lubbock, Texas", "followers_count": 1187, "friends_count": 93, "statues_count": 8038 }, "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-03T10:00:04.000Z"), "id": 683709426063151106, "text": "*yells at top of lungs at other team*\n\"YOU TOO PISSED TO SHAKE HANDS? REALLY GOOD SPORTSMANSHIP\"\nokay mom", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1639205244, "name": "Alice", "screen_name": "__hockey_girl__", "lang": "en", "location": "VHS Hockey", "create_at": date("2013-08-01"), "description": "|when it comes to your future, aim for the top shelf|", "followers_count": 266, "friends_count": 237, "statues_count": 2458 }, "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-01-03T10:00:04.000Z"), "id": 683709426273009664, "text": "Chargers fan for the day. #WhoDey #SDvsDEN #BALvsCIN ⚡️��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WhoDey", "SDvsDEN", "BALvsCIN" }}, "user": { "id": 551188759, "name": "TheBrandonStall", "screen_name": "DatDudeBPS", "lang": "en", "location": "Reading, OH", "create_at": date("2012-04-11"), "description": "RHS '15 UCBA '17 UC '19 Go Reds Go Bengals Go Bearcats Snapchat: brandonstall", "followers_count": 336, "friends_count": 282, "statues_count": 11166 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reading, OH", "id": "1f70979508a8aceb", "name": "Reading", "place_type": "city", "bounding_box": rectangle("-84.454726,39.204223 -84.407099,39.241837") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3965732, "cityName": "Reading" } }
+{ "create_at": datetime("2016-01-03T10:00:04.000Z"), "id": 683709426302234624, "text": "@bmitchell1312 he doesn't ���� https://t.co/YopGocLd5y", "in_reply_to_status": 683708876550615041, "in_reply_to_user": 617615295, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 617615295 }}, "user": { "id": 551482269, "name": "Loran", "screen_name": "Loranchristene", "lang": "en", "location": "Out Of The Crowd ", "create_at": date("2012-04-11"), "description": "The house of blues has my heart❤️, love songs, Motorcycles & Psalms 139:14. PML", "followers_count": 701, "friends_count": 813, "statues_count": 9382 }, "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-03T10:00:04.000Z"), "id": 683709426474192896, "text": "@TwistUpTheGreen BRUHHHHHHHH!!!!! Me af right e ? ����������������������❤️ https://t.co/S2Su10P8MQ", "in_reply_to_status": -1, "in_reply_to_user": 426234324, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 426234324 }}, "user": { "id": 2185529877, "name": "the kid☯", "screen_name": "ZaddyMeech", "lang": "en", "location": "switchin lanes", "create_at": date("2013-11-17"), "description": "lavi$h", "followers_count": 1948, "friends_count": 1537, "statues_count": 73454 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Groves, TX", "id": "9af98223aefe3151", "name": "Groves", "place_type": "city", "bounding_box": rectangle("-93.944772,29.924735 -93.890543,29.96429") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48245, "countyName": "Jefferson", "cityID": 4831328, "cityName": "Groves" } }
+{ "create_at": datetime("2016-01-03T10:00:04.000Z"), "id": 683709426771996672, "text": "@kernymi same", "in_reply_to_status": 683709386888314880, "in_reply_to_user": 357635468, "favorite_count": 0, "coordinate": point("-121.8815652,39.7755215"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 357635468 }}, "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": 380, "friends_count": 395, "statues_count": 19879 }, "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-03T10:00:04.000Z"), "id": 683709426776289280, "text": "@TeamHHS1987 Check out my brand new music video \"Smooth\" #EvolveCrew #Norfolk #KnowingBetter https://t.co/wCNRmlqet0", "in_reply_to_status": -1, "in_reply_to_user": 379430593, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "EvolveCrew", "Norfolk", "KnowingBetter" }}, "user_mentions": {{ 379430593 }}, "user": { "id": 21249017, "name": "Page", "screen_name": "Pagemuzik", "lang": "en", "location": "fethernfinbout2getaboneless VA", "create_at": date("2009-02-18"), "description": "Tryin to spread love & shit to the darkest reaches of the galaxy http://www.datpiff.com/pop-mixtape-download.php?id=m6d433ad", "followers_count": 2148, "friends_count": 380, "statues_count": 14964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dix Hills, NY", "id": "18165a3f2b139f0f", "name": "Dix Hills", "place_type": "city", "bounding_box": rectangle("-73.380745,40.767592 -73.29345,40.83856") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3620687, "cityName": "Dix Hills" } }
+{ "create_at": datetime("2016-01-03T10:00:04.000Z"), "id": 683709426956656640, "text": "@Ashtyn_Billops @jeffrey_kohl who cares it was funny, perf video for perf hashtag", "in_reply_to_status": 683709274653114370, "in_reply_to_user": 316149087, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 316149087, 969567338 }}, "user": { "id": 338636781, "name": "Mal", "screen_name": "malloryham14", "lang": "en", "location": "memphis \n", "create_at": date("2011-07-19"), "description": "null", "followers_count": 765, "friends_count": 522, "statues_count": 17682 }, "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-01-03T10:00:04.000Z"), "id": 683709426960855041, "text": "@hannahep99 me too girl, me too. And it sucks that you're stuck.", "in_reply_to_status": 683707320640090112, "in_reply_to_user": 3306699303, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3306699303 }}, "user": { "id": 325151200, "name": "ℓαυяα кιиℓєу", "screen_name": "Kinley_Vardaman", "lang": "en", "location": "null", "create_at": date("2011-06-27"), "description": "live the life you love, and love the life you live.", "followers_count": 486, "friends_count": 1096, "statues_count": 13520 }, "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-03T10:00:04.000Z"), "id": 683709427065589760, "text": "I'm blessed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 442314426, "name": "AP All Day", "screen_name": "_amainy", "lang": "en", "location": "Maringouin, Louisiana ", "create_at": date("2011-12-20"), "description": "Everyone has a story, mines just haven't been told yet R.I.P MOM ITI'18 Safety Manger #AGNB", "followers_count": 1042, "friends_count": 1576, "statues_count": 7225 }, "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-03T10:00:04.000Z"), "id": 683709427208212482, "text": "hbd to my munchkin �� @Hannahrae115 https://t.co/GlUHXsCoi2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 426193291 }}, "user": { "id": 373154182, "name": "ken", "screen_name": "_kennadavis4", "lang": "en", "location": "null", "create_at": date("2011-09-13"), "description": "Sippin on Dom Pérignon for no reason", "followers_count": 537, "friends_count": 363, "statues_count": 3370 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Windsor, CO", "id": "004a505242d7e20d", "name": "Windsor", "place_type": "city", "bounding_box": rectangle("-105.008977,40.436985 -104.874051,40.492648") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8123, "countyName": "Weld", "cityID": 885485, "cityName": "Windsor" } }
+{ "create_at": datetime("2016-01-03T10:00:04.000Z"), "id": 683709427208314880, "text": "Word! https://t.co/lpEDh4JsSD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29822803, "name": "Darkskin Swaggin", "screen_name": "MudahPacino_973", "lang": "en", "location": "Paterson NJ", "create_at": date("2009-04-08"), "description": "Fuck You Too.", "followers_count": 1966, "friends_count": 737, "statues_count": 128007 }, "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-03T10:00:04.000Z"), "id": 683709427384320000, "text": "When you find out there's no school tomorrow https://t.co/FFv8OVaqvy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2956502514, "name": "PK", "screen_name": "Parkerchittum10", "lang": "en", "location": "Joshua, TX", "create_at": date("2015-01-02"), "description": "⚾️", "followers_count": 854, "friends_count": 555, "statues_count": 3069 }, "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-03T10:00:04.000Z"), "id": 683709427547942912, "text": "Can you recommend anyone for this #job? Physician Neurology - https://t.co/kTECuUwTTq #SpokaneValley, WA #Healthcare #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.2155713,47.6811239"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "SpokaneValley", "Healthcare", "Hiring", "CareerArc" }}, "user": { "id": 28854077, "name": "TMJ-WAK Health Jobs", "screen_name": "tmj_wak_health", "lang": "en", "location": "Spokane, WA", "create_at": date("2009-04-04"), "description": "Follow this account for geo-targeted Healthcare job tweets in Spokane, WA. Need help? Tweet us at @CareerArc!", "followers_count": 468, "friends_count": 286, "statues_count": 187 }, "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.325383,47.627801 -117.143601,47.697352") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5367167, "cityName": "Spokane Valley" } }
+{ "create_at": datetime("2016-01-03T10:00:04.000Z"), "id": 683709427879391235, "text": "I'll have my mom visit me every once in a while since they gonna have to lock my ass up https://t.co/6ECoexChzx", "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": "Your Heavenly Father", "screen_name": "DrZakky", "lang": "en", "location": "St. Cheeks Cathedral ", "create_at": date("2012-03-27"), "description": "Saviour/Shinobi/expert geocacher/pokemaster/found the blueprints and I'm still here", "followers_count": 738, "friends_count": 623, "statues_count": 19140 }, "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-01-03T10:00:04.000Z"), "id": 683709427891847169, "text": "someone please tell me they have a class with me�� https://t.co/msxnfbLbJk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 139458056, "name": "Mel", "screen_name": "MellFabian", "lang": "en", "location": "adventures", "create_at": date("2010-05-02"), "description": "welcome to my life, it's a roller coaster. buckle up or get out because it's a long bumpy ride .", "followers_count": 722, "friends_count": 1097, "statues_count": 13620 }, "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-03T10:00:04.000Z"), "id": 683709427975860224, "text": "❤❤❤❤ https://t.co/KGszUL7niP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 786868418, "name": "Kaetlyn Rayne", "screen_name": "kaylorka", "lang": "en", "location": "Cincinnati, OH", "create_at": date("2012-08-28"), "description": "She acts like summer and walks like rain☔☀ 09.26.2015 || Lillian ❤", "followers_count": 793, "friends_count": 1568, "statues_count": 28343 }, "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-03T10:00:05.000Z"), "id": 683709428017725440, "text": "When I get a house, I'm going to invest in an Internet connection with a static IP (including #IPv6) address.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.33207,47.60621"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "IPv6" }}, "user": { "id": 916596266, "name": "竹下E本", "screen_name": "takenji_ebooks", "lang": "en", "location": "The Past", "create_at": date("2012-10-31"), "description": "Eh, close enough.", "followers_count": 145, "friends_count": 1, "statues_count": 99259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "f42a863798156617", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.459696,47.491912 -122.224433,47.734145") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-01-03T10:00:05.000Z"), "id": 683709428084961280, "text": "#BeatDallas #HTTR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "BeatDallas", "HTTR" }}, "user": { "id": 218376100, "name": "2015 NFC EAST CHAMPS", "screen_name": "Always_HTTR", "lang": "en", "location": "NSU➡️NOVA➡️VCU", "create_at": date("2010-11-21"), "description": "Coulda been anywhere in the world but your here with me that's good for my ego", "followers_count": 551, "friends_count": 412, "statues_count": 32983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manassas Park, VA", "id": "017426a66ed0cb82", "name": "Manassas Park", "place_type": "city", "bounding_box": rectangle("-77.478773,38.753457 -77.421716,38.788313") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51685, "countyName": "Manassas Park", "cityID": 5148968, "cityName": "Manassas Park" } }
+{ "create_at": datetime("2016-01-03T10:00:05.000Z"), "id": 683709428194000896, "text": "Ordered two drinks. Both are wrong. Yup, you guys deserve $15/hr. ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15322625, "name": "Jeremy", "screen_name": "_JeremyBlake", "lang": "en", "location": "Huntington, WV, USA", "create_at": date("2008-07-04"), "description": "Photographer.", "followers_count": 663, "friends_count": 45, "statues_count": 40266 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Barboursville, WV", "id": "bedb151fa68edb66", "name": "Barboursville", "place_type": "city", "bounding_box": rectangle("-82.322131,38.383756 -82.249985,38.427939") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54011, "countyName": "Cabell", "cityID": 5404276, "cityName": "Barboursville" } }
+{ "create_at": datetime("2016-01-03T10:00:05.000Z"), "id": 683709428240011268, "text": "#CareerArc #Labor #Job alert: General Warehouse | CVS Health | #KAPOLEI, HI https://t.co/KqDuTXea5i #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-158.056892,21.3354034"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CareerArc", "Labor", "Job", "KAPOLEI", "Jobs", "Hiring" }}, "user": { "id": 118676555, "name": "TMJ-HI Labor Jobs", "screen_name": "tmj_HI_LABOR", "lang": "en", "location": "Hawaii", "create_at": date("2010-03-01"), "description": "Follow this account for geo-targeted General Labor job tweets in Hawaii Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 284, "friends_count": 279, "statues_count": 4 }, "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-03T10:00:05.000Z"), "id": 683709428491665409, "text": "See our latest #Tempe, AZ #job and click to apply: Cook and Chef - https://t.co/f42nrG2mQA #restaurantjobs #Hospitality #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.9403254,33.4366655"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Tempe", "job", "restaurantjobs", "Hospitality", "Hiring", "CareerArc" }}, "user": { "id": 22488656, "name": "Phoenix Hospitality", "screen_name": "tmj_phx_hrta", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Phoenix, AZ. Need help? Tweet us at @CareerArc!", "followers_count": 508, "friends_count": 293, "statues_count": 1116 }, "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-03T10:00:05.000Z"), "id": 683709428571348992, "text": "I wish my parents would stop asking me if I met someone yet... No I don't even look... I don't care... I'm still trying to just move on", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27732056, "name": "Tambourine", "screen_name": "Hawtt_Mess", "lang": "en", "location": "Miss California ☀️", "create_at": date("2009-03-30"), "description": "when the pimps in the crib ma drop it like its hot ❤️⚰☠✨ I also have the force", "followers_count": 473, "friends_count": 613, "statues_count": 2832 }, "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.585913 -117.553442,33.672953") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 659587, "cityName": "Rancho Santa Margarita" } }
+{ "create_at": datetime("2016-01-03T10:00:05.000Z"), "id": 683709428646809600, "text": "I should consider slowing down on weed ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 330031101, "name": "Wingstop Is Bae", "screen_name": "_DuronTaylorJet", "lang": "en", "location": "Staying Out The Way ", "create_at": date("2011-07-05"), "description": "Dallas.", "followers_count": 1872, "friends_count": 994, "statues_count": 91543 }, "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-03T10:00:05.000Z"), "id": 683709428797837312, "text": "Sunny this afternoon, high 50 (10 C). Low 34 (1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712562, "name": "ElPasoWX", "screen_name": "ElPasoWX", "lang": "en", "location": "El Paso, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for El Paso, Texas", "followers_count": 526, "friends_count": 93, "statues_count": 8096 }, "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-03T10:00:05.000Z"), "id": 683709428969947137, "text": "Wind 5.0 mph WSW. Barometer 29.631 in, Falling Rapidly. Temperature 36.8 °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": 0, "friends_count": 0, "statues_count": 3345 }, "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-03T10:00:05.000Z"), "id": 683709429271769089, "text": "Join the Central Peninsula Hospital team! See our latest #Healthcare #job opening here: https://t.co/Q7zvgQ6N0u https://t.co/pQTNztoToE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-151.0583332,60.4877778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job" }}, "user": { "id": 3278999924, "name": "CPH Jobs", "screen_name": "CPHJobs", "lang": "en", "location": "Soldotna, AK", "create_at": date("2015-07-13"), "description": "At CPH every employee is considered a caregiver, an integral part of our health care team. Follow us to learn more!", "followers_count": 25, "friends_count": 77, "statues_count": 50 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Soldotna, AK", "id": "350518f7c1ac93ff", "name": "Soldotna", "place_type": "city", "bounding_box": rectangle("-151.193024,60.458405 -151.048563,60.511008") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2122, "countyName": "Kenai Peninsula", "cityID": 271640, "cityName": "Soldotna" } }
+{ "create_at": datetime("2016-01-03T10:00:05.000Z"), "id": 683709429301284866, "text": "Get to see Tom Brady, Gronk and the Patriots take on the Dolphins in the season finale!… https://t.co/J3elJAnxb4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.23876791,25.95764033"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 91258382, "name": "Jose Paez", "screen_name": "josepaez8905", "lang": "en", "location": "Miami, FL", "create_at": date("2009-11-19"), "description": "IG: Jose_Paez_8905\nFB: jose.paez.58", "followers_count": 143, "friends_count": 270, "statues_count": 247 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scott Lake, FL", "id": "2da4b3d74e7bf5d6", "name": "Scott Lake", "place_type": "city", "bounding_box": rectangle("-80.246538,25.926454 -80.210528,25.96128") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245060, "cityName": "Miami Gardens" } }
+{ "create_at": datetime("2016-01-03T10:00:05.000Z"), "id": 683709429502447616, "text": "I have lost so many potential Relationship with some really good people , trying to live out a Dream ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 869197026, "name": "Azie Faishon", "screen_name": "HCJayB", "lang": "en", "location": "Oakland, CA", "create_at": date("2012-10-08"), "description": "#SheGoneGoFaDaFro Original", "followers_count": 1092, "friends_count": 917, "statues_count": 32281 }, "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-03T10:00:05.000Z"), "id": 683709429682802688, "text": "This #Media #job might be a great fit for you: On Call Remote Gaffer - https://t.co/68Jvk6wXGp #HSN #HSNCareers #SaintPetersburg, FL #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.64,27.7730556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Media", "job", "HSN", "HSNCareers", "SaintPetersburg", "Hiring" }}, "user": { "id": 48348692, "name": "HSNCareers", "screen_name": "HSNCareers", "lang": "en", "location": "St. Petersburg, Florida", "create_at": date("2009-06-18"), "description": "Check out the GREAT CAREERS at HSN", "followers_count": 574, "friends_count": 77, "statues_count": 842 }, "place": { "country": "United States", "country_code": "United States", "full_name": "St Petersburg, FL", "id": "5d231ed8656fcf5a", "name": "St Petersburg", "place_type": "city", "bounding_box": rectangle("-82.758209,27.694323 -82.587597,27.897116") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1263000, "cityName": "St. Petersburg" } }
+{ "create_at": datetime("2016-01-03T10:00:05.000Z"), "id": 683709430198865921, "text": "���������� https://t.co/UTm4NWYFS2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 34795013, "name": "A.", "screen_name": "whatupallie", "lang": "en", "location": "Cincinnati, OH", "create_at": date("2009-04-23"), "description": "The super mega fat bitch of Clifton.", "followers_count": 1360, "friends_count": 589, "statues_count": 46790 }, "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-01-03T10:00:05.000Z"), "id": 683709430341369857, "text": "Boston Globe reporters and editors hit streets to deliver the Sunday paper https://t.co/Gc6jpUnMWC", "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": 17694, "friends_count": 17579, "statues_count": 66539 }, "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-03T10:00:05.000Z"), "id": 683709430697832448, "text": "@a_Myers9 Every. Single. Can. ��", "in_reply_to_status": 683709051671306240, "in_reply_to_user": 475369380, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 475369380 }}, "user": { "id": 436796988, "name": "Sam Schneider", "screen_name": "Sam_J_Schneider", "lang": "en", "location": "Columbus, OH ➡️ San Diego, Ca", "create_at": date("2011-12-14"), "description": "Local celebrity.", "followers_count": 2008, "friends_count": 411, "statues_count": 85974 }, "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-03T10:00:05.000Z"), "id": 683709431109038087, "text": "Rex Ryan is not the hero we may want, but he is the hero we need", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2480177839, "name": "Stop The Woo", "screen_name": "StopTheWoo", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2014-05-06"), "description": "Respect the game. Respect the players. Respect the fans. Stop The Woo.", "followers_count": 1025, "friends_count": 2004, "statues_count": 8514 }, "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-03T10:00:05.000Z"), "id": 683709431113236480, "text": "@LynnrobertsL do what ������", "in_reply_to_status": 683709198593568769, "in_reply_to_user": 2338488377, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2338488377 }}, "user": { "id": 144398098, "name": "Yo soy el Cabra", "screen_name": "D73__", "lang": "en", "location": "NOLA", "create_at": date("2010-05-15"), "description": "University of Louisiana at Monroe O-D All American", "followers_count": 572, "friends_count": 514, "statues_count": 7396 }, "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-03T10:00:05.000Z"), "id": 683709431255707649, "text": "temperature up 42°F -> 53°F\nhumidity down 76% -> 47%\nwind 0mph -> 3mph\npressure 30.01in rising", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.90588,34.13361"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 213899084, "name": "Azusa Weather", "screen_name": "AzusaCA", "lang": "en", "location": "Azusa, CA", "create_at": date("2010-11-09"), "description": "Weather updates, forecast, warnings and information for Azusa, CA. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 13, "friends_count": 1, "statues_count": 19626 }, "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-03T10:00:05.000Z"), "id": 683709431310241792, "text": "Want to work at Advantage Solutions? We're #hiring in #Honolulu, HI! Click for details: https://t.co/cETdy2uDLT #Marketing #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Honolulu", "Marketing", "Job", "Jobs" }}, "user": { "id": 23493115, "name": "TMJ-HON Advert Jobs", "screen_name": "tmj_hon_adv", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Marketing/Ad/PR job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 290, "friends_count": 193, "statues_count": 9 }, "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-03T10:00:05.000Z"), "id": 683709431356325888, "text": "@jigggdaddy what size is them ? ��", "in_reply_to_status": 683709317648924673, "in_reply_to_user": 521731699, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 521731699 }}, "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", "followers_count": 521, "friends_count": 605, "statues_count": 6544 }, "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-03T10:00:05.000Z"), "id": 683709431603957760, "text": "S T U N N I N G Sunday Morning with John Piper & @rendcollective #passion2016 @ Infinite Energy… https://t.co/kSCrzt2P2a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.0930176,33.9914894"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "passion2016" }}, "user_mentions": {{ 46446547 }}, "user": { "id": 232641796, "name": "Amy Elizabeth Boland", "screen_name": "AmyBoland37", "lang": "en", "location": "Atlanta, GA", "create_at": date("2010-12-31"), "description": "Passionate. Communicator. Door Holder. Lover of The Word, People, Worship & Justice. He gives me beauty for ashes • Ezekiel 37 •", "followers_count": 1262, "friends_count": 761, "statues_count": 8276 }, "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": 13135, "countyName": "Gwinnett" } }
+{ "create_at": datetime("2016-01-03T10:00:05.000Z"), "id": 683709431658364929, "text": "Woke up feeling like shit, overslept and was late to work. Today is gonna be 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": { "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": 2224, "friends_count": 1412, "statues_count": 137276 }, "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-03T10:00:05.000Z"), "id": 683709431847227393, "text": "I just tagged you in this�������������������������������������������� https://t.co/UUXl3Qkjs8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 367415113, "name": "Princess", "screen_name": "xxAriyanxx_", "lang": "en", "location": "in bed", "create_at": date("2011-09-03"), "description": "Princess❤️|Perfectly Imperfect.| DHS'17", "followers_count": 1249, "friends_count": 1183, "statues_count": 11616 }, "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-01-03T10:00:05.000Z"), "id": 683709431863885825, "text": "@Im_BeyondGreat @TheZenSettings he really retweeting old cowboy tweets so I did the same lol", "in_reply_to_status": 683709252624498688, "in_reply_to_user": 291506241, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 450096263, 846642469 }}, "user": { "id": 291506241, "name": "Jasmine B.", "screen_name": "lovejazzy94", "lang": "en", "location": "PG County,MD", "create_at": date("2011-05-01"), "description": "21. God 1st @WMUCSports Volleyball & Basketball reporter for the MarylandTerrapins|#BrooksWeekly writer| #UMD|", "followers_count": 2416, "friends_count": 1516, "statues_count": 245226 }, "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.965622,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-01-03T10:00:06.000Z"), "id": 683709432232988672, "text": "Sunny this afternoon, high 51 (11 C). Low 27 (-3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712512, "name": "AbileneWeather", "screen_name": "AbileneWeather", "lang": "en", "location": "Abilene, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Abilene, Texas", "followers_count": 929, "friends_count": 93, "statues_count": 8136 }, "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-03T10:00:06.000Z"), "id": 683709432484749312, "text": "And this is why Farmer and Pettine will be without jobs tomorrow https://t.co/kRdKmAr1MQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 611364254, "name": "travis", "screen_name": "travisparks24", "lang": "en", "location": "Lusby, MD", "create_at": date("2012-06-17"), "description": "PaxNation, Kamryn #24EVER", "followers_count": 313, "friends_count": 687, "statues_count": 25019 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chesapeake Ranch Estates, MD", "id": "4dd2cff80a3e3636", "name": "Chesapeake Ranch Estates", "place_type": "city", "bounding_box": rectangle("-76.451763,38.319092 -76.380792,38.387749") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24009, "countyName": "Calvert", "cityID": 2416050, "cityName": "Chesapeake Ranch Estates" } }
+{ "create_at": datetime("2016-01-03T10:00:06.000Z"), "id": 683709432597839873, "text": "I cannot walk around like this another day ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 519955512, "name": "Feb23rd♓", "screen_name": "BxtchIm_Desirea", "lang": "en", "location": "null", "create_at": date("2012-03-09"), "description": "I'm being selfish , I'm putting myself in FIRST place .. I'm doing me ❤ This time around ❤❤", "followers_count": 417, "friends_count": 285, "statues_count": 15637 }, "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-03T10:00:06.000Z"), "id": 683709432820285440, "text": "DUQ STARTERS: @AprilRobinson32 // @ChasOmogrosso2 // @Amadea15 // @_Milie // @KadriAnnLass #A10WBB #1world1team1goal", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "A10WBB", "1world1team1goal" }}, "user_mentions": {{ 68759966, 1445192227, 20303924, 31386366, 753325530 }}, "user": { "id": 404158876, "name": "Duquesne WBB", "screen_name": "DuqWBB", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2011-11-03"), "description": "Welcome the Official Duquesne Women's Basketball Twitter! Follow us for updates on the team! Like us at https://www.facebook.com/DuqWBB", "followers_count": 1006, "friends_count": 231, "statues_count": 2287 }, "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-03T10:00:06.000Z"), "id": 683709432832888834, "text": "@Meaghaan__ wow love u too����❤️", "in_reply_to_status": 683708068752945152, "in_reply_to_user": 400478677, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 400478677 }}, "user": { "id": 1312300212, "name": "kass ♔", "screen_name": "kassi_alexiss", "lang": "en", "location": "panthers game or with meaghan", "create_at": date("2013-03-28"), "description": "romans 8:18", "followers_count": 615, "friends_count": 229, "statues_count": 14381 }, "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-01-03T10:00:06.000Z"), "id": 683709432954523649, "text": "#ThankYou22 @MattForte22", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "ThankYou22" }}, "user_mentions": {{ 28183933 }}, "user": { "id": 68301877, "name": "Adolfo Gomez", "screen_name": "adolfoag3", "lang": "en", "location": "Berwyn,IL/Chuck Vegas", "create_at": date("2009-08-23"), "description": "Sometimes I brag like Hov #EIU17", "followers_count": 242, "friends_count": 621, "statues_count": 15814 }, "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-03T10:00:06.000Z"), "id": 683709433084522497, "text": "@LudgerBrinkmann I did one piss video with @kinkmen in 2013. Aside from that, nothing I can think of on the web.", "in_reply_to_status": 683709231254626304, "in_reply_to_user": 455036429, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 455036429, 172048331 }}, "user": { "id": 164705650, "name": "Rob Yaeger", "screen_name": "yaegerman", "lang": "en", "location": "New York, NY", "create_at": date("2010-07-09"), "description": "Omnisexual porn guy. Viking Descendant. Master of Natural Activities (MNA). Equal Opportunity Fucker (EOF): Book me: yaegerman@gmail.com 18+ only #RYE #BRD", "followers_count": 3846, "friends_count": 354, "statues_count": 2836 }, "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-03T10:00:06.000Z"), "id": 683709433130696705, "text": "@marielycruz_ tamo iguale ��", "in_reply_to_status": 683709302700249088, "in_reply_to_user": 2722013060, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 2722013060 }}, "user": { "id": 2202054037, "name": "Luis Angel", "screen_name": "LuiggyD_", "lang": "en", "location": "Bronx NYC", "create_at": date("2013-11-18"), "description": "IG & SC kingluiggy", "followers_count": 319, "friends_count": 289, "statues_count": 37700 }, "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-03T10:00:06.000Z"), "id": 683709433189380098, "text": "summer during winter ❣ @ South Beach, Miami https://t.co/op0WDx24lS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.33370343,25.80941675"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3304442712, "name": "Clara", "screen_name": "claracalavia", "lang": "es", "location": "Miami ", "create_at": date("2015-08-02"), "description": "null", "followers_count": 202, "friends_count": 224, "statues_count": 2656 }, "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-01-03T10:00:06.000Z"), "id": 683709433193611264, "text": "@AlyssaEdwards_1 okay Ms.Thang blessing my twitter with philosophy early in the morning", "in_reply_to_status": 683709163835265025, "in_reply_to_user": 944835650, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 944835650 }}, "user": { "id": 1305803352, "name": "whts kolaud", "screen_name": "WhtsKolaud", "lang": "en", "location": "Morgantown, WV", "create_at": date("2013-03-26"), "description": "whts kolaud", "followers_count": 967, "friends_count": 1862, "statues_count": 16358 }, "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-01-03T10:00:06.000Z"), "id": 683709433508057089, "text": "Can you recommend anyone for this #job? Management Trainee - https://t.co/GuCaxuVQCZ #EWABEACH, HI #Retail #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-158.0131382,21.3161149"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "EWABEACH", "Retail", "Hiring", "CareerArc" }}, "user": { "id": 28625039, "name": "TMJ-HON Retail Jobs", "screen_name": "tmj_hon_retail", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 374, "friends_count": 311, "statues_count": 445 }, "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-01-03T10:00:06.000Z"), "id": 683709433508139008, "text": "@cuz_fuk_you Did you ever make it out of that town where nothing ever happened?", "in_reply_to_status": -1, "in_reply_to_user": 4271248753, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4271248753 }}, "user": { "id": 3045799672, "name": "Chloe Rasmussen", "screen_name": "xChloe888", "lang": "en", "location": "null", "create_at": date("2015-02-19"), "description": "null", "followers_count": 30, "friends_count": 77, "statues_count": 206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Timberlane, IL", "id": "00b5b52740358634", "name": "Timberlane", "place_type": "city", "bounding_box": rectangle("-88.889727,42.305542 -88.824758,42.367382") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17007, "countyName": "Boone", "cityID": 1775360, "cityName": "Timberlane" } }
+{ "create_at": datetime("2016-01-03T10:00:06.000Z"), "id": 683709433629679616, "text": "Right now it's the end of you and me it's too late I can't wait for you to be gone ������ https://t.co/IMTLe5donR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1455493188, "name": "JF", "screen_name": "_JMerita", "lang": "en", "location": "Aua American Samoa ", "create_at": date("2013-05-24"), "description": "Folasi Sonny Iosefo❤", "followers_count": 580, "friends_count": 489, "statues_count": 83141 }, "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-03T10:00:06.000Z"), "id": 683709433633898498, "text": "@ArRecruitingGuy Any Devin White update?", "in_reply_to_status": -1, "in_reply_to_user": 28027757, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28027757 }}, "user": { "id": 623680742, "name": "Tim Culver", "screen_name": "timculver44", "lang": "en", "location": "null", "create_at": date("2012-07-01"), "description": "null", "followers_count": 240, "friends_count": 249, "statues_count": 875 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ozark, AR", "id": "1f6a46e66d42a6b5", "name": "Ozark", "place_type": "city", "bounding_box": rectangle("-93.874687,35.47891 -93.799391,35.50899") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5047, "countyName": "Franklin", "cityID": 552970, "cityName": "Ozark" } }
+{ "create_at": datetime("2016-01-03T10:00:06.000Z"), "id": 683709433755660296, "text": "Stopped smokin' blunts so I can run a mile!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2791951817, "name": "Jeff.Ramos", "screen_name": "JeffRamos32", "lang": "en", "location": "null", "create_at": date("2014-09-29"), "description": "Is hating getting u paid?", "followers_count": 152, "friends_count": 129, "statues_count": 3881 }, "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-03T10:00:06.000Z"), "id": 683709433864572928, "text": "current weather in Madison: cloudy, 27°F\n66% humidity, wind 9mph, visibility 10mi, pressure 30.19in rising", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.38669,43.07295"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 141440102, "name": "Madison Weather", "screen_name": "_MadisonWI", "lang": "en", "location": "Madison, WI", "create_at": date("2010-05-07"), "description": "Weather updates, forecast, warnings and information for Madison, WI. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 145, "friends_count": 1, "statues_count": 23596 }, "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.491889 -86.24955,47.309715") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55025, "countyName": "Dane", "cityID": 5548000, "cityName": "Madison" } }
+{ "create_at": datetime("2016-01-03T10:00:06.000Z"), "id": 683709434346876929, "text": "Shut. Up. Really?! ���������� https://t.co/ORxtaYPHAm", "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": "i be chillin.", "screen_name": "HBfromKC", "lang": "en", "location": "Kansas City, MO", "create_at": date("2011-04-15"), "description": "Sarcastic asshole Mommy whos Anti-Twiminism & loves Drake. One day ima be fine & yall wont be able to tell me shit. Washed Twit Prez. #BBW #25AndUpTwitter", "followers_count": 2072, "friends_count": 1998, "statues_count": 126204 }, "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-03T10:00:06.000Z"), "id": 683709434455982080, "text": "Lol there's too many niggas that can say they fucked my bitch. Simple. https://t.co/6Co7py8KkW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 255309849, "name": "MindFul", "screen_name": "YeaaaaQuannn", "lang": "en", "location": "null", "create_at": date("2011-02-20"), "description": "If Your Knowledge Were Your Wealth Then It Would Be Well Earned - On And On #BackShotSquad", "followers_count": 1681, "friends_count": 1340, "statues_count": 43895 }, "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-03T10:00:06.000Z"), "id": 683709434514649088, "text": "@ravenluke @njpwworld Yup! Immediately after the live airing.", "in_reply_to_status": 683706498501853185, "in_reply_to_user": 45753744, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 45753744, 2889734318 }}, "user": { "id": 637328248, "name": "Voices of Wrestling", "screen_name": "voiceswrestling", "lang": "en", "location": "Illinois & Texas", "create_at": date("2012-07-16"), "description": "Pro wrestling columns, podcasts & event coverage. Tweets: @richkraetsch & @joemlanza.", "followers_count": 4832, "friends_count": 539, "statues_count": 69795 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Park, IL", "id": "c8b021ffd0c94ffc", "name": "Oak Park", "place_type": "city", "bounding_box": rectangle("-87.805698,41.865006 -87.774103,41.909253") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1754885, "cityName": "Oak Park" } }
+{ "create_at": datetime("2016-01-03T10:00:06.000Z"), "id": 683709434560929794, "text": "all I can think about are exams ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1710897997, "name": "mion", "screen_name": "yagirlmion", "lang": "en", "location": "null", "create_at": date("2013-08-29"), "description": "life's better w. him ❤️", "followers_count": 1939, "friends_count": 569, "statues_count": 16498 }, "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-03T10:00:06.000Z"), "id": 683709434879717376, "text": "@QuarterbackDiaz these are my Week 17 winners, in case you didn't understand.", "in_reply_to_status": 683709310648631296, "in_reply_to_user": 334312980, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 334312980 }}, "user": { "id": 334312980, "name": "Jarett Diaz", "screen_name": "QuarterbackDiaz", "lang": "en", "location": "The Boogie Down", "create_at": date("2011-07-12"), "description": "charismatic straight edge atheist quarterback slash slot receiver with a way better instagram account.", "followers_count": 323, "friends_count": 690, "statues_count": 5758 }, "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-03T10:00:06.000Z"), "id": 683709434904862724, "text": "Piscataway Town is hiring! Computer Associ #jobs in PISCATAWAY Apply today https://t.co/KrVmhcwrm4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.464675,40.551764"), "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": 168, "friends_count": 42, "statues_count": 72040 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Piscataway, NJ", "id": "01a0aa02181f66f1", "name": "Piscataway", "place_type": "city", "bounding_box": rectangle("-74.524236,40.506013 -74.40873,40.597526") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34035, "countyName": "Somerset" } }
+{ "create_at": datetime("2016-01-03T10:00:06.000Z"), "id": 683709435231903744, "text": "���� i love u lots liv and youre super sweet and pretty but we still havent hung out so wth!!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1361631596, "name": "kir", "screen_name": "kirasmith6600", "lang": "en", "location": "null", "create_at": date("2013-04-18"), "description": "chris❤️", "followers_count": 570, "friends_count": 302, "statues_count": 8265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Magnolia, TX", "id": "002c3ee30c95b70f", "name": "Magnolia", "place_type": "city", "bounding_box": rectangle("-95.804153,30.18588 -95.723373,30.228262") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48339, "countyName": "Montgomery", "cityID": 4846056, "cityName": "Magnolia" } }
+{ "create_at": datetime("2016-01-03T10:00:06.000Z"), "id": 683709435328385024, "text": "Join meat CCV Lone Hill at 10 for service abs the beginingof a new sermon series! #ccvsocal #sunday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ccvsocal", "sunday" }}, "user": { "id": 123670852, "name": "Lauren LaPorte", "screen_name": "Lauren_LP2", "lang": "en", "location": "Glendora, CA", "create_at": date("2010-03-16"), "description": "Just a girl. Singer, actress, model, baker, techie, random, awkward, Christ follower.", "followers_count": 470, "friends_count": 712, "statues_count": 816 }, "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-03T10:00:06.000Z"), "id": 683709435487731712, "text": "Sunny this afternoon, high 49 (9 C). Low 26 (-3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712530, "name": "MidlandWeather", "screen_name": "MidlandWeather", "lang": "en", "location": "Midland/Odessa, TX", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Midland/Odessa, TX", "followers_count": 880, "friends_count": 93, "statues_count": 8120 }, "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-01-03T10:00:06.000Z"), "id": 683709436188295169, "text": "Piscataway Town is hiring! Business Office #jobs in PISCATAWAY Apply today https://t.co/A7uFMgTPt9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.464675,40.551764"), "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": 168, "friends_count": 42, "statues_count": 72041 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Piscataway, NJ", "id": "01a0aa02181f66f1", "name": "Piscataway", "place_type": "city", "bounding_box": rectangle("-74.524236,40.506013 -74.40873,40.597526") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34035, "countyName": "Somerset" } }
+{ "create_at": datetime("2016-01-03T10:00:06.000Z"), "id": 683709436242866177, "text": "This is so cool https://t.co/uGiHpSwztl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1491266354, "name": "Honéy", "screen_name": "honeyytrish", "lang": "en", "location": "null", "create_at": date("2013-06-07"), "description": "You either win or you learn | insta: _honeytrish", "followers_count": 1278, "friends_count": 280, "statues_count": 82842 }, "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-03T10:00:06.000Z"), "id": 683709436318380032, "text": "@ColeyHarvey @pauldehnerjr @GeoffHobsonCin curiosity ? What's up with Hunt active over Clarke the past few weeks? Hunt better?Clarke worse?", "in_reply_to_status": -1, "in_reply_to_user": 12794312, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 12794312, 49370599, 270462765 }}, "user": { "id": 729700910, "name": "Jeff Linne", "screen_name": "Linne52570", "lang": "en", "location": "Portsmouth, OH", "create_at": date("2012-07-31"), "description": "null", "followers_count": 17, "friends_count": 87, "statues_count": 372 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portsmouth, OH", "id": "8a6345658f3277de", "name": "Portsmouth", "place_type": "city", "bounding_box": rectangle("-83.02068,38.726819 -82.932079,38.786816") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39145, "countyName": "Scioto", "cityID": 3964304, "cityName": "Portsmouth" } }
+{ "create_at": datetime("2016-01-03T10:00:07.000Z"), "id": 683709436674850817, "text": "@CarlyyWilsonn19 good luck", "in_reply_to_status": 683697324879654912, "in_reply_to_user": 338151488, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 338151488 }}, "user": { "id": 793739172, "name": "Evan Warner", "screen_name": "evan_c_warner", "lang": "en", "location": "null", "create_at": date("2012-08-31"), "description": "⚡️R.I.P. 9/18/13 ⚡️ | NPCA '16 | SV Soccer #12 | Revelation 21:6", "followers_count": 232, "friends_count": 269, "statues_count": 7931 }, "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-03T10:00:07.000Z"), "id": 683709437056421888, "text": "WHY do people come to the movies and sit next to you when there are TONS of empty seats!?!? �� #GTFAFM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GTFAFM" }}, "user": { "id": 430200604, "name": "Kimmie Delight", "screen_name": "BrownieLove75", "lang": "en", "location": "Arizona", "create_at": date("2011-12-06"), "description": "from the #DMV living in the #ValleyOfTheSun CrazySexyCool...get to know me \nsnapchat: brownielove75", "followers_count": 324, "friends_count": 744, "statues_count": 4220 }, "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-03T10:00:07.000Z"), "id": 683709437127831552, "text": "13:00:01 |Temp: 49.3ºF | Wind Chill 49.3ºF |Dew Point 38.4ºF | Rain today: 0.00 inches | Wind: 2.0 mph from the WSW, 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": 87949 }, "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-03T10:00:07.000Z"), "id": 683709437236785153, "text": "1/3/2016 - 12:00\nTemp: 52.1F \nHum: 57%\nWind: 1.0 mph\nBaro: 30.204in. & 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": 105, "friends_count": 52, "statues_count": 48661 }, "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-03T10:00:07.000Z"), "id": 683709437354328064, "text": "@Aminotaures mais ptn c flippant", "in_reply_to_status": 683709265920528384, "in_reply_to_user": 545218455, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user_mentions": {{ 545218455 }}, "user": { "id": 2723759169, "name": "DucBilal", "screen_name": "Givefuuck", "lang": "fr", "location": "los angeles ", "create_at": date("2014-07-24"), "description": "Snap : theetalon", "followers_count": 17916, "friends_count": 90, "statues_count": 30990 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-03T10:00:07.000Z"), "id": 683709437534711808, "text": "@ky_marsh https://t.co/J309XotQ5j", "in_reply_to_status": -1, "in_reply_to_user": 1654509818, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1654509818 }}, "user": { "id": 86603723, "name": "ryan", "screen_name": "ryan_mcbabe", "lang": "en", "location": "dhs", "create_at": date("2009-10-31"), "description": "where's my kindle", "followers_count": 692, "friends_count": 431, "statues_count": 14207 }, "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-01-03T10:00:07.000Z"), "id": 683709437568266240, "text": "Holmdel Townshi is hiring! SUPERINTENDENT #jobs in HOLMDEL Apply today https://t.co/U75jGXFUdY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.172726,40.37571"), "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": 168, "friends_count": 42, "statues_count": 72042 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Holmdel, NJ", "id": "00edde1d2ec0a4d4", "name": "Holmdel", "place_type": "city", "bounding_box": rectangle("-74.209064,40.329842 -74.133116,40.426989") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth" } }
+{ "create_at": datetime("2016-01-03T10:00:07.000Z"), "id": 683709437756874753, "text": "Interested in a #job near #Kahului, HI? This could be a great fit: https://t.co/VjN5kQ6Nj0 #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-156.454708,20.881571"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Kahului", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 38082277, "name": "TMJ-MAU Jobs", "screen_name": "tmj_mau_jobs", "lang": "en", "location": "Maui, HI", "create_at": date("2009-05-05"), "description": "Follow this account for geo-targeted Other job tweets in Maui, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 547, "friends_count": 535, "statues_count": 16 }, "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-03T10:00:07.000Z"), "id": 683709437874278400, "text": "If I turn into the old QUAY no one is safe except @pimpdaddy_jass @DarrelSwavey and @DamnnWill ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2595289860, 403097985, 341394622 }}, "user": { "id": 62650045, "name": "FRENCHY!!!!", "screen_name": "saucexgod", "lang": "en", "location": "DFW, Texas", "create_at": date("2009-08-03"), "description": "18-Don't Mind My Tweets I'm just Talking....#QuayPose, A Male TwitterHoney | OMKS | #JassGang | Roll Tide!!!", "followers_count": 3886, "friends_count": 1580, "statues_count": 28908 }, "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-03T10:00:07.000Z"), "id": 683709438331490305, "text": "We're #hiring! Click to apply: STAFF AUDIOLOGIST - https://t.co/Tc3LzJz9yB #Healthcare #Honolulu, HI #Veterans #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Healthcare", "Honolulu", "Veterans", "Job", "Jobs", "CareerArc" }}, "user": { "id": 23526678, "name": "TMJ- HON Health Jobs", "screen_name": "tmj_hon_health", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Healthcare job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 410, "friends_count": 304, "statues_count": 191 }, "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-03T10:00:07.000Z"), "id": 683709438390366208, "text": "How did she get into the party in the first place? Fake ID or? Lol https://t.co/O6ZO8Teak7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29784055, "name": "ImNotANicePerson", "screen_name": "Ms_MWinston", "lang": "en", "location": "Queens,NY", "create_at": date("2009-04-08"), "description": "Pisces♓,Ciara, ToniBraxton❤️ Love4Art, GiantsFan❤️ C: quoted on 5-11-15 & TB: quoted on 3-29-15", "followers_count": 425, "friends_count": 310, "statues_count": 28847 }, "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-03T10:00:07.000Z"), "id": 683709438465671168, "text": "Cloudy this afternoon, high 55 (13 C). Low 43 (6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712593, "name": "CorpusChristiWX", "screen_name": "CorpusChristiWX", "lang": "en", "location": "Corpus Christi, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Corpus Christi, Texas", "followers_count": 1014, "friends_count": 93, "statues_count": 8111 }, "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-03T10:00:07.000Z"), "id": 683709438625202177, "text": "Headway Workfor is hiring! Bilingual Field #jobs in BETSYTOWN Apply today https://t.co/rGyR5rX1zx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.177876,40.67229"), "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": 168, "friends_count": 42, "statues_count": 72043 }, "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-03T10:00:07.000Z"), "id": 683709438671347712, "text": "@jescraven sexy", "in_reply_to_status": 683679516384833536, "in_reply_to_user": 42780496, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 42780496 }}, "user": { "id": 839063761, "name": "#RE2P3CT", "screen_name": "chrishinca42", "lang": "en", "location": "buffalo NY", "create_at": date("2012-09-21"), "description": "null", "followers_count": 417, "friends_count": 157, "statues_count": 24547 }, "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-03T10:00:07.000Z"), "id": 683709438788792321, "text": "Let's go Cowboys https://t.co/HbZPK52E3a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 44278640, "name": "Bryan J Ngati", "screen_name": "Cowboysnation09", "lang": "en", "location": "#Lanham MD ", "create_at": date("2009-06-02"), "description": "#FOE #Wizards ##CSC #SingleLife #God #CowboysNation", "followers_count": 1094, "friends_count": 2007, "statues_count": 142644 }, "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-03T10:00:07.000Z"), "id": 683709438792962048, "text": "The coupes impressive but I come with more features��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 514879562, "name": "Cody Lumpkin", "screen_name": "codylump_king", "lang": "en", "location": "Piqua, Ohio", "create_at": date("2012-03-04"), "description": "#ALPHA", "followers_count": 423, "friends_count": 173, "statues_count": 9682 }, "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-03T10:00:07.000Z"), "id": 683709438847545345, "text": "@ShutYaWeakAssUp I woulda died right there bro I'm sorry you woulda been tight wit me I woulda been so extra", "in_reply_to_status": 683709230264668160, "in_reply_to_user": 170755619, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 170755619 }}, "user": { "id": 427154865, "name": "PFM KICKSTANDBOY", "screen_name": "TikeaceHarris", "lang": "en", "location": "null", "create_at": date("2011-12-02"), "description": "IM ALL ABOUT A GOOD TIME PFM BOOK US IF YOU GOT A PARTY COMING UP WE WILL GET IT POP FOR THE RIGHT PRICE.....", "followers_count": 1697, "friends_count": 1132, "statues_count": 37793 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milford, DE", "id": "ee3ec8d5e41e2677", "name": "Milford", "place_type": "city", "bounding_box": rectangle("-75.45787,38.886046 -75.40221,38.942683") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10005, "countyName": "Sussex", "cityID": 1047420, "cityName": "Milford" } }
+{ "create_at": datetime("2016-01-03T10:00:07.000Z"), "id": 683709438956404737, "text": "We're #hiring! Click to apply: Electrician - https://t.co/uJFDvFH9JJ #SkilledTrade #electrician #Gillette, WY #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.5022205,44.2910915"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "SkilledTrade", "electrician", "Gillette", "Job", "Jobs" }}, "user": { "id": 3011572896, "name": "Cloud Peak Jobs", "screen_name": "CloudPeakJobs", "lang": "en", "location": "null", "create_at": date("2015-02-06"), "description": "Cloud Peak Energy aims to be a leading energy producer operating in a safe, responsible and caring manner. Check here for current job openings.", "followers_count": 56, "friends_count": 270, "statues_count": 15 }, "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-01-03T10:00:07.000Z"), "id": 683709438981730306, "text": "Who wanna go get some food when I get off? 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": 146695829, "name": "Sid ✨", "screen_name": "sid_bootie", "lang": "en", "location": "Champaign, IL", "create_at": date("2010-05-21"), "description": "#JayasWorld. Rest In Peace TeTe ❤️ Parkland. snapchat & insta : sidneytyler", "followers_count": 2431, "friends_count": 1212, "statues_count": 170213 }, "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-03T10:00:07.000Z"), "id": 683709439262588928, "text": "Oh. I missed this... https://t.co/pCDCD4tipA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 358357765, "name": "Alyssa Hansen", "screen_name": "scandalyssa419", "lang": "en", "location": "Irvine, CA", "create_at": date("2011-08-19"), "description": "Darling I'm a nightmare dressed like a daydream.", "followers_count": 221, "friends_count": 651, "statues_count": 4466 }, "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-03T10:00:07.000Z"), "id": 683709439287808000, "text": "@uhmaruh_06 thanks��", "in_reply_to_status": 683708044639744000, "in_reply_to_user": 3322205063, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3322205063 }}, "user": { "id": 381054167, "name": "Kevin", "screen_name": "kgalarza7", "lang": "en", "location": "null", "create_at": date("2011-09-27"), "description": "Waukegan Sc: kevinsito7#", "followers_count": 1009, "friends_count": 887, "statues_count": 38189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waukegan, IL", "id": "b819c5d90b780b57", "name": "Waukegan", "place_type": "city", "bounding_box": rectangle("-87.96368,42.305624 -87.802772,42.431936") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1779293, "cityName": "Waukegan" } }
+{ "create_at": datetime("2016-01-03T10:00:07.000Z"), "id": 683709439287902208, "text": "10 minutes out of the way for #Starbucks on the way to the rink. The things info for my little white girl. #coffee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.771057,41.468282"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Starbucks", "coffee" }}, "user": { "id": 274776380, "name": "stephanne", "screen_name": "stephanne12", "lang": "en", "location": "cleveland, ohio", "create_at": date("2011-03-30"), "description": "null", "followers_count": 135, "friends_count": 571, "statues_count": 4028 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Starbucks", "id": "07d9db7bf8883000", "name": "Starbucks", "place_type": "poi", "bounding_box": rectangle("-81.7710571,41.4682819 -81.771057,41.468282") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3941664, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-01-03T10:00:07.000Z"), "id": 683709439493455872, "text": "@ChrisOflyng you make me happ", "in_reply_to_status": -1, "in_reply_to_user": 532555457, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 532555457 }}, "user": { "id": 1470755810, "name": "mae #2", "screen_name": "chrisobling", "lang": "en", "location": "null", "create_at": date("2013-05-30"), "description": "alright dad", "followers_count": 106, "friends_count": 71, "statues_count": 19309 }, "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-01-03T10:00:07.000Z"), "id": 683709439598174208, "text": "I haven't eaten since New Year'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": 393165067, "name": "Cheif Queef", "screen_name": "SwaglessDillon", "lang": "en", "location": "San Antonio, TX", "create_at": date("2011-10-17"), "description": "i like pizza rolls and long walks on the beach Proud American", "followers_count": 3807, "friends_count": 3386, "statues_count": 65467 }, "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-03T10:00:07.000Z"), "id": 683709439753469952, "text": "Wowee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3354360850, "name": "Mutant", "screen_name": "Trashy_YungRat", "lang": "en", "location": "Florida, USA", "create_at": date("2015-07-01"), "description": "Artist and Photographer. Bass guitarist.Full time rat. Beme: rat-man Instagram: yung_rat. WARNING: SHITPOSTER(sometimes)", "followers_count": 58, "friends_count": 169, "statues_count": 1638 }, "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-03T10:00:07.000Z"), "id": 683709439795400704, "text": "A cornacopia of cool toys #rva #azaleafleamarket #richmondva #iselltoys @ Azalea Flea… https://t.co/x0Z4vrJYG5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.4232874,37.60420627"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "hashtags": {{ "rva", "azaleafleamarket", "richmondva", "iselltoys" }}, "user": { "id": 18808991, "name": "♕ mareesa dawn", "screen_name": "1marketingqueen", "lang": "en", "location": "RVA / world-wide", "create_at": date("2009-01-09"), "description": "**The worst thing you can be in business is INVISIBLE!!! I'm NOT Invisible.. Social Media Manager - Handcrafted earring designer .. i luv #RVA", "followers_count": 5441, "friends_count": 4326, "statues_count": 41849 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glen Allen, VA", "id": "2c98dd736bb2e02f", "name": "Glen Allen", "place_type": "city", "bounding_box": rectangle("-77.527806,37.579878 -77.407641,37.694137") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51087, "countyName": "Henrico", "cityID": 5131200, "cityName": "Glen Allen" } }
+{ "create_at": datetime("2016-01-03T10:00:07.000Z"), "id": 683709439824781313, "text": "Delavau is hiring! Systems Analyst #jobs in PHILADELPHIA Apply today https://t.co/rVWob8h77g", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.1644,39.9525"), "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": 168, "friends_count": 42, "statues_count": 72044 }, "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-03T10:00:07.000Z"), "id": 683709439833059328, "text": "Me: I'm so sad you're leaving. \nYev: Really hard to have an emotional moment with you wearing just that goddamn eye patch.Put on some pants.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15418895, "name": "Mark", "screen_name": "MShrayber", "lang": "en", "location": "San Francisco", "create_at": date("2008-07-13"), "description": "It's just the next level of how great I can be.-Farrah Abraham; m.shrayber@gmail.com for tips and friendship.", "followers_count": 3493, "friends_count": 946, "statues_count": 9148 }, "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-03T10:00:07.000Z"), "id": 683709439904497666, "text": "lmaooooooo https://t.co/CHjopppUUv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 175597817, "name": "cuss bradley", "screen_name": "med11n", "lang": "en", "location": "904lorida", "create_at": date("2010-08-06"), "description": "''he's an asshole online but he's lovely in real life'' aspiring Stanford band member", "followers_count": 2278, "friends_count": 559, "statues_count": 41213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jacksonville Beach, FL", "id": "5e281c17a74c170f", "name": "Jacksonville Beach", "place_type": "city", "bounding_box": rectangle("-81.429213,30.252059 -81.380497,30.307701") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12031, "countyName": "Duval", "cityID": 1235050, "cityName": "Jacksonville Beach" } }
+{ "create_at": datetime("2016-01-03T10:00:07.000Z"), "id": 683709440000835584, "text": "https://t.co/FAZ3vBt5xJ via youtube @USCellular #funny #ads https://t.co/Ryoix7zZlA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "funny", "ads" }}, "user_mentions": {{ 274538956 }}, "user": { "id": 21156125, "name": "TODD NASCA", "screen_name": "toddnasca", "lang": "en", "location": "LA", "create_at": date("2009-02-17"), "description": "CHILLIN' in SoCal! Dad in the Award Winning GEICO UNSKIPPABLE FAMILY and Dancing in Award Winning #EPICSTRUT! Plus so much more to talk about!! #Cheers!", "followers_count": 430, "friends_count": 514, "statues_count": 1846 }, "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-03T10:00:07.000Z"), "id": 683709440160215041, "text": "@Yung_Julio11 gotcha this year", "in_reply_to_status": 683709388469567490, "in_reply_to_user": 1332493554, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1332493554 }}, "user": { "id": 2714622977, "name": "Corey Straughter", "screen_name": "Cs21_era", "lang": "en", "location": "Monroe, LA", "create_at": date("2014-07-16"), "description": "4⭐️Student athlete, Neville High School.", "followers_count": 1628, "friends_count": 797, "statues_count": 13030 }, "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-03T10:00:07.000Z"), "id": 683709440202125312, "text": "@grantgash11 amirite ����", "in_reply_to_status": 683709136219942912, "in_reply_to_user": 284310267, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 284310267 }}, "user": { "id": 204996065, "name": "Haley Brown", "screen_name": "_hbrown00", "lang": "en", "location": "AR➡️HTX", "create_at": date("2010-10-19"), "description": "TRC•Lavaca Alumni•UAFS•Insta: _hbrown00•Gamma Phi Beta Alum", "followers_count": 1473, "friends_count": 561, "statues_count": 37781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-03T10:00:07.000Z"), "id": 683709440273457152, "text": "@MRKGS_ �� got sum to say?", "in_reply_to_status": 683708939310133249, "in_reply_to_user": 2431308788, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2431308788 }}, "user": { "id": 2989257246, "name": "unorthodox", "screen_name": "jdrosariia", "lang": "en", "location": "johnston, ia", "create_at": date("2015-01-18"), "description": "let em hate, stay calm 'n meditate |ㄣ⃒ #blacklivesmatter", "followers_count": 1127, "friends_count": 1123, "statues_count": 40401 }, "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-03T10:00:07.000Z"), "id": 683709440374132737, "text": "@lgbeapart God created us with need. Before sin, there was hunger, thirst, loneliness. God gave us the fulfillment to every need - Himself", "in_reply_to_status": -1, "in_reply_to_user": 2156036993, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2156036993 }}, "user": { "id": 40923323, "name": "Ron Tonkin", "screen_name": "CaptRonT", "lang": "en", "location": "null", "create_at": date("2009-05-18"), "description": "null", "followers_count": 34, "friends_count": 36, "statues_count": 262 }, "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-03T10:00:07.000Z"), "id": 683709440428785666, "text": "Shits can't be real bruh @MightyJoYungin https://t.co/dXQVZLGnMJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 203785630 }}, "user": { "id": 228389157, "name": "リュ", "screen_name": "Felix_Bordeaux", "lang": "en", "location": "Orlando, FL", "create_at": date("2010-12-19"), "description": "KNM 22", "followers_count": 560, "friends_count": 187, "statues_count": 60762 }, "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-01-03T10:00:07.000Z"), "id": 683709440478941184, "text": "@_Robert_Bryant_ The articles I've seen, and there are many, have him asking for $20,000 a game over 10 years= 1.6 million dollars. 2 much.", "in_reply_to_status": 683697220311486465, "in_reply_to_user": 399751441, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 399751441 }}, "user": { "id": 4618406119, "name": "Gavin C. Brown", "screen_name": "gavin_c_brown", "lang": "en", "location": "New Orleans, LA", "create_at": date("2015-12-20"), "description": "Professor of English - Writer - Speaker - Finding the Balance between Righteousness and Relaxation", "followers_count": 186, "friends_count": 985, "statues_count": 513 }, "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-03T10:00:07.000Z"), "id": 683709440516734978, "text": "10 seconds!!! @Patriots", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.31973975,34.26004661"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 31126587 }}, "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": 384, "friends_count": 346, "statues_count": 61465 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-03T10:00:07.000Z"), "id": 683709440558673920, "text": "#CareerArc #Hospitality #Job alert: Shift Supervisor (US) | Starbucks | https://t.co/1fGY81U7Pa #Veterans #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.9283198,21.3950377"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CareerArc", "Hospitality", "Job", "Veterans", "Jobs", "Hiring" }}, "user": { "id": 23897767, "name": "TMJ-HON Hosp. Jobs", "screen_name": "tmj_hon_hrta", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-11"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 410, "friends_count": 292, "statues_count": 264 }, "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-01-03T10:00:08.000Z"), "id": 683709440776888320, "text": "#HereWeGo #Steelers&Bills! #steelersbetterlive", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HereWeGo", "Steelers", "steelersbetterlive" }}, "user": { "id": 16569686, "name": "Karyl", "screen_name": "karylc", "lang": "en", "location": "Franconia, VA", "create_at": date("2008-10-02"), "description": "There is nothing, absolutely nothing, half so much worth doing as messing about in boats.", "followers_count": 385, "friends_count": 457, "statues_count": 2822 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redland, MD", "id": "7c38fd77def3b77e", "name": "Redland", "place_type": "city", "bounding_box": rectangle("-77.185672,39.118326 -77.12162,39.172422") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2465312, "cityName": "Redland" } }
+{ "create_at": datetime("2016-01-03T10:00:08.000Z"), "id": 683709440990691328, "text": "\"Love dnt live here anymore\" ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2568214406, "name": "keep it 1000", "screen_name": "Stayy_blessed", "lang": "en", "location": "Ten Toes Down ent.", "create_at": date("2014-06-14"), "description": "Pray Hard Grind Hard God first then fam free brando, free mark. Ima nobody until Ican shine -Z-Ro", "followers_count": 467, "friends_count": 721, "statues_count": 9608 }, "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-01-03T10:00:08.000Z"), "id": 683709440995028992, "text": "Started with my girlfriend, ended with her homies, started out hustlin, ended up ballin' ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 430831785, "name": "Will", "screen_name": "will_semble", "lang": "en", "location": "New Orleans, LA", "create_at": date("2011-12-07"), "description": "NY || TU '19", "followers_count": 371, "friends_count": 297, "statues_count": 7261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New City, NY", "id": "4993b962771646cb", "name": "New City", "place_type": "city", "bounding_box": rectangle("-74.025285,41.107441 -73.943839,41.192559") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36087, "countyName": "Rockland", "cityID": 3650100, "cityName": "New City" } }
+{ "create_at": datetime("2016-01-03T10:00:08.000Z"), "id": 683709441133383682, "text": "I really stay shut for reasons", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 76817337, "name": "Jan. 18 #KAZWORLD ❤️", "screen_name": "nandidollx3", "lang": "en", "location": "null", "create_at": date("2009-09-23"), "description": "Announce the miracle, not the saint ✨ RIP to my guardian angels Kanil, Najer, Brian & Kazz #BBLU", "followers_count": 660, "friends_count": 1167, "statues_count": 59761 }, "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-03T10:00:08.000Z"), "id": 683709441200398336, "text": "Temp: 28.4°F - Dew Point: 18.2° - Wind: 10.3 mph - Gust: 16.6 - Rain Today: 0.00in. - Pressure: 30.19in, - 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": 11471 }, "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-03T10:00:08.000Z"), "id": 683709441217277952, "text": "To #OOMF �� https://t.co/ydcPVxH2Cp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "OOMF" }}, "user": { "id": 260561115, "name": "Jan. 27th", "screen_name": "Travo_Mc", "lang": "en", "location": "Paris, Ile-de-France", "create_at": date("2011-03-03"), "description": "⚽ #MesQueUnSport #757 #TGOD", "followers_count": 610, "friends_count": 458, "statues_count": 26272 }, "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-03T10:00:08.000Z"), "id": 683709441233977344, "text": "@FreeGuwop bro I swear lmao I'm like wtf I just made a statement", "in_reply_to_status": 683709306630438913, "in_reply_to_user": 260436935, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 260436935 }}, "user": { "id": 3226822026, "name": "Ω̴̩̩̩̥VΩ̴̩̩̩̥", "screen_name": "AndraeO_o", "lang": "en", "location": "Long Beach, CA ✈️ Dallas Tx", "create_at": date("2015-05-25"), "description": "Dreams Money Can Buy Free My Brother @NikeStreetz #DubNation", "followers_count": 1650, "friends_count": 393, "statues_count": 49517 }, "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-03T10:00:08.000Z"), "id": 683709441347198977, "text": "@MokounoME dude building those things is easy", "in_reply_to_status": 683660512836628480, "in_reply_to_user": 95249768, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 95249768 }}, "user": { "id": 118920200, "name": "Derek", "screen_name": "Kitastrophee", "lang": "en", "location": "Loveland", "create_at": date("2010-03-01"), "description": "whats the matter. could it b ur craving my mctweets", "followers_count": 122, "friends_count": 91, "statues_count": 24522 }, "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-03T10:00:08.000Z"), "id": 683709441548529664, "text": "Sunny this afternoon, high 45 (7 C). Low 25 (-4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712706, "name": "TulsaWX", "screen_name": "TulsaWX", "lang": "en", "location": "Tulsa, Oklahoma", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Tulsa, Oklahoma", "followers_count": 517, "friends_count": 93, "statues_count": 8173 }, "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-03T10:00:08.000Z"), "id": 683709441787703296, "text": "@tay_trot07 and I are so excited to see @RachelWinters21 and @madz96schup soon!!! ❤️��❤️ #hurryup #wecantwait", "in_reply_to_status": -1, "in_reply_to_user": 1273912632, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hurryup", "wecantwait" }}, "user_mentions": {{ 1273912632, 355763300, 261878172 }}, "user": { "id": 1384742587, "name": "Sue Trotter", "screen_name": "Sue_Trotter", "lang": "en", "location": "null", "create_at": date("2013-04-27"), "description": "null", "followers_count": 89, "friends_count": 63, "statues_count": 1670 }, "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-01-03T10:00:08.000Z"), "id": 683709441963757568, "text": "@JoshEiniger7: #HiJosh, #CanIPleaseGetAFollowBack! #ThanksJoshFrom, #SoCal...��������", "in_reply_to_status": -1, "in_reply_to_user": 389582305, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "HiJosh", "CanIPleaseGetAFollowBack", "ThanksJoshFrom", "SoCal" }}, "user_mentions": {{ 389582305 }}, "user": { "id": 567426853, "name": "Lita Barron Sheehan", "screen_name": "LitaESheehan", "lang": "en", "location": "Alta Loma, Ca. ", "create_at": date("2012-04-30"), "description": "MomOfKevin❤28 ProudOfMyGaySonKirk❤23⛪LoveGod⛪ImCatholic❤SoCal Born❤Raised❤NewsJunkie @RobertNBCLA❤GreatFriend❤ @PaulMagers:LuvUrBlueEyes❤I know❤@JoshMankiewicz❤", "followers_count": 591, "friends_count": 431, "statues_count": 37445 }, "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-03T10:00:08.000Z"), "id": 683709442009899008, "text": "WHY IS PLEASANTON GOING BACK TO SCHOOL WEDNESDAY WTF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3247833884, "name": "daddy", "screen_name": "dvshxun_", "lang": "en", "location": "texa$", "create_at": date("2015-06-17"), "description": "bitch i got the sauce", "followers_count": 203, "friends_count": 120, "statues_count": 3880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Poteet, TX", "id": "0e4060f232480ec9", "name": "Poteet", "place_type": "city", "bounding_box": rectangle("-98.58807,29.025442 -98.561054,29.050464") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48013, "countyName": "Atascosa", "cityID": 4859084, "cityName": "Poteet" } }
+{ "create_at": datetime("2016-01-03T10:00:08.000Z"), "id": 683709442031009793, "text": "i swear i love food mane ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 86588107, "name": "R A C H E L❣", "screen_name": "ok_sing_", "lang": "en", "location": "Richmond, VA", "create_at": date("2009-10-31"), "description": "20. ferrumcollege18. TraceFace❤️", "followers_count": 1348, "friends_count": 1638, "statues_count": 17613 }, "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-03T10:00:08.000Z"), "id": 683709442093903874, "text": "Gaines is inactive????? So Gilbert and Bademosi at the corners???? Pittsburgh may win by 50.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54319144, "name": "Steele", "screen_name": "JJSteele3", "lang": "en", "location": "New York City", "create_at": date("2009-07-06"), "description": "Akron----SF---- NYC. Work @EW ... Les Fleurs du Mal. Joe Haden, Binx Bolling, Craig Ehlo + Greg Dulli #BikeNYC", "followers_count": 416, "friends_count": 1088, "statues_count": 5618 }, "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-03T10:00:08.000Z"), "id": 683709442257362944, "text": "#Blessed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Blessed" }}, "user": { "id": 1941829435, "name": "A♡", "screen_name": "_xBrea", "lang": "en", "location": "at Chick fil a ", "create_at": date("2013-10-06"), "description": "I go to #SFA..", "followers_count": 637, "friends_count": 589, "statues_count": 8228 }, "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-01-03T10:00:08.000Z"), "id": 683709442311983105, "text": "I could care less https://t.co/sazAGEXFDI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 356195220, "name": "G", "screen_name": "thatgirlcheny", "lang": "en", "location": "Austin, TX", "create_at": date("2011-08-16"), "description": "sales consultant at South Point Dodge | SC | IG--thatgirlcheny & genesissouthpointdodge", "followers_count": 357, "friends_count": 123, "statues_count": 15652 }, "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-03T10:00:08.000Z"), "id": 683709442446196740, "text": "It must suck to be easily offended", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18382259, "name": "lizzi", "screen_name": "ohlizzi", "lang": "en", "location": "null", "create_at": date("2008-12-25"), "description": "null", "followers_count": 350, "friends_count": 198, "statues_count": 21848 }, "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-03T10:00:08.000Z"), "id": 683709442559361024, "text": "Coney Island #coneyisland #beach #winter #beach @ Brighton Beach Area https://t.co/20AYi06AqT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.96044689,40.577846"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "coneyisland", "beach", "winter", "beach" }}, "user": { "id": 131959240, "name": "Carlos Condat", "screen_name": "Karluche", "lang": "en", "location": "Argentina", "create_at": date("2010-04-11"), "description": "null", "followers_count": 69, "friends_count": 84, "statues_count": 367 }, "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-03T10:00:08.000Z"), "id": 683709442655830016, "text": "But at least my Pats will continue to play on 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": 1424487122, "name": "Kyler Ashley", "screen_name": "kylerashley12", "lang": "en", "location": "null", "create_at": date("2013-05-12"), "description": "T.O.C.C alumni | Window Rock, Arizona | Navajo | GOD | Family | Instagram-elite12az | Facebook | snap-eliteka12 |", "followers_count": 1414, "friends_count": 891, "statues_count": 18470 }, "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-03T10:00:08.000Z"), "id": 683709443507269633, "text": "Can you recommend anyone for this #job? NOC Technician - https://t.co/PNHCvNwSk8 #Seattle, WA #IT #Hiring https://t.co/XdCACKFB1Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.3320708,47.6062095"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Seattle", "IT", "Hiring" }}, "user": { "id": 196929096, "name": "Jobs at Oracle", "screen_name": "JobsAtOracle", "lang": "en", "location": "Redwood Shores, CA", "create_at": date("2010-09-30"), "description": "The global feed for jobs at Oracle. #OracleTalent", "followers_count": 3906, "friends_count": 2295, "statues_count": 62934 }, "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-03T10:00:08.000Z"), "id": 683709443691950081, "text": "I’m @ Rooftop Sunday’s Tonight, Doe ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28923411, "name": "Seymour F. Dollars™", "screen_name": "DJ_BMONEY", "lang": "en", "location": "SoFlo ✈️ LA ", "create_at": date("2009-04-04"), "description": "#AppleGenius  #FIU #UnEmployed #OhYouGonSeeThisSixPack #TeamNoTextBackButImOnTwitterTho Booking:BoomingEntServices@gmail.com", "followers_count": 2243, "friends_count": 863, "statues_count": 205648 }, "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-01-03T10:00:08.000Z"), "id": 683709443763269632, "text": "I usually don't post any food related stuff but this was a must! Central/Caribbean food! Pupusas + plátano frito = �� https://t.co/iuWnjUxPla", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 92945562, "name": "Edwin Ortiz López", "screen_name": "Gatoortiz13", "lang": "en", "location": "Santa Ana, CA", "create_at": date("2009-11-27"), "description": "Follow your dreams. Reach your goals. Have faith. Belive in you. Persevere. Work hard. Make sacrifices. Help others. Be humble. Be thankful!", "followers_count": 75, "friends_count": 77, "statues_count": 899 }, "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-03T10:00:08.000Z"), "id": 683709444014780417, "text": "I told my dad I would get out of bed before one today, those odds aren't looking 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": 2419670263, "name": "b", "screen_name": "bellalouisee4", "lang": "en", "location": "wi ✈️ az", "create_at": date("2014-03-30"), "description": "dvhs | club one volleyball | @zhawrylkiw ❤️", "followers_count": 471, "friends_count": 927, "statues_count": 1678 }, "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-03T10:00:08.000Z"), "id": 683709444098662400, "text": "@Estefaniariel_ that shit looked poppin all over my snapxhat �������� I wish I coudlve got ratch with y'all & when do y'all leave ?", "in_reply_to_status": 683690219560808448, "in_reply_to_user": 2835695401, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2835695401 }}, "user": { "id": 193176243, "name": "E.", "screen_name": "ehx96", "lang": "en", "location": "LA - $D", "create_at": date("2010-09-20"), "description": "null", "followers_count": 2112, "friends_count": 235, "statues_count": 66809 }, "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-03T10:00:08.000Z"), "id": 683709444107141120, "text": "fuck niggas stay away from��✋��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 724452781, "name": "livinglavish♉️✨", "screen_name": "AmbriaW_", "lang": "en", "location": "Newberry, SC", "create_at": date("2012-07-29"), "description": "snapchat: Ambria_34 ❣", "followers_count": 3388, "friends_count": 3439, "statues_count": 50790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newberry, SC", "id": "a83e4dd26ae7b004", "name": "Newberry", "place_type": "city", "bounding_box": rectangle("-81.657544,34.255599 -81.571202,34.315219") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45071, "countyName": "Newberry", "cityID": 4549570, "cityName": "Newberry" } }
+{ "create_at": datetime("2016-01-03T10:00:08.000Z"), "id": 683709444123828224, "text": "https://t.co/b7VbZNIkd2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3184374859, "name": "Perry Fedorow", "screen_name": "FedorowPerry", "lang": "en", "location": "null", "create_at": date("2015-05-03"), "description": "Men Anything goes Photography. I love muscular men and I really like a mans chest and nipples oh and a nice butt. The AV picture is me. Adult content be over 18", "followers_count": 7657, "friends_count": 6992, "statues_count": 3858 }, "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-03T10:00:08.000Z"), "id": 683709444211912708, "text": "Sunny this afternoon, high 45 (7 C). Low 25 (-4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712679, "name": "OKCWX", "screen_name": "OKCWX", "lang": "en", "location": "Oklahoma City, Oklahoma", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Oklahoma City, Oklahoma", "followers_count": 185, "friends_count": 93, "statues_count": 8200 }, "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-03T10:00:08.000Z"), "id": 683709444274827264, "text": "God Speed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 881062310, "name": "Autumn JKR", "screen_name": "MindOfJKR", "lang": "en", "location": "GT - Naive JKR", "create_at": date("2012-10-14"), "description": "Dann | 14 | Sniper for @AutumnArises @AutumnDrakos", "followers_count": 1672, "friends_count": 2603, "statues_count": 5420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Billings, MT", "id": "662aa8db4557a744", "name": "Billings", "place_type": "city", "bounding_box": rectangle("-108.692983,45.723722 -108.432965,45.871169") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30111, "countyName": "Yellowstone", "cityID": 3006550, "cityName": "Billings" } }
+{ "create_at": datetime("2016-01-03T10:00:08.000Z"), "id": 683709444388184064, "text": "⛅️��⛅️�� your chill, and I hope you had a good time New Years, we need to hang out more often!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1561270332, "name": "Lord Jyeezy", "screen_name": "jdbullard02", "lang": "en", "location": "bullards trap", "create_at": date("2013-07-01"), "description": "gehs 18'| Flash Elite⚾️| sc~ jacobullard2 | Don't be a King, be a God", "followers_count": 269, "friends_count": 254, "statues_count": 2304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Summerside, OH", "id": "cf09a5a9de326c9a", "name": "Summerside", "place_type": "city", "bounding_box": rectangle("-84.307145,39.0888 -84.216213,39.157379") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39025, "countyName": "Clermont", "cityID": 3975434, "cityName": "Summerside" } }
+{ "create_at": datetime("2016-01-03T10:00:08.000Z"), "id": 683709444547555328, "text": "https://t.co/j7VjvjWy4i", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 130545639, "name": "$teelo PyraDice", "screen_name": "pyrex_Guapo", "lang": "en", "location": "#dirtyjerz", "create_at": date("2010-04-07"), "description": "Music Artist/photography/inquiries,contact: Gmail: straightlaced456@gmail.com instagram: steezalito #Play2win. #$traightlaced", "followers_count": 386, "friends_count": 353, "statues_count": 960 }, "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-03T10:00:08.000Z"), "id": 683709444610379776, "text": "Aren't you a little fat to be a storm trooper? @joeynoland ������ https://t.co/4sqEJWJ98x", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 256596747 }}, "user": { "id": 1428430585, "name": "Jordan Hester", "screen_name": "TheNames_Hester", "lang": "en", "location": "null", "create_at": date("2013-05-14"), "description": "Retired Student-Athlete | FHSU", "followers_count": 398, "friends_count": 218, "statues_count": 6446 }, "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-01-03T10:00:08.000Z"), "id": 683709444748775424, "text": "someone come get me before I murder cache", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2615716805, "name": "мιкαуℓα ☠", "screen_name": "mikaylafreeman4", "lang": "en", "location": "Sulphur, OK", "create_at": date("2014-06-16"), "description": "I'm hungry and I just ate.", "followers_count": 243, "friends_count": 339, "statues_count": 6219 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sulphur, OK", "id": "7fe71783bb4b2eed", "name": "Sulphur", "place_type": "city", "bounding_box": rectangle("-96.998351,34.491603 -96.949858,34.526851") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40099, "countyName": "Murray", "cityID": 4071350, "cityName": "Sulphur" } }
+{ "create_at": datetime("2016-01-03T10:00:08.000Z"), "id": 683709444761497600, "text": "@TheBrightEffect same", "in_reply_to_status": 683701867201839104, "in_reply_to_user": 48231555, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 48231555 }}, "user": { "id": 363839165, "name": "Noelle Keith", "screen_name": "noelle_keith", "lang": "en", "location": "null", "create_at": date("2011-08-28"), "description": "like Christmas", "followers_count": 478, "friends_count": 308, "statues_count": 7729 }, "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-03T10:00:09.000Z"), "id": 683709444811812864, "text": "thank god get her off my back https://t.co/SnsRKOr0Xo", "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": "null", "create_at": date("2011-03-20"), "description": "null", "followers_count": 7262, "friends_count": 1381, "statues_count": 289155 }, "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-01-03T10:00:09.000Z"), "id": 683709444853727233, "text": "@ClubeGaffer it's not terrorism yet. Not even close. Some people are obsessed with it being terrorism & as a liberal myself, I don't get it", "in_reply_to_status": 683651536333848576, "in_reply_to_user": 29922585, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 29922585 }}, "user": { "id": 854024965, "name": "Crom", "screen_name": "CromsCorner", "lang": "en", "location": "United States", "create_at": date("2012-09-29"), "description": "Crom's Corner - a fan blog made out of purple pride. Not for profit. Please check it out! SKOL!", "followers_count": 674, "friends_count": 989, "statues_count": 14333 }, "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-03T10:00:09.000Z"), "id": 683709445008830464, "text": "#Nursing in #Beachwood, OH: State Tested Nursing Assistant - $500... at Menorah Park Center for Senior L... https://t.co/N1EZWlu20e #STNA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4899136,41.5011199"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "Beachwood", "STNA" }}, "user": { "id": 214122801, "name": "MenorahParkJobs", "screen_name": "MenorahParkJobs", "lang": "en", "location": "Beachwood, Ohio", "create_at": date("2010-11-10"), "description": "Menorah Park Center for Senior Living is one of the largest senior living communities in the state of Ohio.", "followers_count": 133, "friends_count": 82, "statues_count": 1380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beachwood, OH", "id": "fa5bd676191716ee", "name": "Beachwood", "place_type": "city", "bounding_box": rectangle("-81.532635,41.449669 -81.48037,41.501448") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3904500, "cityName": "Beachwood" } }
+{ "create_at": datetime("2016-01-03T10:00:09.000Z"), "id": 683709445361106946, "text": "Want to work in #Honolulu, HI? View our latest opening: https://t.co/l6rMf1Gw2Z #Manufacturing #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Honolulu", "Manufacturing", "Job", "Jobs", "Hiring" }}, "user": { "id": 23527770, "name": "TMJ - HON Manuf Jobs", "screen_name": "tmj_hon_manuf", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Manufacturing job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 224, "friends_count": 202, "statues_count": 1 }, "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-03T10:00:09.000Z"), "id": 683709445499559936, "text": "@chrisgendo Is that...my late husband?", "in_reply_to_status": 683482991700541440, "in_reply_to_user": 2399219582, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2399219582 }}, "user": { "id": 1716427862, "name": "Engel♡", "screen_name": "scott_aukerman", "lang": "en", "location": "the South", "create_at": date("2013-08-31"), "description": "❤ savannah / 15 / INTP / cis ♀ (she/her) / bi / the princess of memes / afrikan-amerikan / artist ❤", "followers_count": 247, "friends_count": 613, "statues_count": 12585 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cornelius, NC", "id": "6ac2e48ec4892d20", "name": "Cornelius", "place_type": "city", "bounding_box": rectangle("-80.946496,35.445762 -80.842478,35.507042") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3714700, "cityName": "Cornelius" } }
+{ "create_at": datetime("2016-01-03T10:00:09.000Z"), "id": 683709445600317440, "text": "@rappublicist Check out my brand new music video \"Smooth\" #EvolveCrew #Norfolk #KnowingBetter https://t.co/wCNRmlqet0", "in_reply_to_status": -1, "in_reply_to_user": 23771604, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "EvolveCrew", "Norfolk", "KnowingBetter" }}, "user_mentions": {{ 23771604 }}, "user": { "id": 21249017, "name": "Page", "screen_name": "Pagemuzik", "lang": "en", "location": "fethernfinbout2getaboneless VA", "create_at": date("2009-02-18"), "description": "Tryin to spread love & shit to the darkest reaches of the galaxy http://www.datpiff.com/pop-mixtape-download.php?id=m6d433ad", "followers_count": 2148, "friends_count": 380, "statues_count": 14965 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dix Hills, NY", "id": "18165a3f2b139f0f", "name": "Dix Hills", "place_type": "city", "bounding_box": rectangle("-73.380745,40.767592 -73.29345,40.83856") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3620687, "cityName": "Dix Hills" } }
+{ "create_at": datetime("2016-01-03T10:00:09.000Z"), "id": 683709445730246657, "text": "1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 972266546, "name": "$AUCIN'", "screen_name": "Francisco_txs80", "lang": "en", "location": "Houston, TX", "create_at": date("2012-11-26"), "description": "null", "followers_count": 582, "friends_count": 1561, "statues_count": 4591 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-03T10:00:09.000Z"), "id": 683709445948321794, "text": "hate carrying around cash cause I know eventually imma 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": 116183484, "name": "Dannasia✨", "screen_name": "been0riginal", "lang": "en", "location": "Romans 8:18", "create_at": date("2010-02-21"), "description": "null", "followers_count": 2358, "friends_count": 1111, "statues_count": 58367 }, "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-03T10:00:09.000Z"), "id": 683709446095114242, "text": "Forget you, Santa Fe. With your lame night life and love for breakfast. #ICantWithYou", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ICantWithYou" }}, "user": { "id": 88266208, "name": "Mr. Brightside", "screen_name": "maeday85", "lang": "en", "location": "El Paso, Texas", "create_at": date("2009-11-07"), "description": "Todo lo que fui es todo lo que soy. Con lo que vine ayer es con lo que me voy.", "followers_count": 83, "friends_count": 241, "statues_count": 2960 }, "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-03T10:00:09.000Z"), "id": 683709446149636096, "text": "@MoKhaLatte U play 2 win the game, plus Jalen Ramsey should still be available when they pick & they better draft him or Im filing 4 divorce", "in_reply_to_status": 683708465047408640, "in_reply_to_user": 35231173, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 35231173 }}, "user": { "id": 220664914, "name": "Big Truck ⓑⓡⓨⓐⓝ™", "screen_name": "BryanBHopkins1", "lang": "en", "location": "On a Interstate Somewhere USA", "create_at": date("2010-11-28"), "description": "An ignorant fool is a real cool nerd. II Blessed IV Stress", "followers_count": 1369, "friends_count": 1103, "statues_count": 62593 }, "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-03T10:00:09.000Z"), "id": 683709446237851648, "text": "Always missin ppl I shouldn't be missin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2244232661, "name": "blake lively", "screen_name": "eturner323", "lang": "en", "location": "Springfield, PA", "create_at": date("2013-12-25"), "description": "null", "followers_count": 370, "friends_count": 367, "statues_count": 3655 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Exton, PA", "id": "f05705aeb1de96db", "name": "Exton", "place_type": "city", "bounding_box": rectangle("-75.653372,40.014927 -75.607883,40.045387") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42029, "countyName": "Chester", "cityID": 4224440, "cityName": "Exton" } }
+{ "create_at": datetime("2016-01-03T10:00:09.000Z"), "id": 683709446355316739, "text": "Nobody Fw Jordan Period. Just Know That! 1000 . ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1202723562, "name": "KeDaddy", "screen_name": "__Kmeeks", "lang": "en", "location": "Tk,✈️ Chyna, Derek Hakeem ❤️ ", "create_at": date("2013-02-20"), "description": "-I Got Me, & That's On Me.❣ It's A Meeks Thing , They Wouldn't Understand ✊❤️", "followers_count": 1840, "friends_count": 715, "statues_count": 39376 }, "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.355819 -93.968792,33.507099") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48037, "countyName": "Bowie", "cityID": 4872368, "cityName": "Texarkana" } }
+{ "create_at": datetime("2016-01-03T10:00:09.000Z"), "id": 683709446485360641, "text": "#Boston nostalgia was so real! S/O to my #Phrat and Friend for hosting and feeding me ☺️… https://t.co/g3GGYU3f05", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.01867371,42.36516484"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Boston", "Phrat" }}, "user": { "id": 3303724202, "name": "Murehwa", "screen_name": "TamukaMartin", "lang": "en", "location": "null", "create_at": date("2015-08-01"), "description": "null", "followers_count": 43, "friends_count": 177, "statues_count": 231 }, "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-03T10:00:09.000Z"), "id": 683709446543949824, "text": "This #BusinessMgmt #job might be a great fit for you: Assistant General Manager - Orlando - https://t.co/V056OfbWEo #Orlando, FL #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3792365,28.5383355"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BusinessMgmt", "job", "Orlando", "Hiring" }}, "user": { "id": 2228943546, "name": "Ghirardelli Careers", "screen_name": "ghirardellijobs", "lang": "en", "location": "null", "create_at": date("2013-12-03"), "description": "Ghirardelli invites you to explore our open #jobs and discover a place where your ideas come to life, as part of our delicious legacy.", "followers_count": 51, "friends_count": 1, "statues_count": 84 }, "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-03T10:00:09.000Z"), "id": 683709446602625024, "text": "Secondline with my sisters @Checkbre_out @jiraychance ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1497214770, 3421399833 }}, "user": { "id": 3091704415, "name": "dec10_myday", "screen_name": "jayymika625", "lang": "en", "location": "null", "create_at": date("2015-03-16"), "description": "null", "followers_count": 84, "friends_count": 120, "statues_count": 2207 }, "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-03T10:00:09.000Z"), "id": 683709446803951616, "text": "@BFSEsq why does that emoji have no mouth and a unibrow? ��", "in_reply_to_status": 683708063329710085, "in_reply_to_user": 24836305, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24836305 }}, "user": { "id": 12597852, "name": "holly jolly salzman", "screen_name": "chrissalzman", "lang": "en", "location": "Ann Arbor, MI", "create_at": date("2008-01-23"), "description": "the hipster those bros warned you about and/or a spambot.", "followers_count": 411, "friends_count": 445, "statues_count": 12907 }, "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-03T10:00:09.000Z"), "id": 683709446804115457, "text": "When your mom is going to the Super Bowl and its in Cali 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": 544785696, "name": "Kandy❤️", "screen_name": "Kdstarks11", "lang": "en", "location": "864", "create_at": date("2012-04-03"), "description": "R.I.P Dequan Jones and Destiny Champion❤️ 864➡️803 #Winthrop➡️#Claflin", "followers_count": 1308, "friends_count": 1109, "statues_count": 38731 }, "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-01-03T10:00:09.000Z"), "id": 683709446820769792, "text": "@LP_LaPresse Les joueurs sont écoeurés de Jon Cooper.", "in_reply_to_status": 683708794581291008, "in_reply_to_user": 10681662, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user_mentions": {{ 10681662 }}, "user": { "id": 203375482, "name": "Gil Roy", "screen_name": "GilroyTX", "lang": "en", "location": "null", "create_at": date("2010-10-15"), "description": "null", "followers_count": 93, "friends_count": 1261, "statues_count": 18326 }, "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-03T10:00:09.000Z"), "id": 683709446942404608, "text": "Please don't leave your house if you don't know how to drive in the snow. You're not only endangering yourself.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 320069995, "name": "Kameron Anderson ", "screen_name": "Kamderson19", "lang": "en", "location": "Aloha", "create_at": date("2011-06-19"), "description": "Can't lives on won't street.", "followers_count": 187, "friends_count": 135, "statues_count": 11421 }, "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-03T10:00:09.000Z"), "id": 683709447043153920, "text": "#BillsMafia #Bills win today all ladies must post yoga pants pics #SexySundays\nhehe\n#NYJvsBUF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BillsMafia", "Bills", "SexySundays", "NYJvsBUF" }}, "user": { "id": 175503464, "name": "David Huggins", "screen_name": "buffdada", "lang": "en", "location": "Buffalo NY", "create_at": date("2010-08-06"), "description": "716, Buffalo Bills Fan, Navy Vet.\r\nFaith in God can take you Everywhere.\r\nAnd smoke it if you got it.", "followers_count": 337, "friends_count": 259, "statues_count": 22816 }, "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-03T10:00:09.000Z"), "id": 683709447286448128, "text": "greece loves you austin\n #GreeceNeedsAustin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GreeceNeedsAustin" }}, "user": { "id": 3485825175, "name": "ASecretMahomie", "screen_name": "merna358", "lang": "en", "location": "Miami", "create_at": date("2015-08-29"), "description": "As a Mahomie I am proud to call Austin my idol ❤✌", "followers_count": 47, "friends_count": 58, "statues_count": 1041 }, "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-03T10:00:09.000Z"), "id": 683709447399694336, "text": "@bribyrne11 LOL STOP", "in_reply_to_status": 683709342902693888, "in_reply_to_user": 444175254, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 444175254 }}, "user": { "id": 299162203, "name": "Abby Jansen", "screen_name": "abbyjansen", "lang": "en", "location": "De Pere, WI", "create_at": date("2011-05-15"), "description": "bucks in 6", "followers_count": 1052, "friends_count": 866, "statues_count": 26052 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howard, WI", "id": "fef70a352908470b", "name": "Howard", "place_type": "city", "bounding_box": rectangle("-88.162962,44.53006 -88.043216,44.612231") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55009, "countyName": "Brown", "cityID": 5535950, "cityName": "Howard" } }
+{ "create_at": datetime("2016-01-03T10:00:09.000Z"), "id": 683709447672336384, "text": "Watching the J E T S @MJOConnors", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 112007712 }}, "user": { "id": 147780862, "name": "Rosemarie McGeehan", "screen_name": "RosieMcGeehan", "lang": "en", "location": "Jersey Shore!", "create_at": date("2010-05-24"), "description": "Full-time Realtor ready to help you realize your dreams at the Jersey Shore. Love to spend time at the beach and Monmouth Park. A proud Jersey Girl!", "followers_count": 83, "friends_count": 58, "statues_count": 362 }, "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-03T10:00:09.000Z"), "id": 683709447969964032, "text": "Australian sheperds will forever be my weakness", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 285225348, "name": "Thór⚡️", "screen_name": "Collinalmighty_", "lang": "en", "location": "6'3", "create_at": date("2011-04-20"), "description": "SoI |NT| Marchvembers Very Own |", "followers_count": 676, "friends_count": 201, "statues_count": 18824 }, "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-03T10:00:09.000Z"), "id": 683709448427302912, "text": "Looks like I picked the wrong decade to quit drinking beer in Memphis. #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": 406345942, "name": "Steve Selvidge", "screen_name": "SteveSelvidge", "lang": "en", "location": "Memphis/Rock Tours", "create_at": date("2011-11-06"), "description": "I like to eat crackers and drink whiskey.", "followers_count": 1374, "friends_count": 328, "statues_count": 1072 }, "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-03T10:00:09.000Z"), "id": 683709448582479872, "text": "@pviolante1 @Sons_of_Italy @NOIAW @niaforg @emigrazione_ita @TwitterItalia \n\nWhat ship were you in ?", "in_reply_to_status": 683364887834324992, "in_reply_to_user": 3653648555, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3653648555, 213322411, 254134610, 63775360, 1111756147, 94666311 }}, "user": { "id": 486977483, "name": "Joseph Schillaci", "screen_name": "JosephSchillac1", "lang": "en", "location": "null", "create_at": date("2012-02-08"), "description": "null", "followers_count": 27, "friends_count": 63, "statues_count": 200 }, "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-03T10:00:09.000Z"), "id": 683709448775319552, "text": "@TJohnstone64 thank you Moose <3 ����", "in_reply_to_status": 683700110031597568, "in_reply_to_user": 778914277, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 778914277 }}, "user": { "id": 23525855, "name": "Natalie", "screen_name": "NatalieNClark", "lang": "en", "location": "Lake Oswego, Oregon", "create_at": date("2009-03-09"), "description": "Professional fangirl.", "followers_count": 128, "friends_count": 828, "statues_count": 8334 }, "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-03T10:00:09.000Z"), "id": 683709448855011329, "text": "@TheLegend_AZ whatever keeps them sucking I'm all for ;)", "in_reply_to_status": 683694083391401984, "in_reply_to_user": 551501911, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 551501911 }}, "user": { "id": 26447281, "name": "Dave Zorn", "screen_name": "davezorn72", "lang": "en", "location": "N 33°21' 0'' / W 111°52' 0''", "create_at": date("2009-03-24"), "description": "News Director for KAFF AM/FM, 93.9 The Mountain, Hits 106 FM in Flagstaff. Magic 99.1, Oldies 1450am Prescott. NAU Grad and avid sports fan.", "followers_count": 1693, "friends_count": 543, "statues_count": 142687 }, "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-03T10:00:09.000Z"), "id": 683709448951443457, "text": "@ElianaGueron Love you so much girl�� Thank you!!", "in_reply_to_status": 683707763776552960, "in_reply_to_user": 552328161, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 552328161 }}, "user": { "id": 2443783298, "name": "Leah O'Neil", "screen_name": "loneil354", "lang": "en", "location": "St Paul, MN", "create_at": date("2014-04-14"), "description": "null", "followers_count": 109, "friends_count": 114, "statues_count": 583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomington, MN", "id": "080b8d8543aab399", "name": "Bloomington", "place_type": "city", "bounding_box": rectangle("-93.399443,44.78542 -93.203245,44.863519") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2706616, "cityName": "Bloomington" } }
+{ "create_at": datetime("2016-01-03T10:00:10.000Z"), "id": 683709449077305344, "text": "Jets vs. Bills. Win and the Jets are in. Ryan tries to show the world his team is on the improve. Kick off! #Sportshandicapper #NFL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.3937332,34.4829622"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sportshandicapper", "NFL" }}, "user": { "id": 239675037, "name": "Da Hat", "screen_name": "EddieTheHatWins", "lang": "en", "location": "Southern California", "create_at": date("2011-01-17"), "description": "Professional sports/horse racing handicapper. Wagering at the race track or on a sporting event, you won't find better info anywhere!", "followers_count": 267, "friends_count": 115, "statues_count": 5936 }, "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-03T10:00:10.000Z"), "id": 683709449106755584, "text": "@brianbocek @AtlantaFalcons @M_Ryan02 336 for Matt Ryan, 127 for Julio.", "in_reply_to_status": 683683550374281219, "in_reply_to_user": 210886343, "favorite_count": 0, "retweet_count": 0, "lang": "no", "is_retweet": false, "user_mentions": {{ 210886343, 16347506, 551405395 }}, "user": { "id": 1881490754, "name": "Dalton Wieland", "screen_name": "wieland15", "lang": "en", "location": "Freeburg, Illinois", "create_at": date("2013-09-18"), "description": "FCHS Junior", "followers_count": 194, "friends_count": 452, "statues_count": 685 }, "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-03T10:00:10.000Z"), "id": 683709449115021312, "text": "Wind 2.3 mph NNE. Barometer 30.389 in, Falling slowly. Temperature 56.3 °F. Rain today 0.09in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.11555556,29.6725"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25210383, "name": "New Braunfels WX", "screen_name": "jeffreycentexwx", "lang": "en", "location": "New Braunfels, TX", "create_at": date("2009-03-18"), "description": "Weather Station located on County Line Road in New Braunfels, TX.\r\n\r\nRealtime Weather @ http://www.nbtx-wx.com\r\nFollow me @ http://twitter.com/jeffreycentex", "followers_count": 379, "friends_count": 24, "statues_count": 159903 }, "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-01-03T10:00:10.000Z"), "id": 683709449240973312, "text": "Follow back coach‼️ @CoachBGasser", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 37493847 }}, "user": { "id": 2165509161, "name": "Ezra Naylor II", "screen_name": "305DONNY", "lang": "en", "location": "currently working hard⛏", "create_at": date("2013-11-02"), "description": "North Atlanta HS '16 FB&BB #10 | Glory From Above | 6'4 Wide Receiver| 4.5 Dash| 38 inch Vert| Scholar|1", "followers_count": 209, "friends_count": 209, "statues_count": 684 }, "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-03T10:00:10.000Z"), "id": 683709449303928832, "text": "@EmmaCaparula this", "in_reply_to_status": 683527066273316864, "in_reply_to_user": 592305362, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 592305362 }}, "user": { "id": 257206640, "name": "anne", "screen_name": "malinowskeet", "lang": "en", "location": "null", "create_at": date("2011-02-24"), "description": "I used to be good at uneven bars. Augustana '16.", "followers_count": 280, "friends_count": 198, "statues_count": 8058 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moline, IL", "id": "b8e37f613fedf510", "name": "Moline", "place_type": "city", "bounding_box": rectangle("-90.539878,41.455977 -90.431572,41.518476") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17161, "countyName": "Rock Island", "cityID": 1749867, "cityName": "Moline" } }
+{ "create_at": datetime("2016-01-03T10:00:10.000Z"), "id": 683709449479966720, "text": "@Koooori_ happy birthday ��", "in_reply_to_status": 680506320718401536, "in_reply_to_user": 227855417, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 227855417 }}, "user": { "id": 633381389, "name": "DANGELO", "screen_name": "dangelofaulk", "lang": "en", "location": "null", "create_at": date("2012-07-11"), "description": "You could be a 4 in the face, but your hair can fly you to a 10! ~ D'Angelo Faulk", "followers_count": 170, "friends_count": 206, "statues_count": 1177 }, "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-03T10:00:10.000Z"), "id": 683709449614143488, "text": "Yeah! https://t.co/THmybPJEgn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2386012310, "name": "Meredyth Lynne", "screen_name": "preCUMDOMcupied", "lang": "en", "location": "Los Angeles, CA ", "create_at": date("2014-03-12"), "description": "Actress/Model/Writer\n\nRevenge is sweet, but karma is a bitch! -- by Me!!!", "followers_count": 487, "friends_count": 713, "statues_count": 5908 }, "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-03T10:00:10.000Z"), "id": 683709449777709056, "text": "Can you recommend anyone for this #Sales #job? https://t.co/gSTHpO2tna #Chicago, IL #Hiring #CareerArc", "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", "Hiring", "CareerArc" }}, "user": { "id": 20831231, "name": "TMJ - CHI Sales Jobs", "screen_name": "tmj_chi_sales", "lang": "en", "location": "Chicago, IL", "create_at": date("2009-02-13"), "description": "Follow this account for geo-targeted Sales job tweets in Chicago, IL. Need help? Tweet us at @CareerArc!", "followers_count": 577, "friends_count": 316, "statues_count": 720 }, "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-03T10:00:10.000Z"), "id": 683709449790341121, "text": "Sooooo when I'm at work my siblings be up Af ... & the day I'm off & want to chill with�� they ALL sleep ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 758235979, "name": "L E X X", "screen_name": "GirlThats_Lexx", "lang": "en", "location": "null", "create_at": date("2012-08-14"), "description": "ᴹᴬᴷᴱ ˢᵁᴿᴱ ᵁᴿ ᴾᴱᴿᶠᴱᶜᵀ ᴮᴱᶠᴼᴿᴱ ᵁ ᴶᵁᴰᴳᴱ ᴹᴱ", "followers_count": 5542, "friends_count": 4000, "statues_count": 31336 }, "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-01-03T10:00:10.000Z"), "id": 683709449869983749, "text": "Me all Christmas break https://t.co/pZAosbNrWQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34408469, "name": "Courtney Knowles", "screen_name": "courtneynowles", "lang": "en", "location": "null", "create_at": date("2009-04-22"), "description": "delta phi epsilon • go camels", "followers_count": 1153, "friends_count": 527, "statues_count": 31672 }, "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-03T10:00:10.000Z"), "id": 683709449911963648, "text": "Sunny this afternoon, high 57 (14 C). Low 30 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712452, "name": "FortWorthWX", "screen_name": "FortWorthWX", "lang": "en", "location": "Fort Worth, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Fort Worth, Texas", "followers_count": 218, "friends_count": 93, "statues_count": 8177 }, "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-03T10:00:10.000Z"), "id": 683709449991671809, "text": "I need to 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": 230669938, "name": "#RestEasyRa'Shya ❤️", "screen_name": "__heaaven", "lang": "en", "location": "null", "create_at": date("2010-12-26"), "description": "All I Wanna Do Is Make My Lost Loved Ones Proud ❤️ | #BossmanMovement ✊ #LongLiveCapy ! Everyday Is A Happy Capy Day. ☺️ | RIP Ra'Shya ❤️", "followers_count": 2368, "friends_count": 1938, "statues_count": 23513 }, "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-03T10:00:10.000Z"), "id": 683709450180554752, "text": "@ArrozConDee dm me ;)", "in_reply_to_status": 683709383029690368, "in_reply_to_user": 3245406135, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3245406135 }}, "user": { "id": 3245406135, "name": "Dee R.", "screen_name": "ArrozConDee", "lang": "en", "location": "2nd Circle of Hell, New York", "create_at": date("2015-05-10"), "description": "It's 2016, no one cares.", "followers_count": 251, "friends_count": 284, "statues_count": 17771 }, "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-03T10:00:10.000Z"), "id": 683709450243469312, "text": "Gimme that fucking work https://t.co/QHkyryrWes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 596478770, "name": "JP", "screen_name": "JoshPetrovich", "lang": "en", "location": "null", "create_at": date("2012-06-01"), "description": "♏️", "followers_count": 558, "friends_count": 376, "statues_count": 3001 }, "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-01-03T10:00:10.000Z"), "id": 683709450256003073, "text": "I'm at The Tremont Tap House in Cleveland, OH https://t.co/bXKHSKAc4f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.695208,41.47715452"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 289454623, "name": "Steve Lawn", "screen_name": "slawn34", "lang": "en", "location": "Cleveland, OH", "create_at": date("2011-04-28"), "description": "Craft beer, good food & basketball\r\nUntappd & Instagram: slawn34", "followers_count": 370, "friends_count": 664, "statues_count": 11148 }, "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-03T10:00:10.000Z"), "id": 683709450436251648, "text": "soooooo i'm meeting my soulmate this year? ���� https://t.co/FbBltblaEn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234398012, "name": "Alexz walker", "screen_name": "xo_alexz", "lang": "en", "location": "San Antonio, TX", "create_at": date("2015-06-02"), "description": "whs/ dance (:", "followers_count": 149, "friends_count": 230, "statues_count": 3337 }, "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-03T10:00:10.000Z"), "id": 683709450872565762, "text": "@LuminessAir my compressor took a crap on me :( any specials? I got it for Christmas last year-only used four times. Didn't buy warranty :(", "in_reply_to_status": -1, "in_reply_to_user": 90926904, "favorite_count": 0, "coordinate": point("-80.19477287,40.2620121"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 90926904 }}, "user": { "id": 283213144, "name": "Chrystal Gmiter", "screen_name": "10xtheglam", "lang": "en", "location": "Pittsburgh, Pa", "create_at": date("2011-04-16"), "description": "Merchandiser with Chloe and Isabel. Paul Mitchell Graduate working at Regis Corporation Salon. Platform Artist/Artistic Team (soon to be) w Influance Hair Care", "followers_count": 299, "friends_count": 885, "statues_count": 1369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Canonsburg, PA", "id": "d64d438446aaf66a", "name": "Canonsburg", "place_type": "city", "bounding_box": rectangle("-80.206972,40.242046 -80.158673,40.29827") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42125, "countyName": "Washington", "cityID": 4211152, "cityName": "Canonsburg" } }
+{ "create_at": datetime("2016-01-03T10:00:10.000Z"), "id": 683709451271073793, "text": "I'll take an iced latte delivered to my house so I don't have to put pants on", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30397894, "name": "Bobbie Schumacher", "screen_name": "boobieanneee", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2009-04-10"), "description": "null", "followers_count": 614, "friends_count": 542, "statues_count": 30778 }, "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-03T10:00:10.000Z"), "id": 683709451392561152, "text": "purple leaking out my gut", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2187275647, "name": "C.", "screen_name": "xchelsooo_", "lang": "en", "location": "west", "create_at": date("2013-11-10"), "description": "hard ona ho", "followers_count": 571, "friends_count": 398, "statues_count": 11043 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-03T10:00:10.000Z"), "id": 683709451585601536, "text": "@CassieJoann_ \"sry i was pooping\"", "in_reply_to_status": 683707822102691840, "in_reply_to_user": 535562352, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 535562352 }}, "user": { "id": 503960633, "name": "jessica", "screen_name": "jessicvbrooks", "lang": "en", "location": "Battle Creek, MI", "create_at": date("2012-02-25"), "description": "just another paper cut survivor", "followers_count": 373, "friends_count": 123, "statues_count": 9553 }, "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-03T10:00:10.000Z"), "id": 683709451753275392, "text": "@flargh I think I am too.", "in_reply_to_status": 683709392299110402, "in_reply_to_user": 2960721, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2960721 }}, "user": { "id": 16315757, "name": "Ally Kazmucha", "screen_name": "iMuggle", "lang": "en", "location": "Chicago, IL", "create_at": date("2008-09-16"), "description": "Editor-in-Chief of @app_factor, world traveler, tea connoisseur, coffee drinker, and shibe wrangler. Email: ally@theappfactor.com", "followers_count": 4940, "friends_count": 968, "statues_count": 45147 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Michigan City, IN", "id": "4d24e111b5b23d78", "name": "Michigan City", "place_type": "city", "bounding_box": rectangle("-86.932736,41.665781 -86.816028,41.76024") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18091, "countyName": "LaPorte", "cityID": 1848798, "cityName": "Michigan City" } }
+{ "create_at": datetime("2016-01-03T10:00:10.000Z"), "id": 683709451757596672, "text": "Ayer abrí este tw y estoy apunto de borrarlo !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 4697405858, "name": "Giselle✨", "screen_name": "GiigiiValle", "lang": "en", "location": "Wildwood, NJ", "create_at": date("2016-01-02"), "description": "20•", "followers_count": 20, "friends_count": 32, "statues_count": 57 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rio Grande, NJ", "id": "a57f371a2dcdd622", "name": "Rio Grande", "place_type": "city", "bounding_box": rectangle("-74.899245,38.998451 -74.851833,39.031694") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34009, "countyName": "Cape May", "cityID": 3463180, "cityName": "Rio Grande" } }
+{ "create_at": datetime("2016-01-03T10:00:10.000Z"), "id": 683709451946229760, "text": "#Nursing #Job in #Lubbock, TX: Registered Nurse (RN)-CVICU at UMC https://t.co/psqTodhVra #cardiaccare #nurse #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-101.8919288,33.5881366"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "hashtags": {{ "Nursing", "Job", "Lubbock", "cardiaccare", "nurse", "Jobs", "Hiring" }}, "user": { "id": 2940007148, "name": "UMC Nursing Jobs", "screen_name": "UMCNursingJobs", "lang": "en", "location": "null", "create_at": date("2014-12-22"), "description": "UMC Health System is the leader in comprehensive healthcare delivery in West Texas and Eastern New Mexico. Follow us to learn about our Nursing Jobs!", "followers_count": 91, "friends_count": 278, "statues_count": 133 }, "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-03T10:00:10.000Z"), "id": 683709452118196224, "text": "@HeWay2Playa check out my beats for sale https://t.co/RTUMY9USfc", "in_reply_to_status": -1, "in_reply_to_user": 190529767, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 190529767 }}, "user": { "id": 372882394, "name": "ANGELO", "screen_name": "Drummaboyn23z", "lang": "en", "location": "in your trunk, Richmond, CA", "create_at": date("2011-09-13"), "description": "Upcoming producer, writer. \niMAke Beats, FL Studio (Demo for now)\n*Hu$tle Up ENt. \nRICHMOND, BAY AREA CA\n. LISTEN", "followers_count": 360, "friends_count": 832, "statues_count": 1121 }, "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-03T10:00:10.000Z"), "id": 683709452122472448, "text": "@Cianaf Not with the hands of a 60 year old arthritic he isn't.", "in_reply_to_status": 683709062979137536, "in_reply_to_user": 64076490, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 64076490 }}, "user": { "id": 367551096, "name": "Nightowls, Inc", "screen_name": "nightowlsinc", "lang": "en", "location": "Cleveland-ish", "create_at": date("2011-09-03"), "description": "Lover of football at all levels, cricket, humor, cold weather and B2B. Prone to sarcasm. Bald. Part of an ongoing medical experiment.", "followers_count": 209, "friends_count": 927, "statues_count": 23260 }, "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-01-03T10:00:10.000Z"), "id": 683709452193771521, "text": "������ https://t.co/wNAQ1ZlGUb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 372021999, "name": "Woods", "screen_name": "__JayKeith", "lang": "en", "location": "null", "create_at": date("2011-09-11"), "description": "snapchat:spaceeghost ONLY AS STRONG AS YO WEAKEST LINK. #finessegang #FOE R.I.H Darnell 2016 are year ✊", "followers_count": 687, "friends_count": 1333, "statues_count": 13210 }, "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-01-03T10:00:10.000Z"), "id": 683709452239941633, "text": "I get chills everytime the national anthem plays", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2859059202, "name": "Christan Grey", "screen_name": "casey_erdman24", "lang": "en", "location": "Northern Kentucky", "create_at": date("2014-10-16"), "description": "#TarHeelNation | #WhoDeyNation | Beechwood Graduate | US Air Force Firefighter | Niykee Heaton has retweeted me & liked my insta pix before | @niykeeheaton", "followers_count": 894, "friends_count": 616, "statues_count": 13512 }, "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-03T10:00:10.000Z"), "id": 683709452432769024, "text": "Sunny this afternoon, high 57 (14 C). Low 31 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712424, "name": "DallasWX", "screen_name": "DallasWX", "lang": "en", "location": "Dallas, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Dallas, Texas", "followers_count": 194, "friends_count": 93, "statues_count": 8154 }, "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-03T10:00:10.000Z"), "id": 683709452537708548, "text": "When you realize tomorrow is Monday and you can no longer sleep for 12+ hours https://t.co/tKgznc9SbX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2453838794, "name": "banana", "screen_name": "MarianiHannah", "lang": "en", "location": "the 330", "create_at": date("2014-04-19"), "description": "∆ loving food, hating life ∆", "followers_count": 187, "friends_count": 129, "statues_count": 13547 }, "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-01-03T10:00:10.000Z"), "id": 683709452684378112, "text": "@kee_big *it", "in_reply_to_status": 683709205132410880, "in_reply_to_user": 539550705, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 539550705 }}, "user": { "id": 2736301010, "name": "Drew", "screen_name": "MosCurved", "lang": "en", "location": "Austin, USA", "create_at": date("2014-08-15"), "description": "null", "followers_count": 747, "friends_count": 542, "statues_count": 35049 }, "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-03T10:00:10.000Z"), "id": 683709452722126848, "text": "I'm about to turn 18 but I still look 10", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1699721300, "name": "AL", "screen_name": "allisonhurd_17", "lang": "en", "location": "Tulsa, OK", "create_at": date("2013-08-25"), "description": "ORU soccer '20", "followers_count": 618, "friends_count": 682, "statues_count": 4644 }, "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-03T10:00:10.000Z"), "id": 683709453137399808, "text": "#OregonUnderAttack What would happen if these same men were Muslim or immigrants? Hypocrisy at its finest��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "OregonUnderAttack" }}, "user": { "id": 2402496992, "name": "Brenden Gilliss", "screen_name": "DaftRaven14", "lang": "en", "location": "Colorado", "create_at": date("2014-03-21"), "description": "Obscure music/bands connoisseur", "followers_count": 223, "friends_count": 295, "statues_count": 838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stonegate, CO", "id": "acaa7e6723823030", "name": "Stonegate", "place_type": "city", "bounding_box": rectangle("-104.830074,39.514914 -104.793403,39.554986") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8035, "countyName": "Douglas", "cityID": 874080, "cityName": "Stonegate" } }
+{ "create_at": datetime("2016-01-03T10:00:10.000Z"), "id": 683709453141594112, "text": "I am leaving the country in less than 3 days and haven't packed a thing. Except for my �� just jumped into my suitcase so ig I'm set", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 110260287, "name": "Rosie Uhen", "screen_name": "Rosiepoo_Unowho", "lang": "en", "location": "null", "create_at": date("2010-01-31"), "description": "Well aren't you looking jazzy", "followers_count": 658, "friends_count": 197, "statues_count": 5800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlington, WI", "id": "a6ad1c54aee52c70", "name": "Burlington", "place_type": "city", "bounding_box": rectangle("-88.306198,42.647177 -88.226143,42.712427") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55101, "countyName": "Racine", "cityID": 5511200, "cityName": "Burlington" } }
+{ "create_at": datetime("2016-01-03T10:00:11.000Z"), "id": 683709453242335233, "text": "don't test me.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2989278494, "name": "jan.16 ✨", "screen_name": "yungbarbiex0", "lang": "en", "location": "Queens, NY", "create_at": date("2015-01-18"), "description": "ι ĸιll вιтcнeѕ $$ | ♑️ | panaмanιan", "followers_count": 1181, "friends_count": 511, "statues_count": 30613 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Meadow Woods, FL", "id": "545d3cbd259866ef", "name": "Meadow Woods", "place_type": "city", "bounding_box": rectangle("-81.388834,28.347874 -81.310646,28.40079") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1243800, "cityName": "Meadow Woods" } }
+{ "create_at": datetime("2016-01-03T10:00:11.000Z"), "id": 683709453405822976, "text": "Can you recommend anyone for this #job? CVOR Nurse (RN) needed for Per Diem and Local Contracts in Orlando, FL - https://t.co/PZuOERuNs0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3792365,28.5383355"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job" }}, "user": { "id": 22293089, "name": "TMJ-ORL Nursing Jobs", "screen_name": "tmj_orl_nursing", "lang": "en", "location": "Orlando, FL", "create_at": date("2009-02-28"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Orlando, FL. Need help? Tweet us at @CareerArc!", "followers_count": 475, "friends_count": 302, "statues_count": 178 }, "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-03T10:00:11.000Z"), "id": 683709453577883648, "text": "I'm @ my Family Grocery Store decorticating our Christmas Stuff. @ Wohlner's Grocery and Deli https://t.co/Yy7XZ4UVNA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.9610748,41.2596245"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3098453628, "name": "Michaela Schwartz", "screen_name": "cpcheerdance1", "lang": "en", "location": "null", "create_at": date("2015-03-19"), "description": "null", "followers_count": 11, "friends_count": 87, "statues_count": 1096 }, "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-03T10:00:11.000Z"), "id": 683709453716303872, "text": "Coming soon, I promise! @_QueeenDee https://t.co/urGWRSAmoa", "in_reply_to_status": 683501515277991936, "in_reply_to_user": 625790394, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 625790394 }}, "user": { "id": 22698771, "name": "Morgan Marin", "screen_name": "Morgan_Marin", "lang": "en", "location": "Atlanta, Ga.", "create_at": date("2009-03-03"), "description": "Your Favorite Rapper's Makeup Artist | Professional Gypsy | Curator Of A Dope Life | Aspirational Twerk Goddess", "followers_count": 622, "friends_count": 127, "statues_count": 7965 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buckhead Forest, Atlanta", "id": "50ccc41c8de5437d", "name": "Buckhead Forest", "place_type": "neighborhood", "bounding_box": rectangle("-84.383051,33.84021 -84.371393,33.854335") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-01-03T10:00:11.000Z"), "id": 683709453741363200, "text": "Great seats for the game with @_seeean, _itstristan_ and Nick… https://t.co/3nVxgklCin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.252243,38.9541817"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 401614450 }}, "user": { "id": 377160049, "name": "Travis Green", "screen_name": "Greenboys17", "lang": "en", "location": "Roanoke, TX", "create_at": date("2011-09-20"), "description": "Leave no regrets or doubts in life", "followers_count": 69, "friends_count": 150, "statues_count": 320 }, "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-01-03T10:00:11.000Z"), "id": 683709453951176704, "text": "Bruh ������ https://t.co/BQjh6EpJ9E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3313767258, "name": "shike", "screen_name": "shaqtonio", "lang": "en", "location": "Tampa, FL", "create_at": date("2015-08-12"), "description": "If you following me, preciate it. Eagles, Noles, Lakers and that's it", "followers_count": 366, "friends_count": 333, "statues_count": 32721 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Egypt Lake-Leto, FL", "id": "6d4de57fa00a913f", "name": "Egypt Lake-Leto", "place_type": "city", "bounding_box": rectangle("-82.527472,27.996725 -82.485096,28.037298") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1220108, "cityName": "Egypt Lake-Leto" } }
+{ "create_at": datetime("2016-01-03T10:00:11.000Z"), "id": 683709453984755712, "text": "@nick_of_billman these are the worst pictures but ily ��", "in_reply_to_status": 683709237768359941, "in_reply_to_user": 2356740142, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2356740142 }}, "user": { "id": 2262825490, "name": "sarah", "screen_name": "sarah_bailey_", "lang": "en", "location": "Raleigh, NC", "create_at": date("2014-01-04"), "description": "millbrook", "followers_count": 172, "friends_count": 162, "statues_count": 761 }, "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-03T10:00:11.000Z"), "id": 683709454034931713, "text": "https://t.co/PxaY9W3BJt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 281822817, "name": "That Valley Life", "screen_name": "Showroom_77", "lang": "en", "location": "West Hills, CA", "create_at": date("2011-04-13"), "description": "strangers til we meet...", "followers_count": 14738, "friends_count": 4484, "statues_count": 35416 }, "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-03T10:00:11.000Z"), "id": 683709454060273665, "text": "@MizzMileHigh @bghostboo lmfaoooooo", "in_reply_to_status": 683709343695372288, "in_reply_to_user": 3030079525, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3030079525, 3242284513 }}, "user": { "id": 603332749, "name": "Mushin", "screen_name": "MetalMushin", "lang": "en", "location": "Orange, CT", "create_at": date("2012-06-08"), "description": "#Broncos, #Yankees, #Metalhead, #Guitarist, #Rocker, #Jedi, Professional Wise-Ass. May the Force be with you. You can call me Mushin. The .Gif King.", "followers_count": 2111, "friends_count": 2062, "statues_count": 152348 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CT", "id": "7b62343e287fb7c0", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-73.085611,41.245045 -72.979996,41.31407") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 957670, "cityName": "Orange" } }
+{ "create_at": datetime("2016-01-03T10:00:11.000Z"), "id": 683709454336921600, "text": "Can you recommend anyone for this #job? https://t.co/jzVsFsTUWF #medical #Quincy, IL #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-91.4098726,39.9356016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "medical", "Quincy", "Hiring" }}, "user": { "id": 2166774096, "name": "Quincy MedGroup", "screen_name": "QuincyMedGroup", "lang": "en", "location": "Quincy, IL", "create_at": date("2013-10-31"), "description": "null", "followers_count": 108, "friends_count": 62, "statues_count": 46 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Quincy, IL", "id": "833d51949e9c0bfb", "name": "Quincy", "place_type": "city", "bounding_box": rectangle("-91.42088,39.879982 -91.315192,39.972396") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17001, "countyName": "Adams", "cityID": 1762367, "cityName": "Quincy" } }
+{ "create_at": datetime("2016-01-03T10:00:11.000Z"), "id": 683709454341304320, "text": "@Game_216 that's one way to look at it. But by having courage to speak up, he may have stopped. So in w small way, they are at fault", "in_reply_to_status": 683709189068337153, "in_reply_to_user": 267164954, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 267164954 }}, "user": { "id": 176222758, "name": "Von Redcorn", "screen_name": "JovTheGreat", "lang": "en", "location": "null", "create_at": date("2010-08-08"), "description": "Im just a simple man that is trying to find new and innovative ways to be a setty!", "followers_count": 671, "friends_count": 623, "statues_count": 61189 }, "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-01-03T10:00:11.000Z"), "id": 683709454362226688, "text": "I think ill jus buy sweats and hoodies this winter , im just tryna be warm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 634283622, "name": "J❣", "screen_name": "jaylyn_paige", "lang": "en", "location": "null", "create_at": date("2012-07-12"), "description": "|401⚓️| sc: jaylyn_paigee|⚽️|", "followers_count": 1853, "friends_count": 2002, "statues_count": 28728 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pawtucket, RI", "id": "47623c4cf6ca7957", "name": "Pawtucket", "place_type": "city", "bounding_box": rectangle("-71.422117,41.856092 -71.333899,41.8984") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4454640, "cityName": "Pawtucket" } }
+{ "create_at": datetime("2016-01-03T10:00:11.000Z"), "id": 683709454458712064, "text": "@TurntAlien @CHenderson_20 but u dont turn up????", "in_reply_to_status": 676962773792870400, "in_reply_to_user": 3399224657, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3399224657, 2973955066 }}, "user": { "id": 2536217307, "name": "katherine downs", "screen_name": "downs_katherine", "lang": "en", "location": "madison al", "create_at": date("2014-05-08"), "description": "null", "followers_count": 349, "friends_count": 314, "statues_count": 1196 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Frontenac, MO", "id": "29a119f18820c3ad", "name": "Frontenac", "place_type": "city", "bounding_box": rectangle("-90.433909,38.606163 -90.405462,38.647891") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2926110, "cityName": "Frontenac" } }
+{ "create_at": datetime("2016-01-03T10:00:11.000Z"), "id": 683709454504693760, "text": "Clinical Nurse Neuro Stroke, Rlf B, .4 nights (32003) - Stanford Health Care: (#PaloAlto, CA) https://t.co/4EKjxvr6rV #Nursing #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.1430195,37.4418834"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PaloAlto", "Nursing", "Job", "Jobs" }}, "user": { "id": 3405932663, "name": "SHC Jobs", "screen_name": "SHC_Jobs", "lang": "en", "location": "null", "create_at": date("2015-08-06"), "description": "We’re looking for people like you to help us change the world. Fearless leaders, unstoppable patient advocates, and healing partners. Find your opportunity.", "followers_count": 35, "friends_count": 7, "statues_count": 641 }, "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-01-03T10:00:11.000Z"), "id": 683709454643146752, "text": "@ddlovato Just wanted 2 let u know that my 10 yo daughter modified 'confident.' In her ver, the last chorus is 'NOTHING wrong with bein...!'", "in_reply_to_status": -1, "in_reply_to_user": 21111883, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21111883 }}, "user": { "id": 167150267, "name": "bonnie duritsky", "screen_name": "bduritsky", "lang": "en", "location": "toledo, ohio", "create_at": date("2010-07-15"), "description": "mom, wife, pagan, poet, dyke, artist, warrior, dragon breeder, and pokémon master", "followers_count": 366, "friends_count": 971, "statues_count": 17880 }, "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-03T10:00:11.000Z"), "id": 683709454869639168, "text": "@MrGold1308 Praying for you. Wish I could have met him; I know he was an incredible man.", "in_reply_to_status": 683660832996257793, "in_reply_to_user": 36510572, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 36510572 }}, "user": { "id": 273213104, "name": "ag ❦", "screen_name": "shakenexistence", "lang": "en", "location": "Germany ✈️ Argentina ✈️ Chicago ", "create_at": date("2011-03-27"), "description": "20. Singer/Songwriter.\nDaughter. Girlfriend. Lover. \nLess Human. More Being.", "followers_count": 12568, "friends_count": 4807, "statues_count": 4927 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palatine, IL", "id": "8cc8ba275709bbf3", "name": "Palatine", "place_type": "city", "bounding_box": rectangle("-88.095381,42.066918 -88.003112,42.163339") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1757225, "cityName": "Palatine" } }
+{ "create_at": datetime("2016-01-03T10:00:11.000Z"), "id": 683709455243042816, "text": "Let's go skins! #HTTR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HTTR" }}, "user": { "id": 452554890, "name": "Kim Owens", "screen_name": "KimmeeSue6", "lang": "en", "location": "Centreville, VA", "create_at": date("2012-01-01"), "description": "Arguably the #1 fan of #Dateline & #Mank ! I ♥️sugar, gobstoppers, coca-cola, red dye, preservatives and additives ;) :P . #NATITUDE #HAILTOTHEREDSKINS", "followers_count": 202, "friends_count": 402, "statues_count": 11875 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Centreville, VA", "id": "ffcc53c4a4e7a620", "name": "Centreville", "place_type": "city", "bounding_box": rectangle("-77.479597,38.802143 -77.397429,38.880183") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5114440, "cityName": "Centreville" } }
+{ "create_at": datetime("2016-01-03T10:00:11.000Z"), "id": 683709455570055169, "text": "Aureus Medical Group: RN / REGISTERED NURSE / MED SURG RN / MEDICAL SURGICAL NURSE (#Albany, NY) https://t.co/0lCzVug90v #Nursing #Job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9712488,42.6284995"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Albany", "Nursing", "Job" }}, "user": { "id": 22156004, "name": "Albany Nursing Jobs", "screen_name": "tmj_nya_nursing", "lang": "en", "location": "Albany, NY", "create_at": date("2009-02-27"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Albany, NY. Need help? Tweet us at @CareerArc!", "followers_count": 487, "friends_count": 333, "statues_count": 87 }, "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": 36001, "countyName": "Albany" } }
+{ "create_at": datetime("2016-01-03T10:00:11.000Z"), "id": 683709455742058498, "text": "We're #hiring! Read about our latest #job opening here: Quality Assurance Tester - https://t.co/LhzmDdzFja #QA #Agile #Charlotte, NC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.8431267,35.2270869"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "QA", "Agile", "Charlotte" }}, "user": { "id": 2935068602, "name": "NASCAR Jobs", "screen_name": "NASCARJobs", "lang": "en", "location": "null", "create_at": date("2014-12-18"), "description": "The National Association for Stock Car Auto Racing, Inc. (NASCAR) is the sanctioning body for one of North America's premier sports.", "followers_count": 1088, "friends_count": 14, "statues_count": 160 }, "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-03T10:00:11.000Z"), "id": 683709455834476544, "text": "@bunkkris dollar bills > buffalo bills", "in_reply_to_status": 683703847274082305, "in_reply_to_user": 28886823, "favorite_count": 0, "retweet_count": 0, "lang": "sv", "is_retweet": false, "user_mentions": {{ 28886823 }}, "user": { "id": 343844866, "name": "Mr. Feliciano", "screen_name": "jFel_22", "lang": "en", "location": "null", "create_at": date("2011-07-27"), "description": "Taken By @Short_Cakes_23 | Univ. of South Florida Grad | Sneakerhead | PPPD", "followers_count": 358, "friends_count": 310, "statues_count": 16215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pinellas Park, FL", "id": "d598cb5ad6f7a09f", "name": "Pinellas Park", "place_type": "city", "bounding_box": rectangle("-82.752428,27.824633 -82.663128,27.897396") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1256975, "cityName": "Pinellas Park" } }
+{ "create_at": datetime("2016-01-03T10:00:11.000Z"), "id": 683709455955955713, "text": "@heygregr you got engaged?? Congrats, bro! I must’ve missed that..", "in_reply_to_status": 683477980438331392, "in_reply_to_user": 37971491, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 37971491 }}, "user": { "id": 55590555, "name": "Kris Venden", "screen_name": "mpt45", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-07-10"), "description": "I am a Senior Computer Specialist at University of Washington and enjoy basketball, blu-ray movies, camping, traveling, photography and mobile technology.", "followers_count": 227, "friends_count": 160, "statues_count": 6212 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edmonds, WA", "id": "07419c1d9571aa81", "name": "Edmonds", "place_type": "city", "bounding_box": rectangle("-122.395871,47.777628 -122.319674,47.859617") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5320750, "cityName": "Edmonds" } }
+{ "create_at": datetime("2016-01-03T10:00:11.000Z"), "id": 683709455968632832, "text": "I can't believe I just 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": 238659375, "name": "Ebssss", "screen_name": "ebonijoanna_", "lang": "en", "location": "null", "create_at": date("2011-01-15"), "description": "null", "followers_count": 571, "friends_count": 409, "statues_count": 19656 }, "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-01-03T10:00:11.000Z"), "id": 683709456002121728, "text": "March Madness in Dallas �� https://t.co/UYfPsjG0lU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 375025804, "name": "Papito Carlito", "screen_name": "CharlieScene__", "lang": "en", "location": "Flexas, USA", "create_at": date("2011-09-17"), "description": "5'9 but my ego 6'4.", "followers_count": 1884, "friends_count": 949, "statues_count": 103632 }, "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-03T10:00:11.000Z"), "id": 683709456031432704, "text": "@JoshMalina just let her know...", "in_reply_to_status": 683709205778313216, "in_reply_to_user": 24931027, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24931027 }}, "user": { "id": 39996485, "name": "Tim, a reader", "screen_name": "HeartyLaff", "lang": "en", "location": "Colorado by way of NM & MA", "create_at": date("2009-05-14"), "description": "Podcasts bring me joy.", "followers_count": 492, "friends_count": 1756, "statues_count": 28138 }, "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-03T10:00:11.000Z"), "id": 683709456111251457, "text": "@alliebloom @Johnson56Nic @natelawson01 no fucking way? Wow", "in_reply_to_status": 683706993299816448, "in_reply_to_user": 948238956, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 948238956, 1368354032, 596796037 }}, "user": { "id": 855315704, "name": "yung electrician", "screen_name": "Jallen_61", "lang": "en", "location": "Sunny Hill", "create_at": date("2012-09-30"), "description": "Kaymans twin brother. President of Sunny Hill Estates", "followers_count": 253, "friends_count": 213, "statues_count": 2706 }, "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-03T10:00:11.000Z"), "id": 683709456253906944, "text": "Lmaoo I'm really trying to stop laughing but I just can't imagine letting a female I don't like in 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": 2864985740, "name": "MBW", "screen_name": "RealBD_", "lang": "en", "location": "null", "create_at": date("2014-10-19"), "description": "you do what you want when you popping like me bish .. get there ! snapchat: simbabri ❤️", "followers_count": 446, "friends_count": 342, "statues_count": 6212 }, "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-03T10:00:11.000Z"), "id": 683709456278945792, "text": "How the fuck did I not know it was going to snow today...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 307168005, "name": "Marlene", "screen_name": "MarleneNegrete_", "lang": "en", "location": "Oregon", "create_at": date("2011-05-28"), "description": "null", "followers_count": 204, "friends_count": 62, "statues_count": 8335 }, "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-03T10:00:11.000Z"), "id": 683709456543121408, "text": "Not many people have a boss as great as mine, he calls and says that he's coming in to cover the rest of my shift so I-", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 207819875, "name": "Becky Hatfield", "screen_name": "star_fire_wolf", "lang": "en", "location": "Oregon", "create_at": date("2010-10-25"), "description": "Make you're life a statement of love and compassion, and where it isn't, that's where you're work lies.", "followers_count": 112, "friends_count": 84, "statues_count": 53309 }, "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-01-03T10:00:11.000Z"), "id": 683709456723611648, "text": "Cleanest picture of the week ������ @ Marty's Lounge https://t.co/201KbD2xff", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.9749705,41.76243516"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 44976288, "name": "Stephanie Lynn", "screen_name": "x0stephhh", "lang": "en", "location": "Long Island, New York", "create_at": date("2009-06-05"), "description": "21. ΔΦΕ", "followers_count": 480, "friends_count": 490, "statues_count": 13167 }, "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": 36105, "countyName": "Sullivan" } }
+{ "create_at": datetime("2016-01-03T10:00:11.000Z"), "id": 683709457277272064, "text": "Break time!! (@ Hill Country Barbecue Market - @hillcountryny in New York, NY) https://t.co/VxasxXcDnJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.99042638,40.74434085"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20930651 }}, "user": { "id": 24411441, "name": "K. Thacker", "screen_name": "KeniThacker", "lang": "en", "location": "ÜT: 40.788927,-73.952626", "create_at": date("2009-03-14"), "description": "Writer, Producer, Director, Event Technology Guru, Influencer, Urban Sociallite, Diversity Champion, KREATOR and SoulStoryteller. LIVE. LOVE. CREATE. CHANGE.", "followers_count": 702, "friends_count": 424, "statues_count": 19205 }, "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-03T10:00:12.000Z"), "id": 683709457407315968, "text": "Get Up ! Boy You Alright ? ������ https://t.co/71mQdnqvy2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2719414133, "name": "BIG HMIE PEACH", "screen_name": "PeachesXVII", "lang": "en", "location": "Belle Glade 56♤/Daytona Beach ", "create_at": date("2014-07-20"), "description": "#BCU19 | IG: Peachy.Asf | SC: PeachesXVII | Gators On Saturdays, Patriots On Sundays", "followers_count": 125, "friends_count": 118, "statues_count": 522 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Belle Glade, FL", "id": "bc583764882e890f", "name": "Belle Glade", "place_type": "city", "bounding_box": rectangle("-80.70014,26.649434 -80.624779,26.725792") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1205200, "cityName": "Belle Glade" } }
+{ "create_at": datetime("2016-01-03T10:00:12.000Z"), "id": 683709457587695616, "text": "Very cloudy and chilly 77 degrees today :(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54633009, "name": "Kay", "screen_name": "KayFusion", "lang": "en", "location": "In the City That Never Sleeps", "create_at": date("2009-07-07"), "description": "I color outside the lines. I simply don't believe in limitations!", "followers_count": 1175, "friends_count": 351, "statues_count": 105810 }, "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-03T10:00:12.000Z"), "id": 683709457591762944, "text": "Morning clouds/afternoon sun this afternoon, high 60 (16 C). Low 38 (3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712388, "name": "HoustonWX", "screen_name": "HoustonWX", "lang": "en", "location": "Houston, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Houston, Texas", "followers_count": 1817, "friends_count": 77, "statues_count": 8024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-03T10:00:12.000Z"), "id": 683709457700798464, "text": "Can you recommend anyone for this #job? Cashier - https://t.co/cxQ1fPkTvb #Braintree, MA #Hospitality #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.0235217,42.2189838"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Braintree", "Hospitality", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 22453979, "name": "TMJ-BOS HRTA Jobs", "screen_name": "tmj_bos_hrta", "lang": "en", "location": "Boston, MA", "create_at": date("2009-03-01"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Boston, MA. Need help? Tweet us at @CareerArc!", "followers_count": 426, "friends_count": 299, "statues_count": 562 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Braintree, MA", "id": "3846ebded61e1cc3", "name": "Braintree", "place_type": "city", "bounding_box": rectangle("-71.049556,42.168248 -70.961164,42.23848") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25021, "countyName": "Norfolk", "cityID": 2507740, "cityName": "Braintree Town" } }
+{ "create_at": datetime("2016-01-03T10:00:12.000Z"), "id": 683709457797222400, "text": "I want to buy a bunch of play-doh ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 450927962, "name": "Lo$o", "screen_name": "Infamous_OG", "lang": "en", "location": "Chula Vista, CA ☀️", "create_at": date("2011-12-30"), "description": "don't let emotion cloud your judgement.", "followers_count": 1043, "friends_count": 390, "statues_count": 45114 }, "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-03T10:00:12.000Z"), "id": 683709457981960194, "text": "vou p outlet hoje ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 2421238048, "name": "Emily Saldanha", "screen_name": "_emilysaldanha", "lang": "pt", "location": "São Paulo, Brasil", "create_at": date("2014-03-18"), "description": "✨", "followers_count": 164, "friends_count": 160, "statues_count": 1894 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walpole, MA", "id": "81732523636732fe", "name": "Walpole", "place_type": "city", "bounding_box": rectangle("-71.303783,42.088125 -71.194022,42.204216") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25021, "countyName": "Norfolk", "cityID": 2572460, "cityName": "Walpole" } }
+{ "create_at": datetime("2016-01-03T10:00:12.000Z"), "id": 683709458426499072, "text": "Today's purchase - veggie spiralizer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 98804845, "name": "Molly Stukey", "screen_name": "mollllypocket", "lang": "en", "location": "Flavortown, USA", "create_at": date("2009-12-22"), "description": "https://vine.co/v/eJlPUOH2Xqa", "followers_count": 424, "friends_count": 503, "statues_count": 26339 }, "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-03T10:00:12.000Z"), "id": 683709458627735552, "text": "Parts Delivery - O'Reilly Auto Parts: (#PLAINWELL, MI) https://t.co/P5td9jPHmE #Transportation #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.6708753,42.4495412"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PLAINWELL", "Transportation", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 149735286, "name": "TMJ-KAL Transport.", "screen_name": "tmj_KAL_transp", "lang": "en", "location": "Kalamazoo, MI", "create_at": date("2010-05-29"), "description": "Follow this account for geo-targeted Transportation job tweets in Kalamazoo, MI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 277, "friends_count": 274, "statues_count": 23 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Otsego, MI", "id": "00ff56c42cc1ec7e", "name": "Otsego", "place_type": "city", "bounding_box": rectangle("-85.733071,42.427957 -85.661211,42.468538") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26005, "countyName": "Allegan", "cityID": 2661620, "cityName": "Otsego" } }
+{ "create_at": datetime("2016-01-03T10:00:12.000Z"), "id": 683709458640289792, "text": "Boarding my second flight ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1095787429, "name": "AlmightyNich !", "screen_name": "TheRealNich_", "lang": "en", "location": "null", "create_at": date("2013-01-16"), "description": "• U.S. Navy ⚓️ • Fort Worth , Texas ✈️ Great Lakes , Illinois • SC : Nich1995 •", "followers_count": 1424, "friends_count": 1658, "statues_count": 41365 }, "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-01-03T10:00:12.000Z"), "id": 683709458673872897, "text": "254 days till slfl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2749930973, "name": "destiny", "screen_name": "9inmichael", "lang": "en", "location": "mgc/4", "create_at": date("2014-08-21"), "description": "what the fuck is a 5 sauce?", "followers_count": 2917, "friends_count": 200, "statues_count": 10021 }, "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-03T10:00:12.000Z"), "id": 683709458678087680, "text": "Thought I was playing ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 539373304, "name": "Ty-Flyy✨", "screen_name": "KingBreezy_bish", "lang": "en", "location": "NEW YORK", "create_at": date("2012-03-28"), "description": "✴Hip-hop Dancer\n✴Singer// Humble Yourself..let go of your pride✨ \nSC:Kingbreezy_tyee", "followers_count": 4305, "friends_count": 4186, "statues_count": 51534 }, "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-03T10:00:12.000Z"), "id": 683709458694852608, "text": "#Clerical alert: AD20 - Administrative Assistant II | Kelly Services | #Beaverton, OR https://t.co/qpxZdsg0G3 #KellyJobs #KellyServices", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.8029665,45.4923824"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Clerical", "Beaverton", "KellyJobs", "KellyServices" }}, "user": { "id": 23022397, "name": "TMJ-PDX Cleric. Jobs", "screen_name": "tmj_pdx_cler", "lang": "en", "location": "Portland, OR", "create_at": date("2009-03-05"), "description": "Follow this account for geo-targeted Office/Clerical job tweets in Portland, OR. Need help? Tweet us at @CareerArc!", "followers_count": 290, "friends_count": 201, "statues_count": 85 }, "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-03T10:00:12.000Z"), "id": 683709458715787264, "text": "Woke up late. Great", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2384136489, "name": "Kaelee. ✨", "screen_name": "kaeleeannn", "lang": "en", "location": "Oahu, HI. Poquoson, VA.", "create_at": date("2014-03-04"), "description": "heartless but hilarious (Ithink)", "followers_count": 1626, "friends_count": 875, "statues_count": 66108 }, "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-01-03T10:00:12.000Z"), "id": 683709458808201216, "text": "I'm at Merle Hay Neighborhood https://t.co/VHJougaHfO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.69796757,41.63005447"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18100387, "name": "RyanKolder", "screen_name": "RyanKolder", "lang": "en", "location": "Des Moines Iowa", "create_at": date("2008-12-13"), "description": "null", "followers_count": 564, "friends_count": 1314, "statues_count": 2230 }, "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-03T10:00:12.000Z"), "id": 683709459277852672, "text": "Can't wait for all the \"I haven't see you in a year\" jokes tomorrow", "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": "Childish Petes", "screen_name": "tjpeters27", "lang": "en", "location": "Probably at work", "create_at": date("2012-03-29"), "description": "It's only weird if you make it weird-Madison Chapla Rip AJ #1 #blessed yah know?", "followers_count": 644, "friends_count": 381, "statues_count": 8838 }, "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-01-03T10:00:12.000Z"), "id": 683709459294650369, "text": "Want to work in #Lexington, KY? View our latest opening: https://t.co/9GFNCiTlu6 #IT #Job #Jobs #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": {{ "Lexington", "IT", "Job", "Jobs", "Hiring" }}, "user": { "id": 99868456, "name": "TMJ-KYL IT Adm. Jobs", "screen_name": "tmj_KYL_adm", "lang": "en", "location": "Lexington, KY", "create_at": date("2009-12-27"), "description": "Follow this account for geo-targeted IT-Support/Administration job tweets in Lexington, KY from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 273, "friends_count": 265, "statues_count": 12 }, "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-03T10:00:12.000Z"), "id": 683709459831472128, "text": "Shasta todayyyy ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 609482734, "name": "Hayleybop", "screen_name": "hayley_renee23", "lang": "en", "location": "null", "create_at": date("2012-06-15"), "description": "Above all else, guard your heart, for everything you do flows from it, Proverbs 4:23", "followers_count": 326, "friends_count": 380, "statues_count": 3484 }, "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-03T10:00:12.000Z"), "id": 683709459995193344, "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": 489672889, "name": "cass (-:", "screen_name": "cass_anne_33", "lang": "en", "location": "cincinnati ", "create_at": date("2012-02-11"), "description": "ignorance is bliss | oak hills '18", "followers_count": 564, "friends_count": 533, "statues_count": 23707 }, "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-01-03T10:00:12.000Z"), "id": 683709460079050756, "text": "BABY FEVER https://t.co/OpeA4PFt2O", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2750156345, "name": "laylani✨", "screen_name": "_qveenlaaay_", "lang": "en", "location": "null", "create_at": date("2014-08-22"), "description": "plot twist: queen since '99", "followers_count": 621, "friends_count": 643, "statues_count": 10938 }, "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-03T10:00:12.000Z"), "id": 683709460087488512, "text": "this what i say when sum i don't like happens https://t.co/QEMTnUExej", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2930564329, "name": "ash", "screen_name": "ashleemrivera", "lang": "en", "location": "null", "create_at": date("2014-12-14"), "description": "it be like that sometimes", "followers_count": 862, "friends_count": 432, "statues_count": 16824 }, "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-01-03T10:00:12.000Z"), "id": 683709460179636224, "text": "Sunny this afternoon, high 56 (13 C). Low 26 (-3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575081, "name": "San Angelo Weather", "screen_name": "SanAngeloWX", "lang": "en", "location": "San Angelo, TX", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for San Angelo, TX", "followers_count": 1943, "friends_count": 92, "statues_count": 7914 }, "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-03T10:00:12.000Z"), "id": 683709460242653184, "text": "77k : 7", "in_reply_to_status": 683706745374568448, "in_reply_to_user": 2507576581, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2507576581, "name": "Ziyad Abdullah", "screen_name": "ziyadxd", "lang": "en", "location": "استغفرالله", "create_at": date("2014-05-19"), "description": "فَلا نَامَتْ أعْيُنُ الجُبَنَاء", "followers_count": 454, "friends_count": 178, "statues_count": 39133 }, "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-03T10:00:12.000Z"), "id": 683709460276195328, "text": "“Everybody is a genius. But if you judge a fish by its ability to climb a tree, it will live its whole life believing that it is stupid.”", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1352528348, "name": "MyFight com", "screen_name": "MyFightcom", "lang": "en", "location": "USA", "create_at": date("2013-04-14"), "description": "“When you want something, all the universe conspires in helping you to achieve it.”", "followers_count": 13446, "friends_count": 11711, "statues_count": 10774 }, "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-03T10:00:12.000Z"), "id": 683709460305592320, "text": "Uplink grind", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1375397180, "name": "Squishy LG", "screen_name": "LethalXPancake", "lang": "en", "location": "Baltimore, Maryland", "create_at": date("2013-04-23"), "description": "LT of @LethalGaming_ on xbox one. i only play pubs done with comp. 2014 clan war champions. Snipers=plague.Canucks, orioles and skins fan! Vancouver native", "followers_count": 3638, "friends_count": 1579, "statues_count": 4121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballenger Creek, MD", "id": "be0e62c690c5acbc", "name": "Ballenger Creek", "place_type": "city", "bounding_box": rectangle("-77.467661,39.341113 -77.388726,39.403823") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24021, "countyName": "Frederick", "cityID": 2403800, "cityName": "Ballenger Creek" } }
+{ "create_at": datetime("2016-01-03T10:00:12.000Z"), "id": 683709460548874240, "text": "happy birthday to my sister in law msjaynkins who i'm blessed to call a very dear friend! i… https://t.co/Pe4akw1D0S", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6844,41.8369"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1910027150, "name": "Chai & Pie Co.", "screen_name": "chaiandpieco", "lang": "en", "location": "Bay2Chi", "create_at": date("2013-09-26"), "description": "Chai and Pie. Food. Travel. Home. homestyle pies: order@orderchaiandpie.com #chaiandpie", "followers_count": 241, "friends_count": 499, "statues_count": 1610 }, "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-03T10:00:12.000Z"), "id": 683709460590641152, "text": "@m___beezy Funny! What a great post Madi! You always surprise me young lady. How is school going? Geeze you really are growing up fast.", "in_reply_to_status": 683709192826306561, "in_reply_to_user": 596242799, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 596242799 }}, "user": { "id": 2779205698, "name": "*chanting*", "screen_name": "bryanathompson1", "lang": "en", "location": "quinoa nation", "create_at": date("2014-09-21"), "description": "i love onions and goodwill. • https://twitter.com/m___beezy/status/628631368029851648", "followers_count": 183, "friends_count": 338, "statues_count": 629 }, "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-01-03T10:00:12.000Z"), "id": 683709460787793921, "text": "See our latest #Mclean, VA #job and click to apply: Sr Java Developer - Mclean - https://t.co/Q4rBghiKCN #IT #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.1803048,38.9362225"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Mclean", "job", "IT", "Hiring", "CareerArc" }}, "user": { "id": 113036236, "name": "TMJ-VAV IT Jobs", "screen_name": "tmj_VAV_it", "lang": "en", "location": "Tysons, VA", "create_at": date("2010-02-10"), "description": "Follow this account for geo-targeted Software Dev. - General/IT job tweets in Tysons, VA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 333, "friends_count": 308, "statues_count": 42 }, "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-03T10:00:12.000Z"), "id": 683709460959789056, "text": "This is hysterical", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2155131765, "name": "Amanda Tompkins", "screen_name": "mandatompkins", "lang": "en", "location": "NJ", "create_at": date("2013-10-26"), "description": "7/31/15 it was decent. -@michelledelm ❘ beaded velvet", "followers_count": 160, "friends_count": 145, "statues_count": 1345 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milltown, NJ", "id": "f44c9b25d22e1e74", "name": "Milltown", "place_type": "city", "bounding_box": rectangle("-74.449377,40.434397 -74.423103,40.463151") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3446620, "cityName": "Milltown" } }
+{ "create_at": datetime("2016-01-03T10:00:12.000Z"), "id": 683709461211537409, "text": "Barberton Oh Temp:31.2°F Wind:3 mph Dir:W Baro:Rising slowly Rain2day:0.00in Hum:78% UV:0.0 @ 13:00 01/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": 237, "friends_count": 228, "statues_count": 113645 }, "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-01-03T10:00:12.000Z"), "id": 683709461245001728, "text": "I'm cold ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1110199201, "name": "Catherine Loze' ⚓", "screen_name": "lozecat", "lang": "en", "location": "null", "create_at": date("2013-01-21"), "description": "T.J.K.", "followers_count": 395, "friends_count": 421, "statues_count": 8613 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arabi, LA", "id": "d36c3e15214e76af", "name": "Arabi", "place_type": "city", "bounding_box": rectangle("-90.010442,29.937441 -89.984262,29.974236") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22087, "countyName": "St. Bernard", "cityID": 2202550, "cityName": "Arabi" } }
+{ "create_at": datetime("2016-01-03T10:00:12.000Z"), "id": 683709461375127553, "text": "@JCGibby_81 ILL BE THERE BY LIKE 2:30", "in_reply_to_status": 683709382010494976, "in_reply_to_user": 371062004, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 371062004 }}, "user": { "id": 2293484317, "name": "Hayden Your Milkman™", "screen_name": "ItsYourMilkman", "lang": "en", "location": "I like to make money get,turnt", "create_at": date("2014-01-15"), "description": "- First Team* Wakin up on sunset blvd. Proud AMERICAN-Terrorist Fighter |FAITH| Army Of Christ $ Catching baseballs @ UNOH RV13 #RVFRATHOUSE #THEGOAT", "followers_count": 369, "friends_count": 422, "statues_count": 8178 }, "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-01-03T10:00:12.000Z"), "id": 683709461496610816, "text": "@basedxhokage why so many guys like milf pictures gdi ��", "in_reply_to_status": 683709153039106048, "in_reply_to_user": 266113224, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 266113224 }}, "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": 245, "friends_count": 143, "statues_count": 5525 }, "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-03T10:00:13.000Z"), "id": 683709461697925120, "text": "@KillaKassie thats real ✊��. But the ones that are know wussup��", "in_reply_to_status": 683708294540599297, "in_reply_to_user": 553079494, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 553079494 }}, "user": { "id": 285454575, "name": "B.Neal™", "screen_name": "_Imitated", "lang": "en", "location": "Fort Worth, TX / Goodwell, OK", "create_at": date("2011-04-20"), "description": "Billy Neal Jr. || 5'5 || OPSU Football #22 || Sports & Exercise Management major || July 4th, 1993 || Follow Me?", "followers_count": 1655, "friends_count": 1257, "statues_count": 56553 }, "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-03T10:00:13.000Z"), "id": 683709461974794240, "text": "Me: dad do you have super glue?\nDad: I have an engraver\n\n....thanks", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2533872853, "name": "Hailie Aloevera", "screen_name": "h4ilst0rm", "lang": "en", "location": "Florida State University", "create_at": date("2014-05-29"), "description": "ask me about my hemorrhoids", "followers_count": 115, "friends_count": 110, "statues_count": 2500 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cocoa West, FL", "id": "7083be61368e3417", "name": "Cocoa West", "place_type": "city", "bounding_box": rectangle("-80.794834,28.345469 -80.746665,28.377202") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12009, "countyName": "Brevard", "cityID": 1213225, "cityName": "Cocoa West" } }
+{ "create_at": datetime("2016-01-03T10:00:13.000Z"), "id": 683709462167859200, "text": "I Will Tell You The Real But Don't Ask Me About Your Chick Because I Owe U No Loyality.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 453636714, "name": "Rondo", "screen_name": "Jad_Young", "lang": "en", "location": "Hammond, LA", "create_at": date("2012-01-02"), "description": "23, SC: Jadguar", "followers_count": 940, "friends_count": 493, "statues_count": 21767 }, "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-03T10:00:13.000Z"), "id": 683709462301945856, "text": "S/O markeryskingdom Men with Vision can't be stopped. This man has Vision. �� #SilverLife @ I-285… https://t.co/kScPAeGaYf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.48767616,33.84183699"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SilverLife" }}, "user": { "id": 104159289, "name": "Silvy", "screen_name": "BSilverLife", "lang": "en", "location": "Atlanta, GA", "create_at": date("2010-01-12"), "description": "Artist. Vocalist.Producer. Writer. Entrepreneur. Graphic Designer. Fashion Design. Follow Me to Victory. Not Just A Brand, It's A Life Style...", "followers_count": 1323, "friends_count": 586, "statues_count": 37937 }, "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-01-03T10:00:13.000Z"), "id": 683709462440329216, "text": "RN First Line Supervisor - St Joseph's Medical Center: (#Stockton, CA) https://t.co/7s0mxt45uF #Nursing #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.3255688,37.9727529"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Stockton", "Nursing", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22636267, "name": "TMJ-SAC Nursing Jobs", "screen_name": "tmj_sac_nursing", "lang": "en", "location": "Sacramento, CA", "create_at": date("2009-03-03"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Sacramento, CA. Need help? Tweet us at @CareerArc!", "followers_count": 574, "friends_count": 303, "statues_count": 555 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Country Club, CA", "id": "73d5b6835d824120", "name": "Country Club", "place_type": "city", "bounding_box": rectangle("-121.367279,37.958358 -121.313875,37.980262") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 616651, "cityName": "Country Club" } }
+{ "create_at": datetime("2016-01-03T10:00:13.000Z"), "id": 683709462763274241, "text": "Morning showers this afternoon, high 56 (13 C). Low 39 (4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575147, "name": "Baton Rouge Weather", "screen_name": "BatonRougeWX", "lang": "en", "location": "Baton Rouge, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Baton Rouge, LA", "followers_count": 2872, "friends_count": 92, "statues_count": 7867 }, "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-03T10:00:13.000Z"), "id": 683709462868275200, "text": "What to every hope every body good bot watch sum sun NFL football wack ass giant eagle game flip to steerer brown hame", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4207307308, "name": "Narcoticz Powell Jr", "screen_name": "NARCOTICZJR", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2015-11-16"), "description": "narcoticz i am from harlem ny been doing music for a long time dropped my first single my world in 2012on dj fusions mixtape", "followers_count": 738, "friends_count": 1336, "statues_count": 1899 }, "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-03T10:00:13.000Z"), "id": 683709462897504256, "text": "We're #hiring! Click to apply: Supply Chain Analyst - https://t.co/lueZvde51c #SupplyChain #Columbus, OH #Job #Jobs #CareerArc", "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": {{ "hiring", "SupplyChain", "Columbus", "Job", "Jobs", "CareerArc" }}, "user": { "id": 188185944, "name": "ColumbusOH S-Chain", "screen_name": "tmj_CHH_schn", "lang": "en", "location": "Columbus, OH", "create_at": date("2010-09-07"), "description": "Follow this account for geo-targeted Supply Chain job tweets in Columbus, OH from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 284, "friends_count": 264, "statues_count": 31 }, "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-03T10:00:13.000Z"), "id": 683709462901751808, "text": "The only time stealing is okay is if your taking something someone stole from you back", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3437825834, "name": "damn", "screen_name": "klc_brooke", "lang": "en", "location": "null", "create_at": date("2015-09-03"), "description": "tx | @astralactavis", "followers_count": 136, "friends_count": 129, "statues_count": 828 }, "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-01-03T10:00:13.000Z"), "id": 683709462918619136, "text": "If you don't like girls and big booty unfollow me right tf now.\n\n������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1514738234, "name": "#TheyCallMeFlash ⚡️", "screen_name": "MrUnknownEnt", "lang": "en", "location": "null", "create_at": date("2013-06-13"), "description": "|A realist with rhythm| IG: Strictly_Flash | The founder of #UnknownEnt | Street Performer | Ball Player | #FlashFriday | #TheyCallMeFlash", "followers_count": 1815, "friends_count": 2008, "statues_count": 14860 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Binghamton, NY", "id": "859b7f1502db8fe9", "name": "Binghamton", "place_type": "city", "bounding_box": rectangle("-75.95015,42.067082 -75.857034,42.156489") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36007, "countyName": "Broome", "cityID": 3606607, "cityName": "Binghamton" } }
+{ "create_at": datetime("2016-01-03T10:00:13.000Z"), "id": 683709463245754370, "text": "@chelseagsummers thank YOU", "in_reply_to_status": 683708357329444865, "in_reply_to_user": 14887755, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14887755 }}, "user": { "id": 537756186, "name": "jillian grace", "screen_name": "jllmeehan", "lang": "en", "location": "emerson college", "create_at": date("2012-03-26"), "description": "sleepy teen, managing editor @inconnumag, featured contributor @femsplain", "followers_count": 361, "friends_count": 430, "statues_count": 4392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newtown, PA", "id": "660db7c8f1d85fd3", "name": "Newtown", "place_type": "city", "bounding_box": rectangle("-74.939004,40.220891 -74.924836,40.238012") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42107, "countyName": "Schuylkill", "cityID": 4254264, "cityName": "Newtown" } }
+{ "create_at": datetime("2016-01-03T10:00:13.000Z"), "id": 683709463597985792, "text": "[Sun Jan 3rd, 12:00pm] Temperature: 66.92 F, 19.4 C; Humidity: 40.4%; Light: 13.67%. https://t.co/UI5qWnQ21d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 269430133, "name": "Shoemaker Sandbox", "screen_name": "shoe_sandbox", "lang": "en", "location": "Earth (for now). ", "create_at": date("2011-03-20"), "description": "I am a bot. Developer account for @shoemaker.", "followers_count": 7, "friends_count": 1, "statues_count": 24453 }, "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-03T10:00:13.000Z"), "id": 683709463602151424, "text": "quiero ir a plaza", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-66.4105783,18.376976"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2756024165, "name": "golden", "screen_name": "orspvct66", "lang": "es", "location": "2golden boobs ", "create_at": date("2014-08-30"), "description": "Me golden boobs (.y.) ( @fettywap ) bichote @vince_169 ****JACK Ü, GALANTIS, YELLOW CLAW ****", "followers_count": 1722, "friends_count": 382, "statues_count": 77891 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Quebrada Arenas, Puerto Rico", "id": "014d0e1a79b870a2", "name": "Quebrada Arenas", "place_type": "city", "bounding_box": rectangle("-66.424202,18.368648 -66.404714,18.402386") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72145, "countyName": "Vega Baja" } }
+{ "create_at": datetime("2016-01-03T10:00:13.000Z"), "id": 683709463618928640, "text": "@collette04 @wwwbigbaldhead @NormansCat @dawtiedawn @bolinger_diana @araeropele @PerreaultNora @helloherman1 GM! �� https://t.co/8E6YJRO97t", "in_reply_to_status": 683632020409442306, "in_reply_to_user": 38011127, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 38011127, 25460615, 2215092512, 3572100559, 3802786937, 33313327, 3128113718, 2866024047 }}, "user": { "id": 3186930858, "name": "❤Beth", "screen_name": "BethNelson75", "lang": "en", "location": "Sedona, AZ", "create_at": date("2015-05-06"), "description": "Loving life. Hanging with the dogs and chickens. #TWD #TWDFamily ❤Love NormanReedus❤", "followers_count": 581, "friends_count": 690, "statues_count": 5896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Verde Village, AZ", "id": "00013682435d03c8", "name": "Verde Village", "place_type": "city", "bounding_box": rectangle("-112.036167,34.68845 -111.954999,34.730402") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4025, "countyName": "Yavapai", "cityID": 479830, "cityName": "Verde Village" } }
+{ "create_at": datetime("2016-01-03T10:00:13.000Z"), "id": 683709463648313344, "text": "Siri says \"you're too sexy\" ha ha ha. #iAmYourRealtorForLife https://t.co/vPa4xIXvOT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "iAmYourRealtorForLife" }}, "user": { "id": 583841612, "name": "Chris Narvaez", "screen_name": "NarvaezRealtor", "lang": "en", "location": "San Diego California", "create_at": date("2012-05-18"), "description": "I will provide the best real estate service by paying close attention to what is important to my client. #iAmYourRealtorForLife", "followers_count": 120, "friends_count": 130, "statues_count": 939 }, "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-03T10:00:13.000Z"), "id": 683709464399220738, "text": "I'm constantly deleting Instagram and Snapchat because there's no point to it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4411174240, "name": "omar", "screen_name": "ngtnyzx", "lang": "en", "location": "null", "create_at": date("2015-12-07"), "description": "ny☄ only need you 5", "followers_count": 5, "friends_count": 5, "statues_count": 376 }, "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-03T10:00:13.000Z"), "id": 683709464415879168, "text": "@thenateralph @watchlar66 https://t.co/IXtYNBQS0x", "in_reply_to_status": 683531507332300800, "in_reply_to_user": 74484200, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 74484200, 2811719940 }}, "user": { "id": 209568114, "name": "Devout Coward", "screen_name": "LesJulia9", "lang": "en", "location": "New Orleans", "create_at": date("2010-10-29"), "description": "We can't do anything..my fav subjects racism/philosophy, Your will is not yours, we are all innocent, we all go to heaven", "followers_count": 422, "friends_count": 1085, "statues_count": 36579 }, "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-03T10:00:13.000Z"), "id": 683709464516509698, "text": "We're #hiring! Read about our latest #job opening here: Carpenter - https://t.co/1E1UqfhwrX #Allentown, PA #SkilledTrade #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.4901833,40.6084305"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Allentown", "SkilledTrade", "CareerArc" }}, "user": { "id": 118562628, "name": "TMJ-PA Skl. Trades", "screen_name": "tmj_PA_skltrd", "lang": "en", "location": "Pennsylvania", "create_at": date("2010-02-28"), "description": "Follow this account for geo-targeted Skilled Trade job tweets in Pennsylvania Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 338, "friends_count": 285, "statues_count": 86 }, "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-03T10:00:13.000Z"), "id": 683709464625545216, "text": "Game day, let's get this 1 seed #KeepPounding", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "KeepPounding" }}, "user": { "id": 581456754, "name": "Braden Masten", "screen_name": "McMasty_5", "lang": "en", "location": "Springfield, IL/Normal, IL", "create_at": date("2012-05-15"), "description": "ISU '19 #Kanye2020 Snap: MastenbaterB", "followers_count": 505, "friends_count": 372, "statues_count": 9549 }, "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-01-03T10:00:13.000Z"), "id": 683709464730439680, "text": "@PabloGoP @libbyweir I humbly apologize", "in_reply_to_status": 683708938584330240, "in_reply_to_user": 312721063, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 312721063, 273034520 }}, "user": { "id": 3214654918, "name": "joshua 'danger' prim", "screen_name": "joshprim2", "lang": "en", "location": "in the moment", "create_at": date("2015-04-27"), "description": "null", "followers_count": 479, "friends_count": 314, "statues_count": 1715 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-03T10:00:13.000Z"), "id": 683709464814333952, "text": "Join the The MENTOR Network team! See our latest #Healthcare #job opening here: https://t.co/h4V3FII8WQ #SaintCloud, MN #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.1702778,45.5538889"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "SaintCloud", "Hiring" }}, "user": { "id": 759953168, "name": "MENTOR Network Jobs", "screen_name": "JoinMENTORNetwk", "lang": "en", "location": "null", "create_at": date("2012-08-15"), "description": "null", "followers_count": 23, "friends_count": 0, "statues_count": 2255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "St Cloud, MN", "id": "78775f08fd153aed", "name": "St Cloud", "place_type": "city", "bounding_box": rectangle("-94.260166,45.471059 -94.109661,45.590382") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27145, "countyName": "Stearns", "cityID": 2756896, "cityName": "St. Cloud" } }
+{ "create_at": datetime("2016-01-03T10:00:13.000Z"), "id": 683709464818663425, "text": "�� A belated HAPPY 2016 from all of us at Stratford Library! We're excited for the year ahead --… https://t.co/3cLDqURO2i", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.1310425,41.1906204"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36520288, "name": "Stratford Library CT", "screen_name": "StratfordLib", "lang": "en", "location": "Stratford, CT", "create_at": date("2009-04-29"), "description": "The public library in Stratford, Connecticut", "followers_count": 1531, "friends_count": 1030, "statues_count": 4599 }, "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-03T10:00:13.000Z"), "id": 683709465175154688, "text": "♚ @ Maggiano's Little Italy Buckhead https://t.co/muDqzp7K3o", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.3674927,33.8484917"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 190024815, "name": "Taylor Jorgensen", "screen_name": "taylorrj3", "lang": "en", "location": "null", "create_at": date("2010-09-12"), "description": "null", "followers_count": 655, "friends_count": 192, "statues_count": 8352 }, "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-03T10:00:13.000Z"), "id": 683709465208696833, "text": "At least I can watch most of the bills game", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 973101067, "name": "Samuel", "screen_name": "samuel_utley", "lang": "en", "location": "null", "create_at": date("2012-11-26"), "description": "Let's go Heat.\n\n\n\nFuture husband of @kourtneykardash\n\n\n\nFredonia '18", "followers_count": 510, "friends_count": 323, "statues_count": 21302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gowanda, NY", "id": "fc977b55bb9aac6a", "name": "Gowanda", "place_type": "city", "bounding_box": rectangle("-78.954709,42.449728 -78.919618,42.47557") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3629630, "cityName": "Gowanda" } }
+{ "create_at": datetime("2016-01-03T10:00:13.000Z"), "id": 683709465221189632, "text": "Interested in a #Hospitality #job near #LasVegas, NV? This could be a great fit: https://t.co/3tGTvxEbU7 #VegasLife https://t.co/6UbqU29QHd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.1745805,36.1062236"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "LasVegas", "VegasLife" }}, "user": { "id": 3089847157, "name": "Mandarin Las Vegas", "screen_name": "MO_LVJobs", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-03-16"), "description": "Nevada's only Triple Forbes Five-Star luxury destination located on the world-famous #LasVegas Strip. Apply here to open #jobs.", "followers_count": 83, "friends_count": 59, "statues_count": 44 }, "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-03T10:00:13.000Z"), "id": 683709465460248576, "text": "@kissmeQuan it's all good. I try not to be divisive.", "in_reply_to_status": 683708924218978304, "in_reply_to_user": 27184509, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27184509 }}, "user": { "id": 301699308, "name": "Mean Greeze Okerlund", "screen_name": "DeeGreezy", "lang": "en", "location": "Burning the mistletoe", "create_at": date("2011-05-19"), "description": "Legend. Flowing straight from the survivor scroll.", "followers_count": 1180, "friends_count": 1027, "statues_count": 151802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Farmington Hills, MI", "id": "0b40afc99807b6ac", "name": "Farmington Hills", "place_type": "city", "bounding_box": rectangle("-83.437523,42.439001 -83.316839,42.529556") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2627440, "cityName": "Farmington Hills" } }
+{ "create_at": datetime("2016-01-03T10:00:13.000Z"), "id": 683709465518944256, "text": "Day off chillin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 172809674, "name": "Eleanor Guerra", "screen_name": "EleanorGuerra", "lang": "en", "location": "null", "create_at": date("2010-07-30"), "description": "somebody save me.", "followers_count": 119, "friends_count": 523, "statues_count": 7960 }, "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-03T10:00:13.000Z"), "id": 683709465539956738, "text": "Morning showers this afternoon, high 54 (12 C). Low 45 (7 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575204, "name": "New Orleans Weather", "screen_name": "NewOrleansWX", "lang": "en", "location": "New Orleans, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for New Orleans, LA", "followers_count": 2899, "friends_count": 92, "statues_count": 7885 }, "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-03T10:00:14.000Z"), "id": 683709465812549636, "text": "destroyer_photography shooting with scighera461 @ Polaris Recording Studio https://t.co/JOmQwQvtJA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.9988403,42.3190193"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2672056541, "name": "Brandon James Liddy", "screen_name": "lidforceone", "lang": "en", "location": "Canada", "create_at": date("2014-07-04"), "description": "Professional Photographer * Photojournalist * News Junkie", "followers_count": 144, "friends_count": 439, "statues_count": 994 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Windsor, Ontario", "id": "15b0a643e7bd22c7", "name": "Windsor", "place_type": "city", "bounding_box": rectangle("-83.113623,42.233665 -82.890548,42.356225") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-01-03T10:00:14.000Z"), "id": 683709466013913088, "text": "#besties @ PUB 1848 https://t.co/Ei6UEqe8vr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.51554576,41.50924193"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "besties" }}, "user": { "id": 4123793308, "name": "Audi", "screen_name": "dollfacetattoo8", "lang": "en", "location": "Rock Island, IL", "create_at": date("2015-11-05"), "description": "Simon says, bring me a beer!!", "followers_count": 395, "friends_count": 687, "statues_count": 658 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moline, IL", "id": "b8e37f613fedf510", "name": "Moline", "place_type": "city", "bounding_box": rectangle("-90.539878,41.455977 -90.431572,41.518476") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17161, "countyName": "Rock Island", "cityID": 1749867, "cityName": "Moline" } }
+{ "create_at": datetime("2016-01-03T10:00:14.000Z"), "id": 683709466034966528, "text": "@_mkellzz https://t.co/oGTynATUwZ", "in_reply_to_status": -1, "in_reply_to_user": 3392423147, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3392423147 }}, "user": { "id": 3358131189, "name": "This Bitch", "screen_name": "_jaispainn", "lang": "en", "location": "null", "create_at": date("2015-07-03"), "description": "P I M P 21", "followers_count": 322, "friends_count": 280, "statues_count": 1973 }, "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-03T10:00:14.000Z"), "id": 683709466143928320, "text": "If you're a #Sales professional in #Indianapolis, IN, check out this #job: https://t.co/OqPHA6MM1P #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.1580423,39.7683765"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "Indianapolis", "job", "Hiring", "CareerArc" }}, "user": { "id": 21725407, "name": "TMJ - IND Sales Jobs", "screen_name": "tmj_ind_sales", "lang": "en", "location": "Indianapolis, IN", "create_at": date("2009-02-23"), "description": "Follow this account for geo-targeted Sales job tweets in Indianapolis, IN. Need help? Tweet us at @CareerArc!", "followers_count": 413, "friends_count": 308, "statues_count": 304 }, "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-01-03T10:00:14.000Z"), "id": 683709466257141760, "text": "@MoKhaLatte can't be nice doesn't get me nothing", "in_reply_to_status": 683709293204357120, "in_reply_to_user": 35231173, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 35231173 }}, "user": { "id": 66648758, "name": "NotAaron", "screen_name": "Linuxtherebel", "lang": "en", "location": "Dallas, TX", "create_at": date("2009-08-18"), "description": "my @ name is my gamer tag. black women matter.", "followers_count": 2613, "friends_count": 661, "statues_count": 356640 }, "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-03T10:00:14.000Z"), "id": 683709466269749248, "text": "I wish I had the rolling eyes emoji but I'm to lazy to update fr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 554043432, "name": "JOEYY BOYY WIT 2 YYs", "screen_name": "joeyyy_boyy", "lang": "en", "location": "mindyourznigga", "create_at": date("2012-04-14"), "description": "HAPPY HOLIDAYS MFS!!!! snapchat: yourguyjoe instagram: joeyy_boyy_", "followers_count": 1053, "friends_count": 1204, "statues_count": 25671 }, "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-01-03T10:00:14.000Z"), "id": 683709466496364544, "text": "I would kick one of my children in the face for a gyro right now. 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": 220792339, "name": "April Sapienza", "screen_name": "AprilSapienza", "lang": "en", "location": "Amherst, OH", "create_at": date("2010-11-28"), "description": "null", "followers_count": 199, "friends_count": 398, "statues_count": 15066 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Amherst, OH", "id": "911b22d874c731d0", "name": "Amherst", "place_type": "city", "bounding_box": rectangle("-82.265659,41.373736 -82.164579,41.426754") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39093, "countyName": "Lorain", "cityID": 3901798, "cityName": "Amherst" } }
+{ "create_at": datetime("2016-01-03T10:00:14.000Z"), "id": 683709466508836869, "text": "My back stay itchy. ���� good thing my aunt bought me a back scratcher for xmas. Bc it's cause be a process. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2706917790, "name": "Drippy", "screen_name": "jaaay31", "lang": "en", "location": "El Paso, TX", "create_at": date("2014-08-04"), "description": "null", "followers_count": 431, "friends_count": 358, "statues_count": 7054 }, "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-03T10:00:14.000Z"), "id": 683709466655744000, "text": "@xlataviax_4 thank you tay �� love you too", "in_reply_to_status": 683706009756516356, "in_reply_to_user": 2565311099, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2565311099 }}, "user": { "id": 1195606135, "name": "Birthday Princess ✨♑", "screen_name": "iBeenCoolie__", "lang": "en", "location": "null", "create_at": date("2013-02-18"), "description": "I fuck with myself & myself only ❗", "followers_count": 1066, "friends_count": 459, "statues_count": 16003 }, "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-01-03T10:00:14.000Z"), "id": 683709466802524160, "text": "First Sunday of 2016. I'm radically focused and excited about the future. #sunday #radicalfocus #poshpews https://t.co/01JACPCgFl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.39601628,33.70605473"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sunday", "radicalfocus", "poshpews" }}, "user": { "id": 317525423, "name": "Robyn Young", "screen_name": "PoshPews", "lang": "en", "location": "Atlanta, GA", "create_at": date("2011-06-14"), "description": "|Runner|Corporate Cutie|Consultant|Blogger| IG & Periscope - @PoshPews", "followers_count": 311, "friends_count": 277, "statues_count": 7084 }, "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-03T10:00:14.000Z"), "id": 683709466907262976, "text": "@mindlust @maperformer @Madonna @Rebel_Lance holy shit", "in_reply_to_status": 683707718801047553, "in_reply_to_user": 61239604, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 61239604, 3051752947, 512700138, 151753287 }}, "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 32 YRS & FOR LIFE - Love Shih Tzus for animal rights I like all types of music - Gay rights - Love Jesus -", "followers_count": 8818, "friends_count": 9700, "statues_count": 149935 }, "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-01-03T10:00:14.000Z"), "id": 683709467037425664, "text": "I also need a shredder #ASAP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ASAP" }}, "user": { "id": 201262177, "name": "Chelly ℒℴѵℯ ©", "screen_name": "MsChellyLove", "lang": "en", "location": "Miami, FL", "create_at": date("2010-10-11"), "description": "A winner rebukes and forgives; a loser is too timid to rebuke and too petty to forgive. [☮&♥] ♑ * ツ ♡ #ChellyLove ♡", "followers_count": 923, "friends_count": 652, "statues_count": 47145 }, "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-03T10:00:14.000Z"), "id": 683709467104534530, "text": "@WESTAYCHILL Check out my brand new music video \"Smooth\" #EvolveCrew #Norfolk #KnowingBetter https://t.co/wCNRmlqet0", "in_reply_to_status": -1, "in_reply_to_user": 170899491, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "EvolveCrew", "Norfolk", "KnowingBetter" }}, "user_mentions": {{ 170899491 }}, "user": { "id": 21249017, "name": "Page", "screen_name": "Pagemuzik", "lang": "en", "location": "fethernfinbout2getaboneless VA", "create_at": date("2009-02-18"), "description": "Tryin to spread love & shit to the darkest reaches of the galaxy http://www.datpiff.com/pop-mixtape-download.php?id=m6d433ad", "followers_count": 2148, "friends_count": 380, "statues_count": 14966 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dix Hills, NY", "id": "18165a3f2b139f0f", "name": "Dix Hills", "place_type": "city", "bounding_box": rectangle("-73.380745,40.767592 -73.29345,40.83856") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3620687, "cityName": "Dix Hills" } }
+{ "create_at": datetime("2016-01-03T10:00:14.000Z"), "id": 683709467150684160, "text": "Não sejam o tipo de pessoa que só me fala quando precisa de ajuda", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 923839826, "name": "cate", "screen_name": "thevxrve", "lang": "en", "location": "manhattan, NY", "create_at": date("2012-11-03"), "description": "i've got more wit, a better kiss, a hotter touch, a better fuck Ψ\n#WeMissYouCory", "followers_count": 1255, "friends_count": 282, "statues_count": 89476 }, "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-03T10:00:14.000Z"), "id": 683709467251216385, "text": "We're #hiring! Read about our latest #job opening here: AREA HR MANAGER / Michigan - https://t.co/zBu5LT6dKR #Detroit, MI #HR #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.0457538,42.331427"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Detroit", "HR", "CareerArc" }}, "user": { "id": 23130397, "name": "TMJ-DTW HR Jobs", "screen_name": "tmj_dtw_hr", "lang": "en", "location": "Detroit, MI", "create_at": date("2009-03-06"), "description": "Follow this account for geo-targeted Human Resources job tweets in Detroit, MI. Need help? Tweet us at @CareerArc!", "followers_count": 387, "friends_count": 294, "statues_count": 19 }, "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-03T10:00:14.000Z"), "id": 683709467909734401, "text": "all god want is cooperation .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 304116531, "name": "ble$$ed.", "screen_name": "mandy_jenise", "lang": "en", "location": "following gods footsteps ", "create_at": date("2011-05-23"), "description": "null", "followers_count": 1732, "friends_count": 1971, "statues_count": 31878 }, "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-03T10:00:14.000Z"), "id": 683709468094234624, "text": "Beautiful view to wake up and see from my room. Feel peace today, much needed. #monterey #peace #ocean https://t.co/8KutFneP8W", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.85896,36.61122"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "monterey", "peace", "ocean" }}, "user": { "id": 2387751625, "name": "Stephen Andrew", "screen_name": "StephenAndrewXL", "lang": "en", "location": "Elk Grove, CA", "create_at": date("2014-03-13"), "description": "Advocate of Optimism | Fitness Nut | Gay Male | Member of Humanity | Hearing Impaired", "followers_count": 57, "friends_count": 66, "statues_count": 271 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sandals Lounge", "id": "07d9d76505c86000", "name": "Sandals Lounge", "place_type": "poi", "bounding_box": rectangle("-121.85896009999999,36.6112199 -121.85896,36.61122") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 670742, "cityName": "Seaside" } }
+{ "create_at": datetime("2016-01-03T10:00:14.000Z"), "id": 683709468354441218, "text": "@donmoyn #sigh", "in_reply_to_status": 683708861237313536, "in_reply_to_user": 2804332645, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "sigh" }}, "user_mentions": {{ 2804332645 }}, "user": { "id": 45510372, "name": "andy saultz", "screen_name": "andysaultz", "lang": "en", "location": "null", "create_at": date("2009-06-07"), "description": "Ast. Prof at Miami (OH) in Educational Leadership. Research education policy, governance, and politics. An Oregonian at heart.", "followers_count": 612, "friends_count": 948, "statues_count": 6691 }, "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-03T10:00:14.000Z"), "id": 683709468396265473, "text": "@__Pyh what's the name?", "in_reply_to_status": 683547927453810688, "in_reply_to_user": 433633435, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 433633435 }}, "user": { "id": 2727409304, "name": "QDotTheRebel", "screen_name": "ThatRebelQOffic", "lang": "en", "location": "Gardena, CA", "create_at": date("2014-08-12"), "description": "SC&IG: thatrebelq #1765HennyBoyz", "followers_count": 626, "friends_count": 421, "statues_count": 10423 }, "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-03T10:00:14.000Z"), "id": 683709468580827136, "text": "Snowy day, watching football.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17106455, "name": "Stephanie", "screen_name": "DressUpPrague", "lang": "en", "location": "San Diego", "create_at": date("2008-11-01"), "description": "i take my twist with a shout.", "followers_count": 62, "friends_count": 130, "statues_count": 6564 }, "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-03T10:00:14.000Z"), "id": 683709468673114112, "text": "Letting go of self and focusing on others gives us the ability to change the world...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.3126543,38.755834"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3842705840, "name": "ClassyConfidenceClub", "screen_name": "classysafety", "lang": "en", "location": "null", "create_at": date("2015-10-09"), "description": "null", "followers_count": 9, "friends_count": 8, "statues_count": 55 }, "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-03T10:00:14.000Z"), "id": 683709468937330688, "text": "Want to work in #Greenfield, MA? View our latest opening: https://t.co/kxhs6Bz9Ko #Healthcare #PT #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.5995339,42.5875857"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Greenfield", "Healthcare", "PT", "Job", "Jobs", "Hiring" }}, "user": { "id": 2365967174, "name": "Aureus Medical Jobs", "screen_name": "aureusmedjobs", "lang": "en", "location": "Nationwide", "create_at": date("2014-02-28"), "description": "Follow @aureusmedjobs for #travelnursing, #traveltherapy, imaging, med lab, physicians, NP, and PA #jobs nationwide. Follow our company @aureusmedical.", "followers_count": 372, "friends_count": 1, "statues_count": 10199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenfield, MA", "id": "f9418b036f66d714", "name": "Greenfield", "place_type": "city", "bounding_box": rectangle("-72.639338,42.566451 -72.560588,42.643963") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25011, "countyName": "Franklin", "cityID": 2527100, "cityName": "Greenfield Town" } }
+{ "create_at": datetime("2016-01-03T10:00:14.000Z"), "id": 683709468970991616, "text": "@_gennaayy forsure you're still in lex right? If I come down to celebrate tonight I'll hit you up.", "in_reply_to_status": 683709225571364865, "in_reply_to_user": 1544614015, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1544614015 }}, "user": { "id": 1859881242, "name": "Ben Bonfert", "screen_name": "Bonfert", "lang": "en", "location": "null", "create_at": date("2013-09-13"), "description": "Work Hard,Play Harder", "followers_count": 337, "friends_count": 295, "statues_count": 912 }, "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-03T10:00:14.000Z"), "id": 683709469432258560, "text": "I just think holding you in my arms and laying in bed with in silence would make everything better.", "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": "Young man interested in Architecture. Virgin Billionaires Series. Writers famous and Obscure on Love, Sex, Money, Friendship, Family, Work, and much more.", "followers_count": 1186, "friends_count": 1896, "statues_count": 48441 }, "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-03T10:00:14.000Z"), "id": 683709469595807744, "text": "@JLaupua https://t.co/MR5pAHcp8g", "in_reply_to_status": -1, "in_reply_to_user": 3144878014, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3144878014 }}, "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", "followers_count": 423, "friends_count": 1054, "statues_count": 4431 }, "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-03T10:00:14.000Z"), "id": 683709469671448578, "text": "@amandapeacher keep reporting!", "in_reply_to_status": -1, "in_reply_to_user": 174716364, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 174716364 }}, "user": { "id": 14270329, "name": "Ryan J. Davis", "screen_name": "RyanNewYork", "lang": "en", "location": "Bushwick, Brooklyn, NYC", "create_at": date("2008-03-31"), "description": "@FOUNDERorg. Formerly @BSD, @Vocativ, @GovHowardDean's 2004 Campaign. Ex-Theatre Director. Secular, Progressive, Gay. FB: http://on.fb.me/ov7sDN", "followers_count": 50297, "friends_count": 35540, "statues_count": 60054 }, "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-03T10:00:14.000Z"), "id": 683709469683929088, "text": "It's kickoff time! #WASvsDAL #HTTR #Redskins #BeatDallas #HotelPlanner #ProudSponsor https://t.co/P90H20Wh3R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WASvsDAL", "HTTR", "Redskins", "BeatDallas", "HotelPlanner", "ProudSponsor" }}, "user": { "id": 2828340217, "name": "Meetings.com", "screen_name": "MeetingsDotCom", "lang": "en", "location": "West Palm Beach, FL", "create_at": date("2014-09-23"), "description": "The official Twitter account of http://Meetings.com", "followers_count": 261, "friends_count": 855, "statues_count": 56 }, "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-01-03T10:00:14.000Z"), "id": 683709469793087489, "text": "@coffeewithemily @jordannpowers @shoneiripoll Yeah buddy! ����������������������☀️��⛴����", "in_reply_to_status": 683709008629248000, "in_reply_to_user": 16302483, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16302483, 18738093, 343777789 }}, "user": { "id": 28721177, "name": "Sarahjane", "screen_name": "pattimay0nnaise", "lang": "en", "location": "Mattydale, NY", "create_at": date("2009-04-03"), "description": "22. Syracuse University grad student. Preschool teacher to the cutest two/three year olds on the planet.", "followers_count": 451, "friends_count": 363, "statues_count": 13021 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mattydale, NY", "id": "71883cc7f755b4c1", "name": "Mattydale", "place_type": "city", "bounding_box": rectangle("-76.186018,43.091882 -76.116769,43.119031") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36067, "countyName": "Onondaga", "cityID": 3646151, "cityName": "Mattydale" } }
+{ "create_at": datetime("2016-01-03T10:00:14.000Z"), "id": 683709469868425216, "text": "Go @Patriots!! Just win today. #BeatTheDolphins", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BeatTheDolphins" }}, "user_mentions": {{ 31126587 }}, "user": { "id": 216470929, "name": "Travis ✌️", "screen_name": "kirk2266", "lang": "en", "location": "Diné Bikéyah ", "create_at": date("2010-11-16"), "description": "#StarTrek: TOS & New England #Patriots. #ISupportBrady", "followers_count": 671, "friends_count": 487, "statues_count": 17661 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tse Bonito, NM", "id": "012ca6de2c7339df", "name": "Tse Bonito", "place_type": "city", "bounding_box": rectangle("-109.046352,35.648485 -109.034925,35.660672") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35031, "countyName": "McKinley", "cityID": 3579885, "cityName": "Tse Bonito" } }
+{ "create_at": datetime("2016-01-03T10:00:14.000Z"), "id": 683709469918806016, "text": "I want something real", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1858778556, "name": "HXRMONI", "screen_name": "harmoni_garbade", "lang": "en", "location": "843 ", "create_at": date("2013-09-12"), "description": "Going through a crisis at 17\n\nIG》@ harms2", "followers_count": 445, "friends_count": 316, "statues_count": 3609 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden City, SC", "id": "596fb7b1e3f22a38", "name": "Garden City", "place_type": "city", "bounding_box": rectangle("-79.069664,33.535979 -78.972953,33.637127") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4528455, "cityName": "Garden City" } }
+{ "create_at": datetime("2016-01-03T10:00:15.000Z"), "id": 683709469969235968, "text": "Thug life https://t.co/2JjvCdd8a4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 319947288, "name": "kayla", "screen_name": "steveee_kirwin", "lang": "en", "location": "NJ | VA ", "create_at": date("2011-06-18"), "description": "null", "followers_count": 371, "friends_count": 360, "statues_count": 10462 }, "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-03T10:00:15.000Z"), "id": 683709470057234432, "text": "I may be a bit biased, but this guy is my favorite preacher.… https://t.co/9UzBcIGa86", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.03796,39.678578"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 61257064, "name": "Brittany Crimmel", "screen_name": "brittanycrimmel", "lang": "en", "location": "Denver, CO", "create_at": date("2009-07-29"), "description": "null", "followers_count": 104, "friends_count": 192, "statues_count": 427 }, "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-03T10:00:15.000Z"), "id": 683709470157987840, "text": "Excited to see what the #Eagles have in store for the day #FlyEaglesFly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Eagles", "FlyEaglesFly" }}, "user": { "id": 583028651, "name": "Drox", "screen_name": "derekjones198", "lang": "en", "location": "null", "create_at": date("2012-05-17"), "description": "Beer, Sports, Comics, Tech, yeah that's me", "followers_count": 116, "friends_count": 150, "statues_count": 2179 }, "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-03T10:00:15.000Z"), "id": 683709470229147648, "text": "Sunny this afternoon, high 57 (14 C). Low 35 (2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575255, "name": "Shreveport Weather", "screen_name": "ShreveportWX", "lang": "en", "location": "Shreveport, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Shreveport, LA", "followers_count": 698, "friends_count": 92, "statues_count": 7943 }, "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-03T10:00:15.000Z"), "id": 683709470292230148, "text": "Tonight's forecast is overcast and a low of 21°F. #DrinkingStrawDay #quadrantidsMeteorShower #NationalChocolateCoveredCherryDay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.5333,41.5167"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DrinkingStrawDay", "quadrantidsMeteorShower", "NationalChocolateCoveredCherryDay" }}, "user": { "id": 42666285, "name": "Quad Cities Weather", "screen_name": "qcweather", "lang": "en", "location": "Quad Cities", "create_at": date("2009-05-26"), "description": "Is it going to be warm, wet or cold in the Quad Cities today? You can also find us on Facebook.", "followers_count": 740, "friends_count": 0, "statues_count": 11440 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rock Island Arsenal, IL", "id": "8f966c3da836a196", "name": "Rock Island Arsenal", "place_type": "city", "bounding_box": rectangle("-90.570223,41.509992 -90.511303,41.523635") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17161, "countyName": "Rock Island", "cityID": 1765104, "cityName": "Rock Island Arsenal" } }
+{ "create_at": datetime("2016-01-03T10:00:15.000Z"), "id": 683709470736789504, "text": "I want go see ride along 2 when it come out ...��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2924955865, "name": "ErneshaC❤️", "screen_name": "ErneshaMcM", "lang": "en", "location": "DiamondAboveAll", "create_at": date("2014-12-09"), "description": "TaeMyHeart❤️ Troynesha•Ari•Quinckaa•❤️", "followers_count": 587, "friends_count": 595, "statues_count": 18491 }, "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-03T10:00:15.000Z"), "id": 683709470795403266, "text": "#Retail #Job alert: Sales Associate | Sterling Jewelers | #Aurora, OH https://t.co/9BA4vF3Wt2 #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3858178,41.3535337"), "retweet_count": 0, "lang": "nl", "is_retweet": false, "hashtags": {{ "Retail", "Job", "Aurora", "Jobs", "Hiring" }}, "user": { "id": 3248774184, "name": "Kay Jewelers", "screen_name": "JobsAtKay", "lang": "en", "location": "null", "create_at": date("2015-06-18"), "description": "Your Future Starts Here.", "followers_count": 5, "friends_count": 0, "statues_count": 1108 }, "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-01-03T10:00:15.000Z"), "id": 683709471168851970, "text": "#morningrun #nofilter @ Goleta Pier https://t.co/whyigq923I", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.82915415,34.41652493"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "morningrun", "nofilter" }}, "user": { "id": 56993835, "name": "Happy Dooe Year!", "screen_name": "Dooezer", "lang": "en", "location": "Boston ", "create_at": date("2009-07-15"), "description": "living in the @hereandnow. quite contrary. fluent in journalish and sarcasm. thoughts are merely humble Dooe opinion.", "followers_count": 3622, "friends_count": 894, "statues_count": 30305 }, "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-01-03T10:00:15.000Z"), "id": 683709471273660416, "text": "125 will feature Jon Haas vs Dalton Henderson of VMI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 732920094, "name": "BU Huskies Wrestling", "screen_name": "BU_Wrestling", "lang": "en", "location": "400 E. Second Street ", "create_at": date("2012-08-02"), "description": "null", "followers_count": 1369, "friends_count": 397, "statues_count": 1961 }, "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-03T10:00:15.000Z"), "id": 683709471802015744, "text": "high�� sad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 358863825, "name": "$teezo⛷", "screen_name": "TopFlightRod", "lang": "en", "location": "Dallas ✈️ Baton Rouge", "create_at": date("2011-08-20"), "description": "if you aint got no sauce then you lost, but you also can get lost in the sauce #LSU", "followers_count": 5021, "friends_count": 1581, "statues_count": 144469 }, "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-03T10:00:15.000Z"), "id": 683709471843991552, "text": "Software Developer 1 - Oracle: (#RedwoodShores, CA) https://t.co/DpITAQQWmP #IT #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.2455364,37.5364134"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "hashtags": {{ "RedwoodShores", "IT", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 112596930, "name": "TMJ-CA IT Jobs", "screen_name": "tmj_CA_it", "lang": "en", "location": "California", "create_at": date("2010-02-08"), "description": "Follow this account for geo-targeted Software Dev. - General/IT job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 468, "friends_count": 332, "statues_count": 615 }, "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-01-03T10:00:15.000Z"), "id": 683709471982501888, "text": "Note to self: stay humble...it almost game time ����������������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2432635248, "name": "DDOVE2000⁉", "screen_name": "ddove2000", "lang": "en", "location": "null", "create_at": date("2014-04-07"), "description": "null", "followers_count": 217, "friends_count": 108, "statues_count": 2140 }, "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-03T10:00:15.000Z"), "id": 683709472036917250, "text": "So bittersweet, the final week is the season is kicking off. Only 5 weeks until the void fully sets in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 40547150, "name": "Matt Dittemore", "screen_name": "mdittemore", "lang": "en", "location": "ÜT: 33.469815,-111.988598", "create_at": date("2009-05-16"), "description": "Sometimes I'm offended by the thoughts in my own mind....", "followers_count": 86, "friends_count": 145, "statues_count": 4088 }, "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-03T10:00:15.000Z"), "id": 683709472221470720, "text": "My man Timothy Miller with the National Anthem ugh I 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": 421632942, "name": "mckenzie peace ツ", "screen_name": "youngmiggenz", "lang": "en", "location": "Atlanta ✈️ Philly", "create_at": date("2011-11-25"), "description": "Get rich or die tryin |LaSalleSOFT #13 ⚾️|", "followers_count": 1192, "friends_count": 846, "statues_count": 31505 }, "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-03T10:00:15.000Z"), "id": 683709472234024960, "text": "When I tell grant I'm needy his response kills me. \"Bring it on\" ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1187891796, "name": "Stephanie Schaal", "screen_name": "ThatGirlSteph01", "lang": "en", "location": "null", "create_at": date("2013-02-16"), "description": "Captain Morgan 8/06/09-12/16/15", "followers_count": 342, "friends_count": 322, "statues_count": 17087 }, "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-03T10:00:15.000Z"), "id": 683709472280195072, "text": "can go home because it's snowing and freezing rain, he rocks!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 207819875, "name": "Becky Hatfield", "screen_name": "star_fire_wolf", "lang": "en", "location": "Oregon", "create_at": date("2010-10-25"), "description": "Make you're life a statement of love and compassion, and where it isn't, that's where you're work lies.", "followers_count": 112, "friends_count": 84, "statues_count": 53310 }, "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-01-03T10:00:15.000Z"), "id": 683709472313753601, "text": "Its snowing again!��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.1858682,44.3035282"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2846039290, "name": "Kyle Fleming", "screen_name": "SidepocketSid", "lang": "en", "location": "Redmond, OR", "create_at": date("2014-10-26"), "description": "Music & Travel. Snapchat- kyledahuman", "followers_count": 269, "friends_count": 209, "statues_count": 2871 }, "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-03T10:00:15.000Z"), "id": 683709472322109441, "text": "If you ain't been to Vegas you ain't partied nigga", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 65908413, "name": "Bobby", "screen_name": "_BEASLEY_", "lang": "en", "location": "Houston, TX", "create_at": date("2009-08-15"), "description": "Vegas nigga Houston living #entrepreneur", "followers_count": 73, "friends_count": 411, "statues_count": 4202 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-03T10:00:15.000Z"), "id": 683709472393445376, "text": "Can you recommend anyone for this #Nursing #job? https://t.co/VI44z3rduH #Augusta, ME #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-69.7794897,44.3106241"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "Augusta", "Hiring", "CareerArc" }}, "user": { "id": 22918777, "name": "TMJ-ME Nursing Jobs", "screen_name": "tmj_me_nursing", "lang": "en", "location": "Maine", "create_at": date("2009-03-05"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Maine Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 357, "friends_count": 297, "statues_count": 223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Augusta, ME", "id": "625eb8436720d77f", "name": "Augusta", "place_type": "city", "bounding_box": rectangle("-69.832494,44.290196 -69.735635,44.363964") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23011, "countyName": "Kennebec", "cityID": 2302100, "cityName": "Augusta" } }
+{ "create_at": datetime("2016-01-03T10:00:15.000Z"), "id": 683709472426987520, "text": "@peterkarlberg @aClassicLiberal \"The successful Swedish austerity program in the mid-1990s does not, therefore,\"", "in_reply_to_status": 683708870125056006, "in_reply_to_user": 16088282, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16088282, 103375095 }}, "user": { "id": 799437895, "name": "Jordan Rice", "screen_name": "1jrice", "lang": "en", "location": "PDX (now) SEA (6yrs) NJ (from)", "create_at": date("2012-09-02"), "description": "Director Digital Sensing Innovation @Nike | ex-@Quanttus & ex-@Synapse | Wearable Technologist since before it was cool | Views mine | Sports geek: #EBFG #Cubs", "followers_count": 603, "friends_count": 866, "statues_count": 5822 }, "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-03T10:00:15.000Z"), "id": 683709472431276033, "text": "RT @nypmetro “It was the moment I realized my ex was a monster” https://t.co/8zNXzTL3AH #NYC #Bagels @STL_Blonde @ShannonPoe @cocokonskii", "in_reply_to_status": 683706486472716290, "in_reply_to_user": 1158638582, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NYC", "Bagels" }}, "user_mentions": {{ 1158638582, 64816476, 76969053, 394342473 }}, "user": { "id": 27956098, "name": "Donnie", "screen_name": "easygoer132", "lang": "en", "location": "Manhattan, NYC", "create_at": date("2009-03-31"), "description": "1961. Retired contractor from Nassau County, Long Island, NY. Single; 6'3 195 lbs. Origin of my username --- http://youtu.be/n2g0dME0NzE", "followers_count": 2991, "friends_count": 2850, "statues_count": 105984 }, "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-03T10:00:15.000Z"), "id": 683709472490041346, "text": "from the zone ��⚫️ The 4's ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 615784017, "name": "lul dess.❣", "screen_name": "shewannabedesss", "lang": "en", "location": "Zone 4⚫️", "create_at": date("2012-06-22"), "description": "Unbothered like Kylie ,Mindset Like Quan, Feelings Like Rihanna, Attitude Like Kanye. Daylon ♡ o8/23 & 11/17♥", "followers_count": 574, "friends_count": 365, "statues_count": 40556 }, "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-03T10:00:15.000Z"), "id": 683709472594882560, "text": "@Vitamin_Jkh lmfao bye ��", "in_reply_to_status": 683709413534875649, "in_reply_to_user": 519132443, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 519132443 }}, "user": { "id": 470357445, "name": "C✨", "screen_name": "cm__mayes", "lang": "en", "location": "null", "create_at": date("2012-01-21"), "description": "R.I.P Bryant&Jalon ❤️", "followers_count": 1594, "friends_count": 1071, "statues_count": 65833 }, "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-03T10:00:15.000Z"), "id": 683709472603115521, "text": "@BobbyHrbek I wasn't acting, I am a saint Bobby ��", "in_reply_to_status": 682406239037296640, "in_reply_to_user": 382173421, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 382173421 }}, "user": { "id": 261427787, "name": "Brandi Bennett", "screen_name": "BrandiBennettt", "lang": "en", "location": "Glenwood, IA", "create_at": date("2011-03-05"), "description": "@HerbsBrian ❤️", "followers_count": 714, "friends_count": 402, "statues_count": 19753 }, "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-03T10:00:15.000Z"), "id": 683709472624128000, "text": "Morning clouds/afternoon sun this afternoon, high 57 (14 C). Low 38 (3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575312, "name": "Lake Charles Weather", "screen_name": "LakeCharlesWX", "lang": "en", "location": "Lake Charles, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Lake Charles, LA", "followers_count": 540, "friends_count": 92, "statues_count": 7927 }, "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-03T10:00:15.000Z"), "id": 683709472779407360, "text": "I have never dreaded school 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": 2585907266, "name": "kelseyconklin", "screen_name": "kelseyyconklinn", "lang": "en", "location": "null", "create_at": date("2014-06-24"), "description": "null", "followers_count": 326, "friends_count": 362, "statues_count": 681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sloatsburg, NY", "id": "6e804acc5e0f5a5b", "name": "Sloatsburg", "place_type": "city", "bounding_box": rectangle("-74.211939,41.147774 -74.173078,41.177866") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36087, "countyName": "Rockland", "cityID": 3667708, "cityName": "Sloatsburg" } }
+{ "create_at": datetime("2016-01-03T10:00:15.000Z"), "id": 683709472800256000, "text": "@deleukc you're the exception buddy ��", "in_reply_to_status": 683709298908643328, "in_reply_to_user": 2817852870, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2817852870 }}, "user": { "id": 588837537, "name": "Jackson Barlow", "screen_name": "jackson_barlow", "lang": "en", "location": "null", "create_at": date("2012-05-23"), "description": "Snapchat-jacksonbarlow #TAMUC18 ΣΧceptional. Strive for greatness.", "followers_count": 581, "friends_count": 472, "statues_count": 8677 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Trophy Club, TX", "id": "22b244bb794bfd96", "name": "Trophy Club", "place_type": "city", "bounding_box": rectangle("-97.217637,32.984667 -97.15677,33.016258") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4873710, "cityName": "Trophy Club" } }
+{ "create_at": datetime("2016-01-03T10:00:15.000Z"), "id": 683709472804483072, "text": "Church hit home today. ❤ #GodIsGood", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GodIsGood" }}, "user": { "id": 323311645, "name": "Brittany Waldman", "screen_name": "ItsBrittanyyy27", "lang": "en", "location": "Wichita, KS", "create_at": date("2011-06-24"), "description": "Wichita State Grad Student. Future Sociologist. Lover of the outdoors.People watcher.Jesus lover.Sports fanatic.Overly witty humor. Runner of long distances.", "followers_count": 806, "friends_count": 703, "statues_count": 13140 }, "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-03T10:00:15.000Z"), "id": 683709472959627264, "text": "@pumasgorawr ��", "in_reply_to_status": 683709247599677440, "in_reply_to_user": 2920019168, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2920019168 }}, "user": { "id": 3362867609, "name": "H4ppyH4lloween", "screen_name": "h4ppyh4lloween", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2015-07-06"), "description": "TX / 23 / Wife / Dreamer / Ghuleh / Occultism", "followers_count": 48, "friends_count": 28, "statues_count": 1084 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsboro, TX", "id": "18d224d83fd56985", "name": "Hillsboro", "place_type": "city", "bounding_box": rectangle("-97.145021,31.983016 -97.084257,32.032352") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48217, "countyName": "Hill", "cityID": 4834088, "cityName": "Hillsboro" } }
+{ "create_at": datetime("2016-01-03T10:00:15.000Z"), "id": 683709473211322368, "text": "The hangover is so fucken real I think I'm dying", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3335718429, "name": "këłłÿfornia", "screen_name": "bombskieskelly", "lang": "en", "location": "null", "create_at": date("2015-06-19"), "description": "we all come from the sea but we are not all of the sea", "followers_count": 127, "friends_count": 76, "statues_count": 2280 }, "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-03T10:00:15.000Z"), "id": 683709473228062721, "text": "@leahromo45 omg delete this tweet immediately #spoiler", "in_reply_to_status": 683709311764295680, "in_reply_to_user": 1581403452, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "spoiler" }}, "user_mentions": {{ 1581403452 }}, "user": { "id": 421194786, "name": "lauren greenwald", "screen_name": "1aurengreenwald", "lang": "en", "location": "snap - laurengreenwald", "create_at": date("2011-11-25"), "description": "im searching for something that i cant reach", "followers_count": 638, "friends_count": 152, "statues_count": 15934 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kingston, PA", "id": "717a04fbef676cd2", "name": "Kingston", "place_type": "city", "bounding_box": rectangle("-75.906048,41.250501 -75.869375,41.280364") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42079, "countyName": "Luzerne", "cityID": 4239784, "cityName": "Kingston" } }
+{ "create_at": datetime("2016-01-03T10:00:15.000Z"), "id": 683709473265971204, "text": "#jetsnation ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "jetsnation" }}, "user": { "id": 2478954645, "name": "Stephanie", "screen_name": "StephhSavin00", "lang": "en", "location": "Long Island, NY", "create_at": date("2014-04-12"), "description": "Italian| Longwood High School | LVK|", "followers_count": 626, "friends_count": 1174, "statues_count": 4651 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yaphank, NY", "id": "00eed65f54de2860", "name": "Yaphank", "place_type": "city", "bounding_box": rectangle("-72.974376,40.796768 -72.886039,40.86706") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3683426, "cityName": "Yaphank" } }
+{ "create_at": datetime("2016-01-03T10:00:15.000Z"), "id": 683709473643311105, "text": "Rise and grind lets get it boys #SteelerNation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SteelerNation" }}, "user": { "id": 515056214, "name": "michael lohr", "screen_name": "zackwalker28", "lang": "en", "location": "Chino, CA", "create_at": date("2012-03-04"), "description": "just trying to reach my dreams, aaron is the bro for life.Sarah montoya♥️. #STEELERNATION", "followers_count": 478, "friends_count": 389, "statues_count": 9493 }, "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-03T10:00:15.000Z"), "id": 683709473827848192, "text": "7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4354918516, "name": "Devon", "screen_name": "VonOutThe4", "lang": "en", "location": "704", "create_at": date("2015-12-02"), "description": "Clt ♠️ No mercy #WSSU", "followers_count": 42, "friends_count": 43, "statues_count": 204 }, "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-03T10:00:15.000Z"), "id": 683709473886736384, "text": "1,4,9 https://t.co/iH6eKhJOUl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 226791156, "name": "christian", "screen_name": "WhoChristian", "lang": "en", "location": "null", "create_at": date("2010-12-14"), "description": "Add a bio to your profile", "followers_count": 1496, "friends_count": 354, "statues_count": 104477 }, "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-03T10:00:15.000Z"), "id": 683709474025127937, "text": "My first snow day ��❄️ #californiagirlatheart #oregonexplored @ Beaverton, Oregon https://t.co/VEA6jaPmfv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.81,45.4797"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "californiagirlatheart", "oregonexplored" }}, "user": { "id": 251184998, "name": "Cha Cha", "screen_name": "chelseahulbert", "lang": "en", "location": "Portland, OR", "create_at": date("2011-02-12"), "description": "twenty years of awkward conversations and unintentionally offending people. I make videos on the YouTubes.", "followers_count": 126, "friends_count": 54, "statues_count": 10062 }, "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-03T10:00:15.000Z"), "id": 683709474113077251, "text": "@Fenwaygirl15 Yep. And when you see Julio, ODB, Cam or Peterson go off today, take comfort in that you're not facing them.", "in_reply_to_status": 683708926643277824, "in_reply_to_user": 33067157, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 33067157 }}, "user": { "id": 548944250, "name": "Brian Murphy", "screen_name": "Spokes_Murphy", "lang": "en", "location": "Long Beach, California", "create_at": date("2012-04-08"), "description": "I like sports and I don't care who knows. Writer/copy editor guy. Yankees, Lakers, NY Giants, UCF Knights fan. I walk on wheels.", "followers_count": 1059, "friends_count": 823, "statues_count": 25398 }, "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-03T10:00:16.000Z"), "id": 683709474494877696, "text": "Hiding my alarm clock so i \"accidently\" over sleep tomorrow.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70040213, "name": "lindsey bruce", "screen_name": "lmbruce", "lang": "en", "location": "Fountain Square", "create_at": date("2009-08-29"), "description": "I enjoy making my bed.", "followers_count": 190, "friends_count": 351, "statues_count": 7635 }, "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-01-03T10:00:16.000Z"), "id": 683709474628972544, "text": "https://t.co/1xj5n0d5vk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 281822817, "name": "That Valley Life", "screen_name": "Showroom_77", "lang": "en", "location": "West Hills, CA", "create_at": date("2011-04-13"), "description": "strangers til we meet...", "followers_count": 14738, "friends_count": 4484, "statues_count": 35417 }, "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-03T10:00:16.000Z"), "id": 683709474767548416, "text": "Is posibility detecte lewer fam\nAnd Pastor.frind.curt ho said truh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2844545920, "name": "Gladys Enid Cintron", "screen_name": "EnidGladys", "lang": "en", "location": "null", "create_at": date("2014-10-25"), "description": "null", "followers_count": 43, "friends_count": 284, "statues_count": 446 }, "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-01-03T10:00:16.000Z"), "id": 683709474805313541, "text": "holy hell I just passed 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": 303994331, "name": "Noelle", "screen_name": "NoelleCatz", "lang": "en", "location": "Cleveland, Ohio ", "create_at": date("2011-05-23"), "description": "null", "followers_count": 810, "friends_count": 314, "statues_count": 54906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chesterland, OH", "id": "ece324ede3de342a", "name": "Chesterland", "place_type": "city", "bounding_box": rectangle("-81.390802,41.484452 -81.297148,41.546086") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39055, "countyName": "Geauga", "cityID": 3914100, "cityName": "Chesterland" } }
+{ "create_at": datetime("2016-01-03T10:00:16.000Z"), "id": 683709474843062272, "text": "I swear, all of Kanyes clothes look like something a homeless man would wear ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2337470029, "name": "Connor", "screen_name": "_cxnnor", "lang": "en", "location": "Newport News, VA", "create_at": date("2014-02-10"), "description": "Senior // sc: conswag// Future Police Officer // #Thinblueline // Michaela Lynne❤️", "followers_count": 462, "friends_count": 661, "statues_count": 10200 }, "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-03T10:00:16.000Z"), "id": 683709474943725568, "text": "Ima thief in 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": 2279827566, "name": "garnet☄", "screen_name": "princesslexz", "lang": "en", "location": "my space, my area ", "create_at": date("2014-01-06"), "description": "drake's proud daughter", "followers_count": 371, "friends_count": 227, "statues_count": 19775 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pooler, GA", "id": "bc1dbfe09834a716", "name": "Pooler", "place_type": "city", "bounding_box": rectangle("-81.338145,32.007509 -81.197739,32.191443") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13051, "countyName": "Chatham", "cityID": 1362104, "cityName": "Pooler" } }
+{ "create_at": datetime("2016-01-03T10:00:16.000Z"), "id": 683709474998235136, "text": "@williamclugo I need gang green to lose so the steelers can get in ��.", "in_reply_to_status": 683705556998209536, "in_reply_to_user": 632177453, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 632177453 }}, "user": { "id": 380059040, "name": "Fred Rowe", "screen_name": "fred_rowe", "lang": "en", "location": "null", "create_at": date("2011-09-25"), "description": "null", "followers_count": 430, "friends_count": 328, "statues_count": 4667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monroe, NY", "id": "a3163172df0d66a7", "name": "Monroe", "place_type": "city", "bounding_box": rectangle("-74.221454,41.288729 -74.14913,41.347895") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36071, "countyName": "Orange", "cityID": 3647988, "cityName": "Monroe" } }
+{ "create_at": datetime("2016-01-03T10:00:16.000Z"), "id": 683709475056779264, "text": "@emzanotti And there are many Libs who yearn for societal breakdown. Which this could portend. They forget they'll be 1st against the wall.", "in_reply_to_status": 683706537479548928, "in_reply_to_user": 5447242, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 5447242 }}, "user": { "id": 2322589506, "name": "Sharkman", "screen_name": "Sharkman1963", "lang": "en", "location": "Kirkland, Washington", "create_at": date("2014-02-01"), "description": "RE Broker, Ex-Litigator, Proud Dad, Sports Enthusiast, Catholic, Conservative, Avid Reader & Cook, Ex-Navy, Addicted to Sushi & Politics, Entrepreneur, Dreamer", "followers_count": 487, "friends_count": 884, "statues_count": 1389 }, "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-03T10:00:16.000Z"), "id": 683709475077791744, "text": "happy birthday allie�� miss seeing ur cute self. I hope your day is as fab as you ������������������������������������������ @allie_williamss", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 188140162 }}, "user": { "id": 1674225529, "name": "j♡", "screen_name": "Jayybear24", "lang": "en", "location": "Ogden, UT", "create_at": date("2013-08-15"), "description": "insta: jayybear24 || snapchat: smd_101 || ohs ||", "followers_count": 470, "friends_count": 307, "statues_count": 7174 }, "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-01-03T10:00:16.000Z"), "id": 683709475144925184, "text": "Morning clouds/afternoon sun this afternoon, high 56 (13 C). Low 38 (3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575370, "name": "Lafayette Weather", "screen_name": "LafayetteWX", "lang": "en", "location": "Lafayette, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Lafayette, LA", "followers_count": 471, "friends_count": 92, "statues_count": 7966 }, "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-03T10:00:16.000Z"), "id": 683709475237187584, "text": "@MeiMeiFox thanks for the read", "in_reply_to_status": 682381216901902337, "in_reply_to_user": 15816231, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15816231 }}, "user": { "id": 14606435, "name": "annbadillo", "screen_name": "annbadillo", "lang": "en", "location": "Palo Alto, CA", "create_at": date("2008-04-30"), "description": "Ecosystems | Narrative | Venture Creation | 21st Century | Leadership", "followers_count": 771, "friends_count": 728, "statues_count": 5103 }, "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-01-03T10:00:16.000Z"), "id": 683709475275075588, "text": "#FlyEaglesFly?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "FlyEaglesFly" }}, "user": { "id": 1508048328, "name": "Sean Hughes", "screen_name": "SHughes298", "lang": "en", "location": "Maryland, USA", "create_at": date("2013-06-11"), "description": "I love the Philadelphia Eagles more than some people ● NWHS Junior", "followers_count": 136, "friends_count": 512, "statues_count": 6030 }, "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-03T10:00:16.000Z"), "id": 683709475342036992, "text": "@rui_xu between that, penn state lb, and the Oregon stuff it's been a banner week", "in_reply_to_status": 683709146177208320, "in_reply_to_user": 61799665, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 61799665 }}, "user": { "id": 55305443, "name": "Jeremiah Oshan", "screen_name": "JeremiahOshan", "lang": "en", "location": "Seattle", "create_at": date("2009-07-09"), "description": "SB Nation soccer editor and blog manager; http://SounderAtHeart.com editor; host with that 'Cascadia snarl' on Nos Audietis.", "followers_count": 6417, "friends_count": 776, "statues_count": 77560 }, "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-03T10:00:16.000Z"), "id": 683709475455238144, "text": "I finally figured out who they is", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2528331662, "name": "DECEMBER 23", "screen_name": "YoungWaterHorse", "lang": "en", "location": "null", "create_at": date("2014-05-27"), "description": "TWHS Track and Field '17", "followers_count": 297, "friends_count": 171, "statues_count": 1131 }, "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-03T10:00:16.000Z"), "id": 683709475639955456, "text": "Finally here ������ https://t.co/wnQg4o5CMI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.536021,28.336039"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 63321072, "name": "Josselyne Martinez", "screen_name": "JOSSYBELLA211", "lang": "en", "location": "null", "create_at": date("2009-08-05"), "description": "Respect all, fear none.", "followers_count": 125, "friends_count": 105, "statues_count": 1165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Radisson Resort Orlando - Celebration", "id": "07d9db4d7b481001", "name": "Radisson Resort Orlando - Celebration", "place_type": "poi", "bounding_box": rectangle("-81.5360211,28.3360389 -81.536021,28.336039") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12097, "countyName": "Osceola" } }
+{ "create_at": datetime("2016-01-03T10:00:16.000Z"), "id": 683709475790962689, "text": "I'm at St. Peters Catholic Church in Roswell, NM https://t.co/EvoSjgqLHZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.52245104,33.38560578"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 179658694, "name": "Rosy Rubio", "screen_name": "challorubio", "lang": "en", "location": "Roswell, NM", "create_at": date("2010-08-17"), "description": "null", "followers_count": 75, "friends_count": 137, "statues_count": 4892 }, "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-01-03T10:00:16.000Z"), "id": 683709475946151936, "text": "I'm at E Northfield Rd N S Livingston Ave in Livingston, NJ https://t.co/3Nl9hdmKKM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.32450771,40.77394944"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 62647031, "name": "Arlene Silikovitz", "screen_name": "choralcat", "lang": "en", "location": "null", "create_at": date("2009-08-03"), "description": "null", "followers_count": 520, "friends_count": 2075, "statues_count": 16753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Livingston, NJ", "id": "24661d47b426cdac", "name": "Livingston", "place_type": "city", "bounding_box": rectangle("-74.377326,40.75774 -74.280592,40.818383") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex" } }
+{ "create_at": datetime("2016-01-03T10:00:16.000Z"), "id": 683709476076163072, "text": "Ended the break the best way", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 556367613, "name": "wayne", "screen_name": "Crispy_wayne", "lang": "en", "location": "Detroit,Mi", "create_at": date("2012-04-17"), "description": "staying out the way mention me for a fb FMOI ___itswayne detroit eastside 4⃣8⃣2⃣2⃣4⃣ #forever.1hunnit| 6-20-15 ✊", "followers_count": 1175, "friends_count": 1382, "statues_count": 20438 }, "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-03T10:00:16.000Z"), "id": 683709476222943233, "text": "happy bday soph�� thanks for being such a great friend & my longest streak on snap�� love u forever and ever❤️❤️ https://t.co/LGoFDR85Qt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2332613670, "name": "taylor", "screen_name": "tmartelxo", "lang": "en", "location": "null", "create_at": date("2014-02-07"), "description": "insta: tmartelxo snap: cheerchic412", "followers_count": 241, "friends_count": 336, "statues_count": 593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scarborough, ME", "id": "f39a803949cd2255", "name": "Scarborough", "place_type": "city", "bounding_box": rectangle("-70.401214,43.542566 -70.270921,43.636249") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23005, "countyName": "Cumberland", "cityID": 2366110, "cityName": "Scarborough" } }
+{ "create_at": datetime("2016-01-03T10:00:16.000Z"), "id": 683709476428353537, "text": "@lbozoo lo", "in_reply_to_status": 683709197897216001, "in_reply_to_user": 62374282, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 62374282 }}, "user": { "id": 1125348871, "name": "CJ", "screen_name": "JustSayBi", "lang": "en", "location": "Miami", "create_at": date("2013-01-27"), "description": "I'm a married guy trying to have fun in crazy Miami looking to connect with other cool people and couples see whats out there in Miami", "followers_count": 653, "friends_count": 1432, "statues_count": 4180 }, "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.446969,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-01-03T10:00:16.000Z"), "id": 683709476516401152, "text": "Sister appreciation tweet ������ @ Melika https://t.co/ZRGIqfpjlS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 881100314, "name": "Shayan Ravanassa", "screen_name": "Shayan_Rav", "lang": "en", "location": "Frisco, Texas", "create_at": date("2012-10-14"), "description": "Senior at Frisco High School, Persian/Azeri, Music/Mixing, Piano, Future in medicine.", "followers_count": 316, "friends_count": 348, "statues_count": 3080 }, "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-01-03T10:00:16.000Z"), "id": 683709476931768322, "text": "Why the Cowboys so turned up? They must not know their not going to the playoffs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70465399, "name": "Steelers please win.", "screen_name": "AlmightyLaker", "lang": "en", "location": "Cashville to Memphis", "create_at": date("2009-08-31"), "description": "#LakersSteelersAlabamaBBNHivé |Tennessee State University| #GRiNDHARD |", "followers_count": 2145, "friends_count": 1267, "statues_count": 48399 }, "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-03T10:00:16.000Z"), "id": 683709477007175680, "text": "@_YungJosh yeah June", "in_reply_to_status": 683709278558011399, "in_reply_to_user": 292911379, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 292911379 }}, "user": { "id": 46305981, "name": "Jack of DemBoy$", "screen_name": "ImDatNigga_Jack", "lang": "en", "location": "PVAMU", "create_at": date("2009-06-10"), "description": "I Don't Tweet No Lies Dem Boy$ IG:imdatnigga_jack #PVNation #HighLifePromos #LongLiveChopo Free @JoeSwaggerRight #LongLiveClyde", "followers_count": 5530, "friends_count": 3927, "statues_count": 318334 }, "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-03T10:00:16.000Z"), "id": 683709477149872128, "text": "@jclinger_3 ����", "in_reply_to_status": 683702927354114050, "in_reply_to_user": 983870508, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 983870508 }}, "user": { "id": 462508581, "name": "Marisa", "screen_name": "Marisa_Bowen", "lang": "en", "location": "null", "create_at": date("2012-01-12"), "description": "null", "followers_count": 985, "friends_count": 285, "statues_count": 11523 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake City, PA", "id": "11abf7b76966998e", "name": "Lake City", "place_type": "city", "bounding_box": rectangle("-80.367202,42.006854 -80.31957,42.034855") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42049, "countyName": "Erie", "cityID": 4240960, "cityName": "Lake City" } }
+{ "create_at": datetime("2016-01-03T10:00:16.000Z"), "id": 683709477149929472, "text": "S/o to Maryann for making the best chicken noodle soup ever", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 230080253, "name": "em", "screen_name": "emfosterxo", "lang": "en", "location": "Arizona State University", "create_at": date("2010-12-23"), "description": "ASU Nursing . http://instagram.com/emxfoster", "followers_count": 394, "friends_count": 198, "statues_count": 7606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Poplar Grove, IL", "id": "011eea600255edfc", "name": "Poplar Grove", "place_type": "city", "bounding_box": rectangle("-88.854012,42.354927 -88.813082,42.371629") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17007, "countyName": "Boone", "cityID": 1761145, "cityName": "Poplar Grove" } }
+{ "create_at": datetime("2016-01-03T10:00:16.000Z"), "id": 683709477502226432, "text": "@Mi_Ra208 last week some guy high fived me screamed go jets and tried to take a snapchat with me ��", "in_reply_to_status": 683707068004614145, "in_reply_to_user": 178110125, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 178110125 }}, "user": { "id": 295908825, "name": "corri ♬", "screen_name": "h4rdcorr", "lang": "en", "location": "null", "create_at": date("2011-05-09"), "description": "il faut souffrir pour être belle.", "followers_count": 354, "friends_count": 340, "statues_count": 18114 }, "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-03T10:00:16.000Z"), "id": 683709477510463488, "text": "Who's got Kov white 3? And also A lunch.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 741083948, "name": "Jarrett", "screen_name": "jarretttaft", "lang": "en", "location": "null", "create_at": date("2012-08-06"), "description": "Marissa is boo.", "followers_count": 402, "friends_count": 726, "statues_count": 1357 }, "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-01-03T10:00:16.000Z"), "id": 683709477812453376, "text": "Sunny this afternoon, high 51 (11 C). Low 27 (-3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575409, "name": "Fort Smith Weather", "screen_name": "FortSmithWX", "lang": "en", "location": "Fort Smith, AR", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Fort Smith, AR", "followers_count": 442, "friends_count": 92, "statues_count": 8015 }, "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-03T10:00:16.000Z"), "id": 683709477825073152, "text": "�� https://t.co/jLrWojSpbt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2909299097, "name": "Beans", "screen_name": "Texas__Hammer", "lang": "en", "location": "Magnolia, TX", "create_at": date("2014-12-07"), "description": "#Texans #Astros", "followers_count": 105, "friends_count": 224, "statues_count": 3682 }, "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-03T10:00:16.000Z"), "id": 683709477900533760, "text": "We're #hiring! Read about our latest #job opening here: RN-Med Surg - https://t.co/I8tzvhn9c5 #RN #Westfield, MA #Nursing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.7592109,42.11807"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "RN", "Westfield", "Nursing" }}, "user": { "id": 2688167298, "name": "Noble Hospital Jobs", "screen_name": "JobsatNobleHosp", "lang": "en", "location": "Westfield, MA", "create_at": date("2014-07-28"), "description": "Noble Hospital is a regional community hospital, providing a broad range of expert services to Greater Westfield and surrounding areas.", "followers_count": 17, "friends_count": 0, "statues_count": 94 }, "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-01-03T10:00:16.000Z"), "id": 683709477917425664, "text": "���������� can't get enough", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1583579113, "name": "Angel Face", "screen_name": "AngelFBarrett", "lang": "en", "location": "null", "create_at": date("2013-07-10"), "description": "null", "followers_count": 0, "friends_count": 2, "statues_count": 130955 }, "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-03T10:00:16.000Z"), "id": 683709478097649664, "text": "its crazy how I guy can have a girl that will give him the world and then just drop her for some easy hoe ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2945673403, "name": "paige hicks", "screen_name": "paigehicks0", "lang": "en", "location": "null", "create_at": date("2014-12-27"), "description": "probably with dara or shelbs / OC junior", "followers_count": 796, "friends_count": 463, "statues_count": 13181 }, "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-03T10:00:16.000Z"), "id": 683709478261362688, "text": "@BruhhhComedy @trapicaly @SabrinaQuirola @hilaaarry @PamelaDuBarry", "in_reply_to_status": 683532620295045123, "in_reply_to_user": 3287832937, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3287832937, 2299370352, 1416634022, 527953215, 276626019 }}, "user": { "id": 150033875, "name": "Casey Davies", "screen_name": "casey__davies", "lang": "en", "location": "Jersey City, NJ", "create_at": date("2010-05-30"), "description": "instagram- casey__davies . Njcu", "followers_count": 356, "friends_count": 331, "statues_count": 3660 }, "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-03T10:00:17.000Z"), "id": 683709478554845184, "text": "Want to work in #Dallas, TX? View our latest opening: https://t.co/M2XrSehCZp #Healthcare #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.769923,32.802955"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Dallas", "Healthcare", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 20831777, "name": "TMJ- DFW Health Jobs", "screen_name": "tmj_dfw_health", "lang": "en", "location": "DFW, Texas", "create_at": date("2009-02-13"), "description": "Follow this account for geo-targeted Healthcare job tweets in DFW, Texas. Need help? Tweet us at @CareerArc!", "followers_count": 501, "friends_count": 306, "statues_count": 1493 }, "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-03T10:00:17.000Z"), "id": 683709478705864704, "text": "@yuckfea stop playing with my brain?????", "in_reply_to_status": 683556539605491712, "in_reply_to_user": 2270999028, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2270999028 }}, "user": { "id": 1891100340, "name": "❄ winterry ❄", "screen_name": "terryngyn", "lang": "en", "location": "Garden Grove, CA", "create_at": date("2013-09-21"), "description": "she carried the universe on her back and made it look like wings", "followers_count": 228, "friends_count": 231, "statues_count": 2068 }, "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-03T10:00:17.000Z"), "id": 683709478848495616, "text": "@nene552nene rn", "in_reply_to_status": 683709159150358528, "in_reply_to_user": 793031221, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 793031221 }}, "user": { "id": 381054167, "name": "Kevin", "screen_name": "kgalarza7", "lang": "en", "location": "null", "create_at": date("2011-09-27"), "description": "Waukegan Sc: kevinsito7#", "followers_count": 1009, "friends_count": 887, "statues_count": 38190 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waukegan, IL", "id": "b819c5d90b780b57", "name": "Waukegan", "place_type": "city", "bounding_box": rectangle("-87.96368,42.305624 -87.802772,42.431936") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1779293, "cityName": "Waukegan" } }
+{ "create_at": datetime("2016-01-03T10:00:17.000Z"), "id": 683709478873726977, "text": "@MilitaryHC Thank you!", "in_reply_to_status": 683708629132939264, "in_reply_to_user": 1027468086, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1027468086 }}, "user": { "id": 3068138332, "name": "Beth Metzker", "screen_name": "BethMetzker", "lang": "en", "location": "null", "create_at": date("2015-03-03"), "description": "null", "followers_count": 1091, "friends_count": 1588, "statues_count": 19323 }, "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-01-03T10:00:17.000Z"), "id": 683709479347728384, "text": "We're #winning! Vegetables Likely To Take More Of Your Plate in 2016 #TheWildDiet #MyDietIsBetterThanYours https://t.co/kmiG5Cl1Dc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.442976,33.64159"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "winning", "TheWildDiet", "MyDietIsBetterThanYours" }}, "user": { "id": 376283958, "name": "Abel James", "screen_name": "fatburnman", "lang": "en", "location": "Wilder, TN", "create_at": date("2011-09-19"), "description": "Bestselling author of #TheWildDiet, #biohacker, #Paleo / slow food dude, #entrepreneur, #musician, & health crusader. Starring on ABC's #MyDietIsBetterThanYours", "followers_count": 21016, "friends_count": 751, "statues_count": 8685 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hartsfield-Jackson Atlanta International Airport (ATL)", "id": "07d9c93838c83001", "name": "Hartsfield-Jackson Atlanta International Airport (ATL)", "place_type": "poi", "bounding_box": rectangle("-84.4429761,33.6415899 -84.442976,33.64159") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1317776, "cityName": "College Park" } }
+{ "create_at": datetime("2016-01-03T10:00:17.000Z"), "id": 683709479368568832, "text": "@ronalan908 @BenjaminLives @TheSisko @FSM1988 it takes less faith for me to believe God created than that this thing just happened", "in_reply_to_status": 683708837581434880, "in_reply_to_user": 2906952316, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2906952316, 276493424, 24394452, 4075759937 }}, "user": { "id": 423942749, "name": "David Eliff", "screen_name": "EliffDavid", "lang": "en", "location": "null", "create_at": date("2011-11-28"), "description": "null", "followers_count": 280, "friends_count": 1699, "statues_count": 4514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lexington, TN", "id": "4c8ec4dd375b72c2", "name": "Lexington", "place_type": "city", "bounding_box": rectangle("-88.42873,35.611389 -88.364752,35.713527") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47077, "countyName": "Henderson", "cityID": 4741980, "cityName": "Lexington" } }
+{ "create_at": datetime("2016-01-03T10:00:17.000Z"), "id": 683709479448227844, "text": "Support your brothers and sisters for their success. Always bring them up. ~@minishmael #NOISundays #BuyBlack #BlackDollarsMatter", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NOISundays", "BuyBlack", "BlackDollarsMatter" }}, "user_mentions": {{ 1465946712 }}, "user": { "id": 1660735734, "name": "Maajid A. Muhammad", "screen_name": "NextGenFOI", "lang": "en", "location": "Austin, TX", "create_at": date("2013-08-10"), "description": "I follow Farrakhan and am striving to be RIGHT. @MosqueFlow Team #Artist #Reader #FarrakhanTwitterArmy #MosqueFlow | Follow @LouisFarrakhan | #JusticeOrElse", "followers_count": 606, "friends_count": 790, "statues_count": 3066 }, "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-03T10:00:17.000Z"), "id": 683709479469367296, "text": "Church this morning was great ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 366823532, "name": "Jasmine", "screen_name": "JFGraves_", "lang": "en", "location": "Hattiesburg|Starkville ", "create_at": date("2011-09-02"), "description": "20. Sophmore. Mississippi State University.", "followers_count": 1091, "friends_count": 586, "statues_count": 37009 }, "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-03T10:00:17.000Z"), "id": 683709479477592066, "text": "Stop fwm if you ain't really fwm ����", "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": "✨1.18✨", "screen_name": "_brianna365", "lang": "en", "location": "null", "create_at": date("2014-10-21"), "description": "new twitter follow me", "followers_count": 1327, "friends_count": 962, "statues_count": 23596 }, "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-03T10:00:17.000Z"), "id": 683709479502753793, "text": "Lion Heart \n#TeamSNSD https://t.co/FzrhUqPwVH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TeamSNSD" }}, "user": { "id": 2193588649, "name": "♎ImSorryILikeKpop", "screen_name": "Shayla_Bear15", "lang": "en", "location": "my room", "create_at": date("2013-11-13"), "description": "I have an unhealthy obsession with Yura of Girls Day, ASMR completes me", "followers_count": 275, "friends_count": 270, "statues_count": 7255 }, "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-03T10:00:17.000Z"), "id": 683709479548878850, "text": "������ https://t.co/gZaZNPgoGj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1388586648, "name": "Maxwell", "screen_name": "MaxRichins", "lang": "en", "location": "null", "create_at": date("2013-04-28"), "description": "I have no idea what I'm doing but I know I'm doing it really, really well", "followers_count": 448, "friends_count": 305, "statues_count": 1579 }, "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-03T10:00:17.000Z"), "id": 683709479574044672, "text": "#waytoomanybirds #pismobeach @ Pismo Beach https://t.co/WjEMFTjkTP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-120.64259044,35.13651935"), "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "waytoomanybirds", "pismobeach" }}, "user": { "id": 27594643, "name": "Lauren Quaid", "screen_name": "SL8Rgirl", "lang": "en", "location": "California", "create_at": date("2009-03-29"), "description": "I work to pay my bills, and write and take photographs to feed my soul.", "followers_count": 126, "friends_count": 354, "statues_count": 3155 }, "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-01-03T10:00:17.000Z"), "id": 683709479595147264, "text": "Just lose and guarantee us the second pick in the draft", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 330729737, "name": "Zach Sand", "screen_name": "Mr_SandMan__", "lang": "en", "location": "Kent State University ", "create_at": date("2011-07-06"), "description": "null", "followers_count": 660, "friends_count": 407, "statues_count": 10768 }, "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.314107 -81.876105,41.39158") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3958422, "cityName": "Olmsted Falls" } }
+{ "create_at": datetime("2016-01-03T10:00:17.000Z"), "id": 683709479662317568, "text": "I won't hold back ...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 98103892, "name": "Queen", "screen_name": "_TeeSooBougie", "lang": "en", "location": "Mitten", "create_at": date("2009-12-20"), "description": "#DVO ♐ .", "followers_count": 857, "friends_count": 895, "statues_count": 42417 }, "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-03T10:00:17.000Z"), "id": 683709479905554433, "text": "Flipping the script on me thats your forte", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 187251982, "name": "Monet Park", "screen_name": "koreadoll", "lang": "en", "location": "null", "create_at": date("2010-09-05"), "description": "you know when your mom used to say she just wants the best for you ? yeah she was talking about me", "followers_count": 167, "friends_count": 343, "statues_count": 12595 }, "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-01-03T10:00:17.000Z"), "id": 683709480035463168, "text": "Trying to figure out if I spent more money at the nightclub or McDonald's last night, also wondering which is worse.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54764475, "name": "King Kaiju ゴジラ", "screen_name": "xGOJIRAx", "lang": "en", "location": "Los Angeles", "create_at": date("2009-07-07"), "description": "DJ with Samurai lineage • Creative Director of @ShiftyRhythms • Head of Kaiju MGMT • @TeamBandL All-Star", "followers_count": 524, "friends_count": 338, "statues_count": 1616 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-03T10:00:17.000Z"), "id": 683709480098500608, "text": "@NeauxlaDre Just thinking about it makes me break out into hives", "in_reply_to_status": 683709269003337728, "in_reply_to_user": 136052573, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 136052573 }}, "user": { "id": 1609518120, "name": "A Fool to Cry", "screen_name": "afooltocry", "lang": "en", "location": "null", "create_at": date("2013-07-20"), "description": "Loving vampires, horror movies, Halloween, sexy people, sarcasm, tv, and my New Orleans Saints. Certified Law & Order SVU Addict. #NOH8", "followers_count": 6052, "friends_count": 4223, "statues_count": 44456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Decatur, GA", "id": "ae8761907bd9c420", "name": "Decatur", "place_type": "city", "bounding_box": rectangle("-84.315731,33.751009 -84.277985,33.793376") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1322052, "cityName": "Decatur" } }
+{ "create_at": datetime("2016-01-03T10:00:17.000Z"), "id": 683709480236892160, "text": "@AlexJMalley @MoChroi1965 @StudyingLiberty \n\nCheck your privilege.", "in_reply_to_status": 683708550917582849, "in_reply_to_user": 52316885, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 52316885, 589283044, 2483010143 }}, "user": { "id": 825518317, "name": "Brian Nichols", "screen_name": "BNicholsLiberty", "lang": "en", "location": "null", "create_at": date("2012-09-15"), "description": "Fundraising, Political Consulting, Political Strategy | B.A. in Political Science & A.S. in Business Administration | Libertarian-Republican", "followers_count": 1009, "friends_count": 1554, "statues_count": 9847 }, "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-03T10:00:17.000Z"), "id": 683709481302257664, "text": "When you talk to somebody younger because you get bored but the lil nigga start feeling himself �� https://t.co/kA5JWEmT8E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 233304656, "name": "CHANEL N•3", "screen_name": "MUVAAVE", "lang": "en", "location": "Louisiana, USA", "create_at": date("2011-01-02"), "description": "passionate black girl✨", "followers_count": 989, "friends_count": 538, "statues_count": 39567 }, "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-03T10:00:17.000Z"), "id": 683709481314680832, "text": "I both want to get everything on my do list done and also lay in bed/get breakfast taco.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24610986, "name": "Diana Ascarrunz", "screen_name": "dianascarrunz", "lang": "en", "location": "Austin, TX", "create_at": date("2009-03-15"), "description": "| Austin Photographer | Loves coffee, Freaks & Geeks, and korean food http://Instagram.com/dianascarrunz", "followers_count": 299, "friends_count": 854, "statues_count": 2095 }, "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-03T10:00:17.000Z"), "id": 683709481675456512, "text": "Want to work in #SpiritLake, IA? View our latest opening: https://t.co/Gk7bvD1DmO #Manufacturing #Job #Jobs #Hiring https://t.co/IeyaRzEiqA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.111409,43.422766"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SpiritLake", "Manufacturing", "Job", "Jobs", "Hiring" }}, "user": { "id": 159220275, "name": "Doherty Jobs", "screen_name": "DohertyJobs", "lang": "en", "location": "Minnesota", "create_at": date("2010-06-24"), "description": "Doherty is your source for temporary employment in Minnesota, Iowa, and Wisconsin. Check here for our latest openings!", "followers_count": 394, "friends_count": 487, "statues_count": 2777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spirit Lake, IA", "id": "68fcc799aefa94c0", "name": "Spirit Lake", "place_type": "city", "bounding_box": rectangle("-95.155131,43.392918 -95.063912,43.430821") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19059, "countyName": "Dickinson", "cityID": 1974415, "cityName": "Spirit Lake" } }
+{ "create_at": datetime("2016-01-03T10:00:17.000Z"), "id": 683709481826578432, "text": "Sabes bien que no puedes olvidar ni mucho menos engañarme...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2377599160, "name": "Nayeli ARCHIVOKO_NY", "screen_name": "nayeli_nava98", "lang": "en", "location": "Kevin Ortiz Medina Landia", "create_at": date("2014-03-02"), "description": "ARCHIVOKO_NY Conosi a @kevinortiznet el 22/6/2014 y el 24/5/ 2015 100% Mexicana", "followers_count": 241, "friends_count": 252, "statues_count": 11007 }, "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-03T10:00:17.000Z"), "id": 683709481851695104, "text": "Finally using birthday gift card from back in August! (at @Outback Steakhouse in Kennesaw, GA w/ @elainebow) https://t.co/IGgm0jINcr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.574988,34.007787"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20722851, 54227996 }}, "user": { "id": 16382824, "name": "Dwayne Kilbourne", "screen_name": "dwaynekilbourne", "lang": "en", "location": "Kennesaw, GA", "create_at": date("2008-09-20"), "description": "Author | @Coaster_Addict founder | Fitness Fanatic | 1/2 of @4sqLoveStory | Future World Record Holder | Roller Coaster Marathoner | USMCR | Web Dev & Mrkting", "followers_count": 3147, "friends_count": 2837, "statues_count": 71310 }, "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-01-03T10:00:17.000Z"), "id": 683709481855746049, "text": "\"I will love you until the day my heart stops beating and I drop dead.\" -AW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2307426200, "name": "Megan", "screen_name": "smoshymegan", "lang": "en", "location": "Tallahassee, FL", "create_at": date("2014-01-23"), "description": "Florida State Biology", "followers_count": 257, "friends_count": 208, "statues_count": 2959 }, "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-01-03T10:00:17.000Z"), "id": 683709481864290308, "text": "Beastly Verse by #jooheeyoon is a beast of beauty. @enchantedlion @ Enchanted Lion Books https://t.co/CsiZG1ORNY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0129318,40.6769714"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "jooheeyoon" }}, "user_mentions": {{ 2521035690 }}, "user": { "id": 17560799, "name": "Jason Sturgill", "screen_name": "Sturgill", "lang": "en", "location": "iPhone: 45.496758,-122.634804", "create_at": date("2008-11-22"), "description": "Illustrator • Portland, Oregon", "followers_count": 1515, "friends_count": 1018, "statues_count": 4413 }, "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-03T10:00:17.000Z"), "id": 683709481973317633, "text": "Why is it unacceptable to wear sweatpants to work? ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 319324394, "name": "Melissa Butler", "screen_name": "m_butler13", "lang": "en", "location": "Maryland", "create_at": date("2011-06-17"), "description": "null", "followers_count": 290, "friends_count": 418, "statues_count": 15862 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bel Air North, MD", "id": "a586c0a0034ff5a4", "name": "Bel Air North", "place_type": "city", "bounding_box": rectangle("-76.434737,39.515467 -76.289209,39.599101") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24025, "countyName": "Harford", "cityID": 2405825, "cityName": "Bel Air North" } }
+{ "create_at": datetime("2016-01-03T10:00:17.000Z"), "id": 683709482099195904, "text": "Passing by the #coastline of #SanJuan with my baby #KoopaTroopa. #happyness @ Ventana del Mar https://t.co/XOrfKPHpyq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-66.07490592,18.45787592"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "coastline", "SanJuan", "KoopaTroopa", "happyness" }}, "user": { "id": 445340475, "name": "Laobrademarte", "screen_name": "Laobrademarte", "lang": "en", "location": "Puerto Rico", "create_at": date("2011-12-24"), "description": "BANDA POPPUNK DE PUERTO RICO CANTADO EN ESPAÑOL CON INFLUENCIAS DE BANDAS COMO: THE KILLERS, FALL OUT BOY, PXNDX, INSITE, FINDE, MANÁ, LA SECTA, ENTRE OTROS", "followers_count": 1645, "friends_count": 1888, "statues_count": 5076 }, "place": { "country": "Puerto Rico", "country_code": "Puerto Rico", "full_name": "San Juan, Puerto Rico", "id": "c6b95e2be3678862", "name": "San Juan", "place_type": "city", "bounding_box": rectangle("-66.129403,18.329169 -65.991542,18.480827") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72127, "countyName": "San Juan", "cityID": 7276770, "cityName": "San Juan" } }
+{ "create_at": datetime("2016-01-03T10:00:17.000Z"), "id": 683709482292121601, "text": "In America it pays to be the 'right' skin color #OregonUnderAttack https://t.co/DjUNpE3S9u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "OregonUnderAttack" }}, "user": { "id": 280288703, "name": "Amr No 2 CC", "screen_name": "Cairo67Unedited", "lang": "en", "location": "New York City", "create_at": date("2011-04-10"), "description": "Independent Journalist, frequently published, always outspoken, \nimpained by Egypt's reality and working to change it.\nصحفى مستقل يعشق تراب مصر", "followers_count": 6164, "friends_count": 664, "statues_count": 129840 }, "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-03T10:00:17.000Z"), "id": 683709482321317888, "text": "I need like 80 plain colored shirts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 846257570, "name": "lil cozy", "screen_name": "SenzuLean", "lang": "en", "location": "san marcos ", "create_at": date("2012-09-25"), "description": "these cheesefries are gangster", "followers_count": 568, "friends_count": 167, "statues_count": 30131 }, "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-03T10:00:17.000Z"), "id": 683709482430500865, "text": "Let's make this last game count @Giants", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 240734425 }}, "user": { "id": 579453701, "name": "fidel ⚓️", "screen_name": "fidelmartinez5_", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2012-05-13"), "description": "The thunder is just Cliff making heaven heavy⚡️ #ViscaBarça", "followers_count": 2325, "friends_count": 1988, "statues_count": 34187 }, "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-03T10:00:17.000Z"), "id": 683709482451468288, "text": "@BestBuySupport @BestBuySupport A gift was bought with a gift card and I have the receipt but i need the $80 more so I returned it and", "in_reply_to_status": 683701371120386048, "in_reply_to_user": 987335952, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 987335952, 987335952 }}, "user": { "id": 335100981, "name": "MO Radio Show", "screen_name": "MORadioShow", "lang": "en", "location": "Fort Wayne, Indiana", "create_at": date("2011-07-13"), "description": "Co Host of @BSSportstheFan Show, big time guests, expert breakdowns, Rants, and even entertaining at times!! 7-9 pm est daily. some days I don't even cry", "followers_count": 37264, "friends_count": 22431, "statues_count": 27691 }, "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-03T10:00:18.000Z"), "id": 683709482598281218, "text": "FINS UP!!!!�������� https://t.co/DAOjy9LiZe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.238776,25.9579"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1542093314, "name": "Tori Wiles", "screen_name": "ToriWiles", "lang": "en", "location": "null", "create_at": date("2013-06-23"), "description": "null", "followers_count": 379, "friends_count": 263, "statues_count": 1285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sun Life Stadium", "id": "07d9d22295c85000", "name": "Sun Life Stadium", "place_type": "poi", "bounding_box": rectangle("-80.2387761,25.957899899999997 -80.238776,25.9579") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245060, "cityName": "Miami Gardens" } }
+{ "create_at": datetime("2016-01-03T10:00:18.000Z"), "id": 683709482606563328, "text": "@jennschiffer he was hilarious, I miss my txt messages with him the most.", "in_reply_to_status": 683708825225007107, "in_reply_to_user": 12524622, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 12524622 }}, "user": { "id": 668423, "name": "Mikeal Rogers", "screen_name": "mikeal", "lang": "en", "location": "San Francisco, CA", "create_at": date("2007-01-19"), "description": "Creator of NodeConf & request. Community @ Node.js Foundation. All gifs from One-Punch Man :)", "followers_count": 13445, "friends_count": 454, "statues_count": 37103 }, "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-03T10:00:18.000Z"), "id": 683709482732392448, "text": "I hate a runny nose!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3164594514, "name": "Maddie Lynch", "screen_name": "maddielynchh1", "lang": "en", "location": "null", "create_at": date("2015-04-19"), "description": "null", "followers_count": 28, "friends_count": 22, "statues_count": 71 }, "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-01-03T10:00:18.000Z"), "id": 683709482816241664, "text": "Sunny this afternoon, high 42 (6 C). Low 22 (-6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26195145, "name": "Wichita Weather", "screen_name": "WichitaWeather", "lang": "en", "location": "Wichita, KS", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Wichita, KS", "followers_count": 1847, "friends_count": 88, "statues_count": 7744 }, "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-03T10:00:18.000Z"), "id": 683709482875109376, "text": "@doctn38 @chipwilliamsjr @SainAsylum I expect Chalmers to be back. I would say Barnes as well, but with his love of the west coast..", "in_reply_to_status": 683709334791045120, "in_reply_to_user": 632592796, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 297568748, 125853306, 2269833884 }}, "user": { "id": 632592796, "name": "Grizzn16(18-17)(4-11", "screen_name": "MCSportsShow", "lang": "en", "location": "Memphis,tn", "create_at": date("2012-07-10"), "description": "Host of The Mancave w/Isaac Simpson every Tue 1-3pm CST @730YAHOOSPORTS THE SPORTS GIANT! I also cover U of Memphis Lady Tigers bb for TigerSportsReport dot com", "followers_count": 1375, "friends_count": 2095, "statues_count": 39459 }, "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-03T10:00:18.000Z"), "id": 683709483139256320, "text": "I said i was going to church over break but look where im at lol the crib", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 724491188, "name": "❄️", "screen_name": "bosser_thanYOU", "lang": "en", "location": "kzoo ", "create_at": date("2012-07-29"), "description": "go to school and she gorgeous", "followers_count": 2930, "friends_count": 1226, "statues_count": 92594 }, "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-03T10:00:18.000Z"), "id": 683709483193921541, "text": "@ConradKehn @DLudwigMusic Which I guess is a way of saying that I agree, but maybe notated comp. isn't the way to get young students there.", "in_reply_to_status": 683691250638127104, "in_reply_to_user": 4631811978, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4631811978, 22549433 }}, "user": { "id": 534496190, "name": "Greg Simon", "screen_name": "gregsimonmusic", "lang": "en", "location": "Ann Arbor, MI", "create_at": date("2012-03-23"), "description": "Composer, jazz trumpeter, music writer. Faculty @brevardmusic. Learned things @umich, @CUBoulder + @pugetsound. Fan of hockey, whiskey, and short stories.", "followers_count": 534, "friends_count": 510, "statues_count": 4335 }, "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-01-03T10:00:18.000Z"), "id": 683709483734937604, "text": "Who's the best point guard in 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": 3054654729, "name": "Logan Scott", "screen_name": "slimsmeegy16", "lang": "en", "location": "Independence, KY", "create_at": date("2015-02-22"), "description": "New Twitter... meet me up high somewhere by the sqaure #BBN", "followers_count": 315, "friends_count": 237, "statues_count": 1088 }, "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-01-03T10:00:18.000Z"), "id": 683709483760115712, "text": "It is currently cloudy and 27ºF. #MorrisWeather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.9,45.59"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MorrisWeather" }}, "user": { "id": 1931463259, "name": "Morris Weather", "screen_name": "MorrisMNWeather", "lang": "en", "location": "Morris, MN", "create_at": date("2013-10-03"), "description": "A place to find the current weather in Morris, MN. Run by @bman4789\n\nScheduled tweets are at 7 AM, 12 PM, 3 PM, 6 PM, and 10 PM. Daily forecast at 6 AM.", "followers_count": 21, "friends_count": 1, "statues_count": 1773 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morris, MN", "id": "dc943406a248a125", "name": "Morris", "place_type": "city", "bounding_box": rectangle("-95.933929,45.564348 -95.883709,45.607334") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27149, "countyName": "Stevens", "cityID": 2744242, "cityName": "Morris" } }
+{ "create_at": datetime("2016-01-03T10:00:18.000Z"), "id": 683709483818835970, "text": "845➡️631", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 550211841, "name": "lor", "screen_name": "l_orrie", "lang": "en", "location": "null", "create_at": date("2012-04-10"), "description": "UA, you know", "followers_count": 1052, "friends_count": 766, "statues_count": 34613 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monticello, NY", "id": "d0e3f7b148d1a3bd", "name": "Monticello", "place_type": "city", "bounding_box": rectangle("-74.729486,41.636471 -74.667195,41.67351") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36105, "countyName": "Sullivan", "cityID": 3648175, "cityName": "Monticello" } }
+{ "create_at": datetime("2016-01-03T10:00:18.000Z"), "id": 683709483864829952, "text": "And another in which a mancrush was carrying me and running across a beach and I was somehow 50 feet in the air", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17468876, "name": "Caroline Gottlieb", "screen_name": "CarolineBG", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2008-11-18"), "description": "actress, wannabe poet, baby whisperer, believer of aliens. SAG-AFTRA", "followers_count": 412, "friends_count": 320, "statues_count": 7876 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-03T10:00:18.000Z"), "id": 683709484104036352, "text": "�������� https://t.co/qYrGscD4o3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2347654228, "name": "rae.", "screen_name": "raaevenn", "lang": "en", "location": "dmv", "create_at": date("2014-02-16"), "description": "null", "followers_count": 1567, "friends_count": 1257, "statues_count": 16752 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Centreville, VA", "id": "ffcc53c4a4e7a620", "name": "Centreville", "place_type": "city", "bounding_box": rectangle("-77.479597,38.802143 -77.397429,38.880183") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5114440, "cityName": "Centreville" } }
+{ "create_at": datetime("2016-01-03T10:00:18.000Z"), "id": 683709484254900225, "text": "Can you recommend anyone for this #job? Dishwasher - https://t.co/DU70owZive #Napa, CA #Hospitality #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.2988889,38.3047222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Napa", "Hospitality", "Hiring" }}, "user": { "id": 731027906, "name": "Guckenheimer Careers", "screen_name": "GuckenheimerJob", "lang": "en", "location": "null", "create_at": date("2012-08-01"), "description": "We are a high end food service company providing restaurant quality food in corporate settings. We are looking to fill a variety of positions across the US.", "followers_count": 136, "friends_count": 2, "statues_count": 298 }, "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-03T10:00:18.000Z"), "id": 683709484267601921, "text": "Watch https://t.co/rUBqABGNfT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 427154865, "name": "PFM KICKSTANDBOY", "screen_name": "TikeaceHarris", "lang": "en", "location": "null", "create_at": date("2011-12-02"), "description": "IM ALL ABOUT A GOOD TIME PFM BOOK US IF YOU GOT A PARTY COMING UP WE WILL GET IT POP FOR THE RIGHT PRICE.....", "followers_count": 1697, "friends_count": 1132, "statues_count": 37794 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milford, DE", "id": "ee3ec8d5e41e2677", "name": "Milford", "place_type": "city", "bounding_box": rectangle("-75.45787,38.886046 -75.40221,38.942683") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10005, "countyName": "Sussex", "cityID": 1047420, "cityName": "Milford" } }
+{ "create_at": datetime("2016-01-03T10:00:18.000Z"), "id": 683709484292648961, "text": "Baby don't be mad jus come stay 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": 4568331223, "name": "Daysha Lynn.", "screen_name": "queen_dayshh", "lang": "en", "location": "null", "create_at": date("2015-12-15"), "description": "KenedyTX.", "followers_count": 133, "friends_count": 294, "statues_count": 333 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kenedy, TX", "id": "1742e827abaf7628", "name": "Kenedy", "place_type": "city", "bounding_box": rectangle("-97.8703,28.795099 -97.832351,28.832469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48255, "countyName": "Karnes", "cityID": 4838860, "cityName": "Kenedy" } }
+{ "create_at": datetime("2016-01-03T10:00:18.000Z"), "id": 683709484305379329, "text": "@NathanSykes could you follow me PLZ", "in_reply_to_status": 683709241136291840, "in_reply_to_user": 90896009, "favorite_count": 0, "coordinate": point("-82.45436037,28.92205827"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 90896009 }}, "user": { "id": 601459322, "name": "Becca loves Edwin/27", "screen_name": "becks__hunter", "lang": "en", "location": "Beverly Hills, FL", "create_at": date("2012-06-06"), "description": "Don't let anyone tell you can't chase dreams-kalin and myles jake and myles follow Cameron faved me on 9/2/15 Galaxy 6 2015-2016", "followers_count": 2461, "friends_count": 2721, "statues_count": 64383 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beverly Hills, FL", "id": "e629c744157f3740", "name": "Beverly Hills", "place_type": "city", "bounding_box": rectangle("-82.476559,28.904041 -82.435463,28.936022") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12017, "countyName": "Citrus", "cityID": 1206125, "cityName": "Beverly Hills" } }
+{ "create_at": datetime("2016-01-03T10:00:18.000Z"), "id": 683709484632424448, "text": "Department of Veterans Affairs #Clerical #Job: Program Specialist (#NewOrleans, LA) https://t.co/BqQn1IGYoq #Veterans #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.0705556,29.9647222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Clerical", "Job", "NewOrleans", "Veterans", "Jobs", "Hiring" }}, "user": { "id": 31625510, "name": "TMJ-LAN Cleric. Jobs", "screen_name": "tmj_lan_cler", "lang": "en", "location": "New Orleans, LA", "create_at": date("2009-04-15"), "description": "Follow this account for geo-targeted Office/Clerical job tweets in New Orleans, LA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 239, "friends_count": 204, "statues_count": 16 }, "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-03T10:00:18.000Z"), "id": 683709485223813124, "text": "#Sales #Job in #Naples, FL: Event Specialist Part Time Sales at Advantage Solutions https://t.co/z7Z9NAYny2 #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.7948103,26.1420358"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "Job", "Naples", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22327673, "name": "TMJ-FL-US Sales Jobs", "screen_name": "tmj_fl_sales", "lang": "en", "location": "Florida", "create_at": date("2009-02-28"), "description": "Follow this account for geo-targeted Sales job tweets in Florida Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 474, "friends_count": 304, "statues_count": 654 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Naples, FL", "id": "a3f971b69fefb699", "name": "Naples", "place_type": "city", "bounding_box": rectangle("-81.817216,26.093538 -81.766738,26.211277") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12021, "countyName": "Collier", "cityID": 1247625, "cityName": "Naples" } }
+{ "create_at": datetime("2016-01-03T10:00:18.000Z"), "id": 683709485450309632, "text": "Mostly sunny this afternoon, high 27 (-3 C). Low 13 (-11 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26199170, "name": "Omaha Weather", "screen_name": "OmahaWX", "lang": "en", "location": "Omaha, Nebraska", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Omaha, Nebraska", "followers_count": 2590, "friends_count": 88, "statues_count": 7737 }, "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-03T10:00:18.000Z"), "id": 683709485534285825, "text": "Stop believing everything ppl say", "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": "Just Ask", "followers_count": 1670, "friends_count": 675, "statues_count": 8693 }, "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-03T10:00:18.000Z"), "id": 683709485806936064, "text": "death gotta be easy cuz 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": 350662005, "name": "Lo ♛❥", "screen_name": "love_laurennnnn", "lang": "en", "location": "NYC ", "create_at": date("2011-08-07"), "description": "null", "followers_count": 346, "friends_count": 390, "statues_count": 31926 }, "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-03T10:00:18.000Z"), "id": 683709485823737856, "text": "@NutriNancy Yes, great top choice, Nancy. Happy New Year!", "in_reply_to_status": 683708865561509888, "in_reply_to_user": 519539251, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 519539251 }}, "user": { "id": 523657706, "name": "Cindy Silver RD", "screen_name": "CindySilverRD", "lang": "en", "location": "Winston-Salem, NC", "create_at": date("2012-03-13"), "description": "An experienced retail food industry Registered Dietitian teaching clients tasty\r\nmeal planning, smart grocery shopping, simple & nutritious cooking at home.", "followers_count": 3259, "friends_count": 3003, "statues_count": 11958 }, "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-03T10:00:18.000Z"), "id": 683709486301855744, "text": "@lares_paul wait really?", "in_reply_to_status": 683703056534470656, "in_reply_to_user": 487207490, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 487207490 }}, "user": { "id": 2870921739, "name": "Kota", "screen_name": "dakota_davis_", "lang": "en", "location": "Warren, OH", "create_at": date("2014-11-10"), "description": "I was born and now I'm here • XXX", "followers_count": 396, "friends_count": 220, "statues_count": 7712 }, "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-03T10:00:18.000Z"), "id": 683709486301863937, "text": "@____Stola Check out my brand new music video \"Smooth\" #EvolveCrew #Norfolk #KnowingBetter https://t.co/wCNRmlqet0", "in_reply_to_status": -1, "in_reply_to_user": 1309488104, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "EvolveCrew", "Norfolk", "KnowingBetter" }}, "user_mentions": {{ 1309488104 }}, "user": { "id": 21249017, "name": "Page", "screen_name": "Pagemuzik", "lang": "en", "location": "fethernfinbout2getaboneless VA", "create_at": date("2009-02-18"), "description": "Tryin to spread love & shit to the darkest reaches of the galaxy http://www.datpiff.com/pop-mixtape-download.php?id=m6d433ad", "followers_count": 2148, "friends_count": 380, "statues_count": 14967 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dix Hills, NY", "id": "18165a3f2b139f0f", "name": "Dix Hills", "place_type": "city", "bounding_box": rectangle("-73.380745,40.767592 -73.29345,40.83856") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3620687, "cityName": "Dix Hills" } }
+{ "create_at": datetime("2016-01-03T10:00:18.000Z"), "id": 683709486377271297, "text": "@talius (cont) Serpentine channel reactor shows parallel reactor +feeder channels separated by nanoporous membrane https://t.co/GqPrsUqjS1", "in_reply_to_status": 683709116578164738, "in_reply_to_user": 14518761, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14518761 }}, "user": { "id": 14518761, "name": "Tali", "screen_name": "talius", "lang": "en", "location": "CA, US", "create_at": date("2008-04-24"), "description": "Med, Science & Web dev expert, Get TOP TIPS for medicine, #organic health, #fitness, #budgeting, discount #coupons, cooking , #fashion&beauty. Join me on fb", "followers_count": 9605, "friends_count": 9353, "statues_count": 59125 }, "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-03T10:00:19.000Z"), "id": 683709487107059712, "text": "Patriots and bills today", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1142373764, "name": "JT Holmes", "screen_name": "travfourr", "lang": "en", "location": "AUSTIN", "create_at": date("2013-02-02"), "description": "Married someone too good for me. Father of 2 girls. Love the stock market, horses beaches and cruising. Meat cutter.", "followers_count": 878, "friends_count": 965, "statues_count": 31510 }, "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-03T10:00:19.000Z"), "id": 683709487153344512, "text": "@Thebestsydney @Brynnnnnie mhmm Syd��", "in_reply_to_status": 683709318089314304, "in_reply_to_user": 546940124, "favorite_count": 0, "retweet_count": 0, "lang": "et", "is_retweet": false, "user_mentions": {{ 546940124, 739363285 }}, "user": { "id": 502248265, "name": "Jewoin", "screen_name": "jewoin", "lang": "en", "location": "null", "create_at": date("2012-02-24"), "description": "tshs|sc-jewoin", "followers_count": 1227, "friends_count": 652, "statues_count": 16408 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Holiday, FL", "id": "8c1a061c37aea043", "name": "Holiday", "place_type": "city", "bounding_box": rectangle("-82.78516,28.172768 -82.694184,28.204041") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12101, "countyName": "Pasco", "cityID": 1231075, "cityName": "Holiday" } }
+{ "create_at": datetime("2016-01-03T10:00:19.000Z"), "id": 683709487157395456, "text": "Mom's First Texans Game @WeAreTexans_ https://t.co/wsusUns7s9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.410724,29.684697"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 940662012 }}, "user": { "id": 2673861846, "name": "Audree M Hernandez", "screen_name": "AudreeSATX", "lang": "en", "location": "null", "create_at": date("2014-07-23"), "description": "Passionate about opportunity for all! Regional Director, College Advising Corps, but my tweets are my own.", "followers_count": 92, "friends_count": 178, "statues_count": 330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "NRG Stadium", "id": "07d9e3ab78486002", "name": "NRG Stadium", "place_type": "poi", "bounding_box": rectangle("-95.4107241,29.6846969 -95.410724,29.684697") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-03T10:00:19.000Z"), "id": 683709487283331072, "text": "Can't just drive by #Plattsburgh without a stop for michigan dogs #northcountryeats @ Michigans… https://t.co/0bDBWyTXAi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.477417,44.6970901"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Plattsburgh", "northcountryeats" }}, "user": { "id": 1414564596, "name": "Chopsticks Optional", "screen_name": "chopsticksopt", "lang": "en", "location": "Albany, NY", "create_at": date("2013-05-08"), "description": "J+R exploring #518eats (and beyond) one bite at a time.", "followers_count": 341, "friends_count": 639, "statues_count": 1382 }, "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-03T10:00:19.000Z"), "id": 683709487333650432, "text": "Not trynna let me leave��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1195154804, "name": "ÑBÃ.⭐️", "screen_name": "jaelen_parker", "lang": "en", "location": "Carroll", "create_at": date("2013-02-18"), "description": "Philippians 4:13 / Never Broke Again", "followers_count": 123, "friends_count": 131, "statues_count": 69 }, "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-03T10:00:19.000Z"), "id": 683709487736336385, "text": "@pabloalboran Till we meet again. Always on my mind❤️ #MissYou https://t.co/deLCMZX6bK", "in_reply_to_status": -1, "in_reply_to_user": 119776110, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MissYou" }}, "user_mentions": {{ 119776110 }}, "user": { "id": 344450249, "name": "Gabby Q.", "screen_name": "gabiotita89", "lang": "en", "location": "Illinois, USA", "create_at": date("2011-07-28"), "description": "My musical tastes are ECLECTIC.", "followers_count": 855, "friends_count": 733, "statues_count": 30315 }, "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-03T10:00:19.000Z"), "id": 683709487887167488, "text": "weather outside is so nice mashallah ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 390050261, "name": "amir", "screen_name": "aeldouh_", "lang": "en", "location": "egypt | tx ", "create_at": date("2011-10-13"), "description": "lord willin, inshallah", "followers_count": 450, "friends_count": 344, "statues_count": 13871 }, "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-03T10:00:19.000Z"), "id": 683709487987986432, "text": "Piper. #passion2016 @ Infinite Energy Center https://t.co/X5SDaymPBk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.0930176,33.9914894"), "retweet_count": 0, "lang": "it", "is_retweet": false, "hashtags": {{ "passion2016" }}, "user": { "id": 364411127, "name": "Montgomery Loehlein", "screen_name": "Montyql", "lang": "en", "location": "null", "create_at": date("2011-08-29"), "description": "...the three men I admire most, the Father, Son, and the Holy Ghost", "followers_count": 107, "friends_count": 217, "statues_count": 3132 }, "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": 13135, "countyName": "Gwinnett" } }
+{ "create_at": datetime("2016-01-03T10:00:19.000Z"), "id": 683709488780546048, "text": "@jordanpearson25 https://t.co/6ybenO2NSV", "in_reply_to_status": -1, "in_reply_to_user": 69123405, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 69123405 }}, "user": { "id": 2850764084, "name": "THE Lesbian", "screen_name": "taylor_mulcahy1", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2014-10-10"), "description": "Hi my names Taylor, everyone just refers to me as THE Lesbian. feel free to call me either. Sandra Day O'Connor high school. senior, class of 2016. 17, Pisces.", "followers_count": 117, "friends_count": 97, "statues_count": 2188 }, "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-03T10:00:19.000Z"), "id": 683709488780578816, "text": "Mostly sunny this afternoon, high 39 (4 C). Low 24 (-4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433220, "name": "St. Louis Weather", "screen_name": "StLouisWeather", "lang": "en", "location": "St. Louis Metro Area", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for St. Louis Metro Area", "followers_count": 3677, "friends_count": 88, "statues_count": 7710 }, "place": { "country": "United States", "country_code": "United States", "full_name": "St. Louis, MO", "id": "60e6df5778ff9dac", "name": "St. Louis", "place_type": "city", "bounding_box": rectangle("-90.320515,38.531852 -90.166409,38.774346") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29510, "countyName": "St. Louis", "cityID": 2965000, "cityName": "St. Louis" } }
+{ "create_at": datetime("2016-01-03T10:00:19.000Z"), "id": 683709488826810368, "text": "@dooditserika sports team activities it's easy you don't really do anything", "in_reply_to_status": 683709211079933952, "in_reply_to_user": 597123952, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 597123952 }}, "user": { "id": 177118131, "name": "❄️Bradley❄️", "screen_name": "brad_mcclelland", "lang": "en", "location": "Eating A Pizza Somewhere", "create_at": date("2010-08-11"), "description": "I love SOUTHSIDE SPORTS #SeanStrong #ShakaBrah 1-22-16", "followers_count": 667, "friends_count": 546, "statues_count": 20520 }, "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-03T10:00:19.000Z"), "id": 683709488910725121, "text": "mmm mmm yeah yeah all that could say was #GreeceNeedsAustin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GreeceNeedsAustin" }}, "user": { "id": 2593277406, "name": "maria mahomie", "screen_name": "mariamahomie74", "lang": "el", "location": "Nafplio, Lefkakia", "create_at": date("2014-06-28"), "description": "@AustinMahone I love you so much and I'm so proud couse I'm mahomie and plzzz Austin follow me plzzz is so important for me 33", "followers_count": 257, "friends_count": 452, "statues_count": 2599 }, "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-03T10:00:19.000Z"), "id": 683709488956755968, "text": "If you got a great girl down for you, why would you want to flirt with other girls? It's 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": 998379391, "name": "Doah ॐ", "screen_name": "shenaandoah", "lang": "en", "location": "Bellflower, CA.", "create_at": date("2012-12-08"), "description": "One Of A Kind ❂", "followers_count": 922, "friends_count": 965, "statues_count": 19418 }, "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-03T10:00:19.000Z"), "id": 683709489099325440, "text": "#Sales #Job alert: Event Specialist Part Time Sales | Advantage Solutions | #Jacksonville, NC https://t.co/BPFndFy1Tr #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.4302414,34.7540524"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "Job", "Jacksonville", "Jobs", "Hiring" }}, "user": { "id": 67350081, "name": "TMJ-NC Sales Jobs", "screen_name": "tmj_nc_sales", "lang": "en", "location": "North Carolina", "create_at": date("2009-08-20"), "description": "Follow this account for geo-targeted Sales job tweets in North Carolina. Need help? Tweet us at @CareerArc!", "followers_count": 369, "friends_count": 302, "statues_count": 183 }, "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.32187,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37133, "countyName": "Onslow", "cityID": 3734200, "cityName": "Jacksonville" } }
+{ "create_at": datetime("2016-01-03T10:00:19.000Z"), "id": 683709489313230848, "text": "Pregame warmups are underway in Fargo #DU1NATION #SummitMBB https://t.co/YKfG8CLu0O", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DU1NATION", "SummitMBB" }}, "user": { "id": 40279630, "name": "Denver Men's Hoops", "screen_name": "DU_MHoops", "lang": "en", "location": "Denver, Colorado", "create_at": date("2009-05-15"), "description": "Official Twitter Site for the University of Denver Men's Basketball Team #DU1NATION #DUGameDay", "followers_count": 1869, "friends_count": 470, "statues_count": 6538 }, "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-03T10:00:19.000Z"), "id": 683709489392963584, "text": "Can you recommend anyone for this #Healthcare #job? https://t.co/LxH9wfAEGn #SanMateo, CA #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.3255254,37.5629917"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "SanMateo", "Hiring", "CareerArc" }}, "user": { "id": 120915363, "name": "TMJ-CAP Health Jobs", "screen_name": "tmj_CAP_health", "lang": "en", "location": "The Peninsula, CA", "create_at": date("2010-03-07"), "description": "Follow this account for geo-targeted Healthcare job tweets in The Peninsula, CA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 354, "friends_count": 279, "statues_count": 113 }, "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-03T10:00:19.000Z"), "id": 683709489397170176, "text": "Man i rock Kentucky blue ���� on these hoes ... Shrugs *", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1687788324, "name": "➕Šhüń➕", "screen_name": "blessed_boyz", "lang": "en", "location": "@ the night show ", "create_at": date("2013-08-21"), "description": ".. +ŁØĘ+ if you leave shun's world its a long walk home", "followers_count": 566, "friends_count": 658, "statues_count": 24321 }, "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-03T10:00:19.000Z"), "id": 683709489443307520, "text": "I'm fucking weak �������������� https://t.co/BNS3vXc8XP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 565743631, "name": "droffaG ydoC", "screen_name": "codygafford94", "lang": "en", "location": "River Ridge, LA", "create_at": date("2012-04-28"), "description": "Groovy like a drive in movie. Snapchat: codygafford", "followers_count": 495, "friends_count": 795, "statues_count": 12965 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Violet, LA", "id": "f6158506a43e47cc", "name": "Violet", "place_type": "city", "bounding_box": rectangle("-89.907768,29.875438 -89.874714,29.924759") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22087, "countyName": "St. Bernard", "cityID": 2278855, "cityName": "Violet" } }
+{ "create_at": datetime("2016-01-03T10:00:19.000Z"), "id": 683709489573457921, "text": "The fact that Lauren and I stayed up till 5:30 watching prison break", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 312275503, "name": "jacqueline ✨", "screen_name": "jackietlapierre", "lang": "en", "location": "null", "create_at": date("2011-06-06"), "description": "just be happy✌️", "followers_count": 472, "friends_count": 476, "statues_count": 18754 }, "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-03T10:00:19.000Z"), "id": 683709489887973376, "text": "#2016 #Hoes @ Atlanta, Georgia https://t.co/SVt0sEOFWK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.3881,33.7489"), "retweet_count": 0, "lang": "lt", "is_retweet": false, "hashtags": {{ "Hoes" }}, "user": { "id": 209377292, "name": "Bbra 9027", "screen_name": "bbra9027", "lang": "en", "location": "Georgia (USA)", "create_at": date("2010-10-28"), "description": "Hello, this is the twitter page for my ebay account bbra9027 as well as my youtube. http://www.youtube.com/user/atlcardcollector. Thanks for looking. X-)", "followers_count": 334, "friends_count": 2019, "statues_count": 6705 }, "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-03T10:00:19.000Z"), "id": 683709489946750976, "text": "Sex more that's what I was on , talk less that's what I was on", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.1676257,39.978568"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 200566730, "name": "January 3rd 3", "screen_name": "ChocoliteDutch", "lang": "en", "location": "Blowin Loud wit a Badd Nigga", "create_at": date("2010-10-09"), "description": "null", "followers_count": 299, "friends_count": 331, "statues_count": 36242 }, "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-03T10:00:19.000Z"), "id": 683709490013822977, "text": "10 Pound by @keznamdi studio style w/ @naturalhighja Chalace Palace series ��✂️ @veganfoodshare… https://t.co/uQAUzVfqfI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.48267001,34.40944999"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 397734945, 1663377121, 543082598 }}, "user": { "id": 159037035, "name": "CHAMPION", "screen_name": "Tesfai7", "lang": "en", "location": "International ", "create_at": date("2010-06-24"), "description": "Live to Love; Road Manager @keznamdi Booking: Contact keznamdibooking@gmail.com", "followers_count": 602, "friends_count": 248, "statues_count": 14408 }, "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-03T10:00:19.000Z"), "id": 683709490114498560, "text": "Got to watch this game just cause it's probably the last one", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 151223468, "name": "Tylim$", "screen_name": "SkinnySco", "lang": "en", "location": "Braddock, PA Drowning In Rain ", "create_at": date("2010-06-02"), "description": "Originality ILC ❤️", "followers_count": 509, "friends_count": 354, "statues_count": 14841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Pittsburgh, PA", "id": "4f178f93e5a98b8a", "name": "East Pittsburgh", "place_type": "city", "bounding_box": rectangle("-79.845093,40.390824 -79.829844,40.405892") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4221712, "cityName": "East Pittsburgh" } }
+{ "create_at": datetime("2016-01-03T10:00:19.000Z"), "id": 683709490277957632, "text": "Newports and Seawater https://t.co/D4sKY1GfqA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4147638556, "name": "Tre Wilson", "screen_name": "TreMfCuddii", "lang": "en", "location": "null", "create_at": date("2015-11-08"), "description": "Just Call Me Daddy", "followers_count": 558, "friends_count": 1083, "statues_count": 3154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxford, AL", "id": "008b18ef28c3d13b", "name": "Oxford", "place_type": "city", "bounding_box": rectangle("-85.925825,33.552521 -85.715332,33.639362") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1015, "countyName": "Calhoun", "cityID": 157576, "cityName": "Oxford" } }
+{ "create_at": datetime("2016-01-03T10:00:19.000Z"), "id": 683709490328276993, "text": "What?????! Couldn't be any clearer, Quaker Oats. #pleaseexplain @ Venice, California https://t.co/kgzuDTeJUN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.459,33.9908"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "pleaseexplain" }}, "user": { "id": 18254423, "name": "Madonna Deverson", "screen_name": "madnyc", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2008-12-19"), "description": "EVP/Brand Intelligence, Deutsch Inc. Tweets not reflective of company. Sometimes I vent about culture, news + life.", "followers_count": 1098, "friends_count": 1926, "statues_count": 9495 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-03T10:00:19.000Z"), "id": 683709490349404160, "text": "@armskyrs_ BITCH�� https://t.co/EGQuvC2Klo", "in_reply_to_status": -1, "in_reply_to_user": 354484818, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 354484818 }}, "user": { "id": 605117281, "name": "alaya", "screen_name": "alayadivine", "lang": "en", "location": "null", "create_at": date("2012-06-10"), "description": "null", "followers_count": 2057, "friends_count": 857, "statues_count": 51549 }, "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-03T10:00:19.000Z"), "id": 683709490353549312, "text": "Mixtape drop Feb 6tg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 2264692822, "name": "H⭕️tboy Mac", "screen_name": "_yo_babydaddy", "lang": "en", "location": "901 〽️", "create_at": date("2014-01-05"), "description": "| Man of God | Ru.I.P. Slancho | MTSU", "followers_count": 1900, "friends_count": 1136, "statues_count": 14292 }, "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-03T10:00:20.000Z"), "id": 683709490940788737, "text": "when Teresa makes you do workout videos that you can't do @td9620", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1726055161 }}, "user": { "id": 1353088710, "name": "Sabrina Gargano", "screen_name": "SabrinaaGargano", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2013-04-14"), "description": "ermhs", "followers_count": 285, "friends_count": 630, "statues_count": 5207 }, "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-03T10:00:20.000Z"), "id": 683709490953359360, "text": "smh a whole group of girls kept their hats on during the national anthem", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 494513169, "name": "lil b", "screen_name": "Brooke__Barnes", "lang": "en", "location": "122914", "create_at": date("2012-02-16"), "description": "rugby", "followers_count": 493, "friends_count": 479, "statues_count": 19454 }, "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-03T10:00:20.000Z"), "id": 683709491108421633, "text": "Faith and science are different things. https://t.co/kdHuoxa9pw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 374792308, "name": "Kenneth Bruno", "screen_name": "KSBruno9", "lang": "en", "location": "Walnut Creek, CA", "create_at": date("2011-09-16"), "description": "Father, Husband, fungal molecular biologist, Texas Aggie. Football, cooking, and science", "followers_count": 535, "friends_count": 1639, "statues_count": 20950 }, "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-03T10:00:20.000Z"), "id": 683709491431387136, "text": "Mostly sunny this afternoon, high 35 (2 C). Low 20 (-7 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433608, "name": "Kansas City Weather", "screen_name": "KansasCityWX", "lang": "en", "location": "Kansas City Metro Area", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Kansas City Metro Area", "followers_count": 1931, "friends_count": 88, "statues_count": 7830 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "ddabe75a8f44fe5a", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.765917,38.827176 -94.385522,39.356662") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-01-03T10:00:20.000Z"), "id": 683709491578273793, "text": "@Matt_Restrepo95 @MCFC they know how to get dicked down", "in_reply_to_status": 683696260088500224, "in_reply_to_user": 3197756570, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3197756570, 14573900 }}, "user": { "id": 101360601, "name": "Tony", "screen_name": "AnthonyDen18", "lang": "en", "location": "in hell", "create_at": date("2010-01-02"), "description": "J Cole ruined rap and we don't wear joggers", "followers_count": 687, "friends_count": 294, "statues_count": 76618 }, "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-03T10:00:20.000Z"), "id": 683709491624448001, "text": "@antoniambruno @cathryncarmona we're all so ugly that it's cute", "in_reply_to_status": 683709306177499137, "in_reply_to_user": 2986451068, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2986451068, 1000671410 }}, "user": { "id": 545708125, "name": "Happy Holid(aja)s!", "screen_name": "AjaLapine6", "lang": "en", "location": "my name is pronounced Asia ", "create_at": date("2012-04-04"), "description": "let's just play in the dirt, it's free", "followers_count": 333, "friends_count": 290, "statues_count": 8756 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Patchogue, NY", "id": "4e7078258214531e", "name": "North Patchogue", "place_type": "city", "bounding_box": rectangle("-73.042337,40.772357 -73.007588,40.795224") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3653319, "cityName": "North Patchogue" } }
+{ "create_at": datetime("2016-01-03T10:00:20.000Z"), "id": 683709491901259778, "text": "Who's sick of the Gyro George commercial already? #Uncle", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Uncle" }}, "user": { "id": 23568996, "name": "Mandy Sloan", "screen_name": "mjsloan84", "lang": "en", "location": "Ashland, OH", "create_at": date("2009-03-10"), "description": "Love Jesus, my husband, family, friends and country. Avid fan of Cleveland sports, video games and a good book/movie.", "followers_count": 302, "friends_count": 150, "statues_count": 15254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ashland, OH", "id": "fefd06f07572907a", "name": "Ashland", "place_type": "city", "bounding_box": rectangle("-82.352896,40.835537 -82.275563,40.893194") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39005, "countyName": "Ashland", "cityID": 3902568, "cityName": "Ashland" } }
+{ "create_at": datetime("2016-01-03T10:00:20.000Z"), "id": 683709491980955649, "text": "you don't know how you're gonna wake up for school tomorrow....we 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": 801583698, "name": "toni puma", "screen_name": "tonipumaa", "lang": "en", "location": "on the field", "create_at": date("2012-09-03"), "description": "amidst a dream with no exit doors", "followers_count": 1186, "friends_count": 836, "statues_count": 98109 }, "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-03T10:00:20.000Z"), "id": 683709492106670080, "text": "@WitMiBBall probably not", "in_reply_to_status": 683709426881196032, "in_reply_to_user": 2279318168, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2279318168 }}, "user": { "id": 57814942, "name": "Patrick Fenelon", "screen_name": "Patrick_Fenelon", "lang": "en", "location": "Saint Paul--NOT MINNEAPOLIS ", "create_at": date("2009-07-17"), "description": "Obsessions: Foreign Policy, History, The NBA. Mike Dunleavy enthusiast. Available for podcasts & to miss writing deadlines. Follow me to freedom, NBA Writers!", "followers_count": 3085, "friends_count": 2232, "statues_count": 170450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Apple Valley, MN", "id": "2dbc9973dd80d3a3", "name": "Apple Valley", "place_type": "city", "bounding_box": rectangle("-93.248871,44.717447 -93.156006,44.775899") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27037, "countyName": "Dakota", "cityID": 2701900, "cityName": "Apple Valley" } }
+{ "create_at": datetime("2016-01-03T10:00:20.000Z"), "id": 683709492278640640, "text": "temperature up 39°F -> 46°F\nhumidity down 81% -> 57%\nwind 0mph -> 7mph\npressure 30.32in falling", "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": 96, "friends_count": 3, "statues_count": 23458 }, "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-03T10:00:20.000Z"), "id": 683709492408774656, "text": "just now waking up�� who wants to bring me food lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 618643774, "name": "☪", "screen_name": "emwxo_", "lang": "en", "location": "440", "create_at": date("2012-06-25"), "description": "SC: princess.emilyy", "followers_count": 1162, "friends_count": 382, "statues_count": 19900 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Amherst, OH", "id": "911b22d874c731d0", "name": "Amherst", "place_type": "city", "bounding_box": rectangle("-82.265659,41.373736 -82.164579,41.426754") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39093, "countyName": "Lorain", "cityID": 3901798, "cityName": "Amherst" } }
+{ "create_at": datetime("2016-01-03T10:00:20.000Z"), "id": 683709492702396416, "text": "@yunggchip0tle You Raw", "in_reply_to_status": 683708850936135684, "in_reply_to_user": 345812812, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 345812812 }}, "user": { "id": 155648620, "name": "Kay Banks", "screen_name": "NoNameKayBanks", "lang": "en", "location": "Cleveland / Macon Ga", "create_at": date("2010-06-14"), "description": "#NoName Headquarter | Song Writer | http://soundcloud.com/NoKayBanks | , #RBMG .. #FreeDoeBoy", "followers_count": 3122, "friends_count": 795, "statues_count": 77248 }, "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-03T10:00:20.000Z"), "id": 683709492744159232, "text": "@Oakunani_ I just seen you arguing with her .. That hoe cop friendly so don't bother her .. She called the cops on me ��", "in_reply_to_status": -1, "in_reply_to_user": 403656278, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 403656278 }}, "user": { "id": 389747581, "name": "Lani Mommy .", "screen_name": "lovelanijade", "lang": "en", "location": "HighSociety .", "create_at": date("2011-10-12"), "description": "null", "followers_count": 2247, "friends_count": 1829, "statues_count": 99683 }, "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-03T10:00:20.000Z"), "id": 683709492786143237, "text": "Can 5 get here faster ?! I wanna go home and watch movies .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 32192423, "name": "Sav Formoso", "screen_name": "savvv_f", "lang": "en", "location": "Cleveland, OH", "create_at": date("2009-04-16"), "description": "How can I be a hero when I'm the one needing saved #LaceUp #EST19XX", "followers_count": 1060, "friends_count": 461, "statues_count": 57626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beachwood, OH", "id": "fa5bd676191716ee", "name": "Beachwood", "place_type": "city", "bounding_box": rectangle("-81.532635,41.449669 -81.48037,41.501448") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3904500, "cityName": "Beachwood" } }
+{ "create_at": datetime("2016-01-03T10:00:20.000Z"), "id": 683709493536935936, "text": "@FormulaBotanica Ditto! Thanks for thinking of us! Happy new year to you too!", "in_reply_to_status": 682606622921850880, "in_reply_to_user": 600092894, "favorite_count": 0, "coordinate": point("-79.98241481,40.43059068"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 600092894 }}, "user": { "id": 45289787, "name": "FutureDerm", "screen_name": "futurederm", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2009-06-06"), "description": "The Science of Beauty. #Follow and tweet me your #skincare and beauty questions! I love hearing from you!", "followers_count": 9203, "friends_count": 2891, "statues_count": 17150 }, "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-03T10:00:20.000Z"), "id": 683709493595783170, "text": "Can't wait to get this wrap off my hand tomorrow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 233087083, "name": "Rob Orell©4⃣", "screen_name": "The1nOnly_Rob", "lang": "en", "location": "Worcester, MA", "create_at": date("2011-01-01"), "description": "Its 5⃣5⃣ in the fall, track star during spring Instagram: @The1nOnly_Rob Snapchat: The1nOnly_Rob", "followers_count": 737, "friends_count": 775, "statues_count": 19090 }, "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-03T10:00:20.000Z"), "id": 683709494245785601, "text": "Sunny this afternoon, high 52 (11 C). Low 28 (-2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433972, "name": "Memphis Weather", "screen_name": "MemphisWX", "lang": "en", "location": "Memphis, TN", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Memphis, TN", "followers_count": 1688, "friends_count": 88, "statues_count": 7742 }, "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-03T10:00:20.000Z"), "id": 683709494493360138, "text": "Still missing this night. @borgore was too good to us. �� @ Decadence https://t.co/IiTosUvejr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.847336,33.330383"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 66995579 }}, "user": { "id": 494397698, "name": "Giovanna Galeano", "screen_name": "TheRealGthanggg", "lang": "en", "location": "Chandler, AZ", "create_at": date("2012-02-16"), "description": "null", "followers_count": 281, "friends_count": 393, "statues_count": 6780 }, "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-03T10:00:20.000Z"), "id": 683709494522580993, "text": "Apparently it's supposed to snow tonight. Man, I don't want school tomirror", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1016798149, "name": "Conner Furu", "screen_name": "FURU_BHH25", "lang": "en", "location": "Oly WA", "create_at": date("2012-12-16"), "description": "BHHS RB C/O 2017⚡️ HARDWORK BEATS TALENT WHEN TALENT DOESNT WORK HARD ENOUGH ⚡️IG: CONNERFURU_BHH5", "followers_count": 376, "friends_count": 350, "statues_count": 1925 }, "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-03T10:00:20.000Z"), "id": 683709494589698048, "text": "The #Texans secondary will have its hands full - B. Bottles (35) second to only T. Brady (36) in TD passes.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Texans" }}, "user": { "id": 123747926, "name": "Casey Keirnan", "screen_name": "CaseyKeirnan", "lang": "en", "location": "San Antonio, TX", "create_at": date("2010-03-16"), "description": "Strikes and gutters. Ups and downs. | Sports reporter and sometimes anchor for @News4SA & @KABBFOX29 - cdkeirnan@sbgtv.com", "followers_count": 1028, "friends_count": 699, "statues_count": 6250 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-03T10:00:20.000Z"), "id": 683709494719823873, "text": "Hoping all our clients and friends had a happy and healthy new year! Come in and visit our showroom to take advantage of January savings!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2727030678, "name": "EthanAllenStatenIsld", "screen_name": "EA_StatenIsland", "lang": "en", "location": "2935 Veterans Road West", "create_at": date("2014-08-12"), "description": "Our talented design pros are ready to inspire and assist you. We're here to work with you in making your space a beautiful place to live in. 718.984.6882", "followers_count": 191, "friends_count": 593, "statues_count": 214 }, "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-03T10:00:20.000Z"), "id": 683709494933622784, "text": "dude i was poppin yesterday idc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2352772854, "name": "alex", "screen_name": "oooalexus", "lang": "en", "location": "ktx", "create_at": date("2014-02-19"), "description": "everything is possible; (4/12)", "followers_count": 1514, "friends_count": 851, "statues_count": 74027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-03T10:00:20.000Z"), "id": 683709495055400960, "text": "Last day to enter to win this Patriots neon light! https://t.co/VoAbRE3oXg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 65488191, "name": "Laurie", "screen_name": "PatsSoxBruins", "lang": "en", "location": "null", "create_at": date("2009-08-13"), "description": "*New England Patriots Diehard fan* #Loyal #SpeakMyMind #TB12 #BostonStrong #PatriotsNation #RedSox #TVD #GFY #SaveWildlife #ISF #JE11 #87 #BeOriginalBeYourself", "followers_count": 426, "friends_count": 229, "statues_count": 24486 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nashua, NH", "id": "db6cee8d55fe4a29", "name": "Nashua", "place_type": "city", "bounding_box": rectangle("-71.5749,42.700085 -71.433334,42.805852") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33011, "countyName": "Hillsborough", "cityID": 3350260, "cityName": "Nashua" } }
+{ "create_at": datetime("2016-01-03T10:00:20.000Z"), "id": 683709495101538304, "text": "Nice to see Fenway Park again. Driving by", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2369687699, "name": "Damony", "screen_name": "CrowellDamony", "lang": "en", "location": "null", "create_at": date("2014-02-27"), "description": "D$Money/ PHS Senior, CLASS OF 2016/ Basketball/ Fan of NIKE+JORDAN+KDs/ 18 yrs old/ #BallisLife/ (Avocado = Amazing person)", "followers_count": 86, "friends_count": 153, "statues_count": 5171 }, "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-03T10:00:21.000Z"), "id": 683709495395008512, "text": "sorry im not on this god forsaken website every second of the day https://t.co/aGpmbrYgfb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 546319636, "name": "mac lé more", "screen_name": "hotironedpenis", "lang": "en", "location": "JM JC", "create_at": date("2012-04-05"), "description": "sometimes i do the art, ever heard of it? probably not.", "followers_count": 390, "friends_count": 276, "statues_count": 3317 }, "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-03T10:00:21.000Z"), "id": 683709495600611329, "text": "@_PrettyRiot the back not that bright it's just the front", "in_reply_to_status": 683709371222638592, "in_reply_to_user": 71281160, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 71281160 }}, "user": { "id": 236218853, "name": "Gab Gab.", "screen_name": "Gabb_ayeeeeee", "lang": "en", "location": "Chicago ❤️", "create_at": date("2011-01-09"), "description": "...grown as fuck so idgaf / Freshman at #UIC19 / Need bundles? I'm the plug", "followers_count": 1193, "friends_count": 798, "statues_count": 60571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Matteson, IL", "id": "5bdca5ea70b98771", "name": "Matteson", "place_type": "city", "bounding_box": rectangle("-87.79136,41.491202 -87.690665,41.528476") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1747540, "cityName": "Matteson" } }
+{ "create_at": datetime("2016-01-03T10:00:21.000Z"), "id": 683709495634165760, "text": "Girls just be liking to fight because its they think it'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": 2463691261, "name": "Goodnews™", "screen_name": "yb_friends", "lang": "en", "location": "Maryland, USA", "create_at": date("2014-04-25"), "description": "Baltimore|LK☁️|#tracknation", "followers_count": 7030, "friends_count": 3925, "statues_count": 8476 }, "place": { "country": "United States", "country_code": "United States", "full_name": "White Marsh, MD", "id": "3d15925dbe48f7e6", "name": "White Marsh", "place_type": "city", "bounding_box": rectangle("-76.500885,39.359887 -76.389052,39.415545") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2484350, "cityName": "White Marsh" } }
+{ "create_at": datetime("2016-01-03T10:00:21.000Z"), "id": 683709495705485313, "text": "\"And now you know....the rest of the story!\" �������� https://t.co/tCwdGiPvbD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 105560657, "name": "Ruben D. Sanchez Jr", "screen_name": "rdsanchezjr", "lang": "en", "location": "✈️Chicago✈️NYC✈️Los Angeles✈️", "create_at": date("2010-01-16"), "description": "US Army National Guard/ @USC alum I may be in your town!", "followers_count": 6922, "friends_count": 6474, "statues_count": 52230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "SOHO, Manhattan", "id": "2597f2b91a5fcdb1", "name": "SOHO", "place_type": "neighborhood", "bounding_box": rectangle("-74.011573,40.718004 -73.995378,40.729087") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-03T10:00:21.000Z"), "id": 683709495797792770, "text": "#RT #GlamByAmanda visit https://t.co/TyOnFTHqdy @_ErinRachel @_MorganBrittany https://t.co/nKvzhZ1LDW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RT", "GlamByAmanda" }}, "user_mentions": {{ 45168886, 61828060 }}, "user": { "id": 129937618, "name": "Emile Lee III", "screen_name": "EmileLee3", "lang": "en", "location": "Laplace, LA", "create_at": date("2010-04-05"), "description": "President & CEO of IMS a Certified DBE Construction Company. Specializing in Hotels, Apartments, Condominiums and Assisted Living Projects.", "followers_count": 166019, "friends_count": 11, "statues_count": 21072 }, "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-01-03T10:00:21.000Z"), "id": 683709495856508928, "text": "So excited to go up to FSU @HadleyLane1 ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 848663552 }}, "user": { "id": 3234003545, "name": "linds", "screen_name": "Rhyses_piecez", "lang": "en", "location": "kappa delta @ quinnipiac ", "create_at": date("2015-05-04"), "description": "i like pizza rolls & dom perignon", "followers_count": 200, "friends_count": 261, "statues_count": 1275 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tequesta, FL", "id": "e4ea9dc0fe7ff9fd", "name": "Tequesta", "place_type": "city", "bounding_box": rectangle("-80.140709,26.951196 -80.074634,27.000828") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1271525, "cityName": "Tequesta" } }
+{ "create_at": datetime("2016-01-03T10:00:21.000Z"), "id": 683709495978123265, "text": "@__danielllle https://t.co/NHi6OF44yU", "in_reply_to_status": 683709388029300736, "in_reply_to_user": 357944642, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 357944642 }}, "user": { "id": 65945072, "name": "Darrick Mills", "screen_name": "Hoop4life_Mills", "lang": "en", "location": "null", "create_at": date("2009-08-15"), "description": "Snap Chat me : Bigsexy-nigga Kik: Dmills102. The devil stay testing cause if you chase the pussy it's a sin but if it falls in your lap it's a blessing CHICAGO!", "followers_count": 4055, "friends_count": 3070, "statues_count": 21140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burr Ridge, IL", "id": "0e0b9a037618a313", "name": "Burr Ridge", "place_type": "city", "bounding_box": rectangle("-87.958775,41.701335 -87.89258,41.789783") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1709980, "cityName": "Burr Ridge" } }
+{ "create_at": datetime("2016-01-03T10:00:21.000Z"), "id": 683709495986401280, "text": "#MuzikkZone\n.@Sammi_Gemini @Princess3Rina @TheCinemafanVip @MREESE06 @sweetsexy_39 @PATOO_S @MuzikkZone @MuzikkZone_UK @SaphyreRain", "in_reply_to_status": 683683408707514368, "in_reply_to_user": 200694474, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "MuzikkZone" }}, "user_mentions": {{ 200694474, 1677894596, 271851224, 177506241, 713354671, 232905913, 580223886, 1848442092, 3275185693 }}, "user": { "id": 532876187, "name": "Wolfx #MuzikkZone LM", "screen_name": "Cordle2012", "lang": "en", "location": "Worldwide-But Still Expanding", "create_at": date("2012-03-21"), "description": "@NancySinatra @yesofficial @foreverstilldk @DespiteOfficial @iamjebe @avamaha @DIAMANTEband @Dashni_Morad @laurawilde @AnimaInside @MotionVDevice @NmirzHanns", "followers_count": 52102, "friends_count": 52123, "statues_count": 98569 }, "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-03T10:00:21.000Z"), "id": 683709496779223040, "text": "I think it got when Austin mad I like when he mad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2438884929, "name": "Carolynb", "screen_name": "Cbaustinmahone", "lang": "en", "location": "Baltimore Md ", "create_at": date("2014-03-26"), "description": "null", "followers_count": 446, "friends_count": 781, "statues_count": 9988 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lutherville, MD", "id": "01fa921708f70923", "name": "Lutherville", "place_type": "city", "bounding_box": rectangle("-76.646721,39.412912 -76.566297,39.466288") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2448875, "cityName": "Lutherville" } }
+{ "create_at": datetime("2016-01-03T10:00:21.000Z"), "id": 683709497001521152, "text": "Last time I saw a place this packed The Young Bucks were wrestling #HillaryClinton @NickJacksonYB @MattJackson13 https://t.co/2rpuGTgtAS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HillaryClinton" }}, "user_mentions": {{ 61691554, 33055971 }}, "user": { "id": 19179514, "name": "Jon Pine", "screen_name": "Jonpine24", "lang": "en", "location": "Providence, Rhode Island", "create_at": date("2009-01-19"), "description": "Quinnipiac University '15\nPolitical junkie, Rhode Islander, Retired High School Tennis Player\nHillary for New Hampshire\n#FITN\nRT≠endorsement", "followers_count": 456, "friends_count": 559, "statues_count": 21371 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Derry, NH", "id": "01fa7dbf73290a3b", "name": "Derry", "place_type": "city", "bounding_box": rectangle("-71.346646,42.840671 -71.206356,42.9418") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33015, "countyName": "Rockingham", "cityID": 3317860, "cityName": "Derry" } }
+{ "create_at": datetime("2016-01-03T10:00:21.000Z"), "id": 683709497030873088, "text": "Lol work https://t.co/zC2UpG2VY9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 38758010, "name": "Amanda Hostetler", "screen_name": "amalynnn", "lang": "en", "location": "Wadsworth, OH", "create_at": date("2009-05-08"), "description": "I'm female football fanatic and long-suffering CLE sports fan // IG: amandahostetler // #Browns #Buckeyes #Indians #Cavs ♡", "followers_count": 497, "friends_count": 327, "statues_count": 6303 }, "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-01-03T10:00:21.000Z"), "id": 683709497336971264, "text": "@asiaanrapunzel ����", "in_reply_to_status": 683709366051061761, "in_reply_to_user": 363274715, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 363274715 }}, "user": { "id": 3908729414, "name": "tay ❤️", "screen_name": "_Tay_Juliaaa", "lang": "en", "location": "null", "create_at": date("2015-10-15"), "description": "I drive the Jetta with the missing gas cap", "followers_count": 257, "friends_count": 468, "statues_count": 224 }, "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-01-03T10:00:21.000Z"), "id": 683709497366429697, "text": "An ornate ivory sound-hole from an 18th century guitar. \n\nAs annoying it might seem to other… https://t.co/Ehw4EZufam", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.09425492,42.33927783"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 518711065, "name": "Francis Flisiuk", "screen_name": "FrancisFlisiuk", "lang": "en", "location": "Portland, Maine", "create_at": date("2012-03-08"), "description": "Storyteller @PortlandPhoenix. Media Curator. Photographer. @VICE News Junkie. Podcast addict. PS4 Gamer. Fellow journalists get an instant follow back!", "followers_count": 27158, "friends_count": 4185, "statues_count": 6246 }, "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-03T10:00:21.000Z"), "id": 683709497655689216, "text": "@aggiebrett I'm sure if you looked at anti-war, anti-capitalist & other liberal causes during Bush years you'd see similar correlation", "in_reply_to_status": 683709229761351680, "in_reply_to_user": 2792512567, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14124991 }}, "user": { "id": 2792512567, "name": "Old Army Fight", "screen_name": "OldArmyFight", "lang": "en", "location": "Aggieland", "create_at": date("2014-09-05"), "description": "Old Army fight, New Army shenanigans #Stitt4OC #StittHappens", "followers_count": 432, "friends_count": 1015, "statues_count": 5778 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-03T10:00:21.000Z"), "id": 683709497903288322, "text": "E\nA\nG\nL\nAhhh fuck it nevermind", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 268372911, "name": "BEETLE BAYLEY", "screen_name": "beetlebayley219", "lang": "en", "location": "Linwood Pa.", "create_at": date("2011-03-18"), "description": "philly sports is my passion #Phillies #Eagles #Sixers #Flyers", "followers_count": 988, "friends_count": 2406, "statues_count": 26510 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Linwood, PA", "id": "815620d0c6b20d09", "name": "Linwood", "place_type": "city", "bounding_box": rectangle("-75.434927,39.817495 -75.414458,39.831297") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42045, "countyName": "Delaware", "cityID": 4243720, "cityName": "Linwood" } }
+{ "create_at": datetime("2016-01-03T10:00:21.000Z"), "id": 683709497915777024, "text": "@Yakboy600 Nbs ion even look they way", "in_reply_to_status": 683626541234958336, "in_reply_to_user": 4691627136, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4691627136 }}, "user": { "id": 1571632549, "name": "Playa Man Zmac", "screen_name": "zmac__", "lang": "en", "location": "#LamarUniv. #BlackUniversity ", "create_at": date("2013-07-05"), "description": "#TYP", "followers_count": 698, "friends_count": 502, "statues_count": 10050 }, "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-01-03T10:00:21.000Z"), "id": 683709497982861312, "text": "The perfect Sunday....the Green Goddess, the coffee, and the planning space.... @ New York, New… https://t.co/Pzk7D6hsTH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0064,40.7142"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 39391231, "name": "Kelly Spaulding", "screen_name": "kellyspaulding", "lang": "en", "location": "Raleigh, North Carolina", "create_at": date("2009-05-11"), "description": "Sales pro w/ a passion for techology & social selling. Avid hot yogi (@IndigoHotYoga), Trying to connect my family through http://spauldingfamily.com", "followers_count": 3994, "friends_count": 4439, "statues_count": 19209 }, "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-03T10:00:21.000Z"), "id": 683709498280681472, "text": "We're baaacccckk #SEC https://t.co/4kSbw6qMmR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SEC" }}, "user": { "id": 1123839613, "name": "LSU Grad Girl", "screen_name": "SerenaCrawford_", "lang": "en", "location": "Proverbs 4:23", "create_at": date("2013-01-26"), "description": "UNO Alumna||ZTA Alum|| LSU||TigerTV", "followers_count": 415, "friends_count": 471, "statues_count": 9456 }, "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-03T10:00:21.000Z"), "id": 683709498314326016, "text": "Everything About You Is Apart 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": 3035136853, "name": "##QueennJay##❤️", "screen_name": "TheyenvyJada", "lang": "en", "location": "null", "create_at": date("2015-02-21"), "description": "Holding My Own ❤️", "followers_count": 187, "friends_count": 505, "statues_count": 5972 }, "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-03T10:00:21.000Z"), "id": 683709498519846912, "text": "Hungry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3102255874, "name": "Katsandra", "screen_name": "KATsandrajayne", "lang": "en", "location": "Zanesville, OH", "create_at": date("2015-03-21"), "description": "snapchat & instagram @kassandrajayne", "followers_count": 569, "friends_count": 548, "statues_count": 6630 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Zanesville, OH", "id": "f4a27b9e3d616757", "name": "Zanesville", "place_type": "city", "bounding_box": rectangle("-82.121534,39.908312 -81.976901,39.999527") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39119, "countyName": "Muskingum", "cityID": 3988084, "cityName": "Zanesville" } }
+{ "create_at": datetime("2016-01-03T10:00:21.000Z"), "id": 683709498746339328, "text": "انت أحلى شي صار في 2015 ��❤ https://t.co/R7bvzWADp2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 568345556, "name": "Heبa ♡", "screen_name": "HebaElissian", "lang": "en", "location": "Florida, USA", "create_at": date("2012-05-01"), "description": "null", "followers_count": 17804, "friends_count": 184, "statues_count": 61396 }, "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-01-03T10:00:21.000Z"), "id": 683709498758836224, "text": "Hot chocolate, but I can't drink it . https://t.co/AgBcKN2HvR", "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": 728, "friends_count": 89, "statues_count": 18675 }, "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-03T10:00:21.000Z"), "id": 683709498997915648, "text": "Join the Tractor Supply Company team! See our latest #Retail #job opening here: https://t.co/BqIysvntyu #Pensacola, FL #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.2777793,30.5405535"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "Pensacola", "Hiring", "CareerArc" }}, "user": { "id": 59695864, "name": "TMJ-FL Retail Jobs", "screen_name": "tmj_fl_retail", "lang": "en", "location": "Florida", "create_at": date("2009-07-23"), "description": "Follow this account for geo-targeted Retail job tweets in Florida Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 475, "friends_count": 306, "statues_count": 6390 }, "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-03T10:00:21.000Z"), "id": 683709499027275777, "text": "A beautiful day in the city #aussieinnyc #shieldskids #iloveny @… https://t.co/UfUQWCQVXg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.96906,40.779447"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "aussieinnyc", "shieldskids", "iloveny" }}, "user": { "id": 261800602, "name": "Carmen Shields", "screen_name": "MissCS84", "lang": "en", "location": "New York City", "create_at": date("2011-03-06"), "description": "null", "followers_count": 34, "friends_count": 103, "statues_count": 315 }, "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-03T10:00:21.000Z"), "id": 683709499132227585, "text": "@kylemartinez410 @FreaKy_SeRg22 @ClaySmitt @Mike_Denton03 if more people go I will", "in_reply_to_status": 683709238389137408, "in_reply_to_user": 2614177232, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2614177232, 489802075, 492719650, 497544294 }}, "user": { "id": 365845736, "name": "Collin Stasch", "screen_name": "C_rawww24", "lang": "en", "location": "null", "create_at": date("2011-08-31"), "description": "Nhs 2016 #Belmontabbeylax2020 #Resteasychevy", "followers_count": 715, "friends_count": 386, "statues_count": 12983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riviera Beach, MD", "id": "d73113961dca5386", "name": "Riviera Beach", "place_type": "city", "bounding_box": rectangle("-76.530821,39.138187 -76.500351,39.172629") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24003, "countyName": "Anne Arundel", "cityID": 2466850, "cityName": "Riviera Beach" } }
+{ "create_at": datetime("2016-01-03T10:00:21.000Z"), "id": 683709499174080513, "text": "Not physically or mentally prepared for 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": 1340166547, "name": "Allie Schoenly", "screen_name": "AllieSchoenly", "lang": "en", "location": "null", "create_at": date("2013-04-09"), "description": "EHS '17 | Proverbs 17:22", "followers_count": 637, "friends_count": 512, "statues_count": 4422 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Emmaus, PA", "id": "f171e3489b3af856", "name": "Emmaus", "place_type": "city", "bounding_box": rectangle("-75.520168,40.517734 -75.479534,40.556371") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42077, "countyName": "Lehigh", "cityID": 4223584, "cityName": "Emmaus" } }
+{ "create_at": datetime("2016-01-03T10:00:22.000Z"), "id": 683709499622866944, "text": "A little snow in #Pearl #Portland #PDXWX isn't stopping our #hummingbird friends this morning. https://t.co/Jfq1qL4baM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Pearl", "Portland", "PDXWX", "hummingbird" }}, "user": { "id": 66799541, "name": "Val Stouffer", "screen_name": "valstouffer", "lang": "en", "location": "Portland, OR", "create_at": date("2009-08-18"), "description": "Radio Chick - currently anchoring Portland's Afternoon News on KXL Radio 4p-7p.", "followers_count": 46, "friends_count": 163, "statues_count": 75 }, "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-03T10:00:22.000Z"), "id": 683709500071612416, "text": "Join the Advantage Solutions team! See our latest #Sales #job opening here: https://t.co/STHmAIAlId #Atoka, TN #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.7781379,35.4411945"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "job", "Atoka", "Hiring", "CareerArc" }}, "user": { "id": 22003362, "name": "TMJ-TN Sales Jobs", "screen_name": "tmj_tn_sales", "lang": "en", "location": "Tennessee", "create_at": date("2009-02-26"), "description": "Follow this account for geo-targeted Sales job tweets in Tennessee Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 367, "friends_count": 298, "statues_count": 203 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atoka, TN", "id": "014fa6605f6fcfaf", "name": "Atoka", "place_type": "city", "bounding_box": rectangle("-89.855206,35.390621 -89.74451,35.466582") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47167, "countyName": "Tipton", "cityID": 4702340, "cityName": "Atoka" } }
+{ "create_at": datetime("2016-01-03T10:00:22.000Z"), "id": 683709500298100737, "text": "everybody was like latte get up & I'm like naw ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 999778022, "name": "5.^", "screen_name": "Queensmokess", "lang": "en", "location": "Nawf dallas", "create_at": date("2012-12-09"), "description": "19", "followers_count": 4246, "friends_count": 208, "statues_count": 112274 }, "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-03T10:00:22.000Z"), "id": 683709500336005120, "text": "Say what you want about them but they have the best anthem by far @… https://t.co/z5IIMmd8Ek", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.61662242,41.86209486"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 184420853, "name": "Marc Bircham", "screen_name": "marcbircham", "lang": "en", "location": "null", "create_at": date("2010-08-29"), "description": "Ex player and enjoys his family football and having a cheeky few drinks now and again and these are my thoughts and noone else", "followers_count": 18243, "friends_count": 368, "statues_count": 6731 }, "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-03T10:00:22.000Z"), "id": 683709500348608512, "text": "#AssistFireDept at 6901-6998 Aloma Ave, Winter Park, FL 32792. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2960411,28.606959"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AssistFireDept", "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": 95, "friends_count": 1, "statues_count": 3355 }, "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-01-03T10:00:22.000Z"), "id": 683709500512141314, "text": "Remember the Alamo!!! The San Antonio River Walk is beautiful. I'm gonna miss these guys!… https://t.co/tpflqU57xw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.48611111,29.42583333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 313638456, "name": "Evan Craft", "screen_name": "EvanCraft", "lang": "en", "location": "null", "create_at": date("2011-06-08"), "description": "Este es mi viaje, estoy reclamando lo que Dios ya me ha dado. Puedes encontrar #PrincipioYFin en iTunes/Spotify!!", "followers_count": 48713, "friends_count": 538, "statues_count": 4657 }, "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-03T10:00:22.000Z"), "id": 683709500566683649, "text": "@Gabyalvzz you won't", "in_reply_to_status": 683694993089916933, "in_reply_to_user": 1115643918, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1115643918 }}, "user": { "id": 489139689, "name": "J.び", "screen_name": "_julian09", "lang": "en", "location": "null", "create_at": date("2012-02-10"), "description": "don't worry bout me", "followers_count": 724, "friends_count": 412, "statues_count": 22519 }, "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-03T10:00:22.000Z"), "id": 683709500612804610, "text": "I'm so sad winter break 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": 3337077376, "name": "Vicky", "screen_name": "vict0riaamariaa", "lang": "en", "location": "Global ", "create_at": date("2015-06-20"), "description": "Not a worry in the world", "followers_count": 592, "friends_count": 379, "statues_count": 2159 }, "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-01-03T10:00:22.000Z"), "id": 683709500851793921, "text": "Temp 39.8°F Rising, Pressure 30.260in Rising slowly, Dew point 27.0°, Wind NNW 4mph, Rain today 0.00in https://t.co/gPtnr2dDtV #STLwx #MOwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.35555556,38.47694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "STLwx", "MOwx" }}, "user": { "id": 1191304230, "name": "Oakville Weather", "screen_name": "OakvilleWx", "lang": "en", "location": "Oakville, Missouri, USA", "create_at": date("2013-02-17"), "description": "Current conditions on the hour brought to you by @DaveStLou", "followers_count": 103, "friends_count": 80, "statues_count": 24569 }, "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-01-03T10:00:22.000Z"), "id": 683709500893745152, "text": "Mostly sunny this afternoon, high 50 (10 C). Low 29 (-2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119697197, "name": "Huntsville Weather", "screen_name": "HuntsvilleWX", "lang": "en", "location": "null", "create_at": date("2010-03-04"), "description": "null", "followers_count": 668, "friends_count": 65, "statues_count": 6053 }, "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-03T10:00:22.000Z"), "id": 683709501036474368, "text": "How is @camerondallas so flawless ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 405728790 }}, "user": { "id": 3047833645, "name": "Rachael Cyphert ღ", "screen_name": "20Walruses", "lang": "en", "location": "FL", "create_at": date("2015-02-27"), "description": "big fans of 5sosfam, lovatics, Shawn Mendes, Cameron Dallas. that's is all bye :)", "followers_count": 66, "friends_count": 191, "statues_count": 344 }, "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-01-03T10:00:22.000Z"), "id": 683709501325873152, "text": "She said I'm losing my best feature. ���� oh hell no", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1219896097, "name": "ⓖⓡⓔⓖⓞⓡⓨ", "screen_name": "DuceIsBack_", "lang": "en", "location": "null", "create_at": date("2013-02-25"), "description": "The Takeover ⌚✈ | #BeGreat |\nNavy | #NLMB", "followers_count": 751, "friends_count": 676, "statues_count": 13818 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaver Falls, PA", "id": "579173dcb66e54d5", "name": "Beaver Falls", "place_type": "city", "bounding_box": rectangle("-80.335207,40.739691 -80.309549,40.790082") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42007, "countyName": "Beaver", "cityID": 4204792, "cityName": "Beaver Falls" } }
+{ "create_at": datetime("2016-01-03T10:00:22.000Z"), "id": 683709501481041929, "text": "#SuspiciousPerson at 4000-4199 Pine Hills Cir, Pine Hills, FL 32808. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4528745,28.5940523"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SuspiciousPerson", "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": 213, "friends_count": 1, "statues_count": 75303 }, "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-03T10:00:22.000Z"), "id": 683709501535576064, "text": "HUGE s/o to @emilywilt7 for finally completing her life goal and beating a wong����!!! maybe we don't need the plan after all...��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2796741877 }}, "user": { "id": 2396762786, "name": "madison", "screen_name": "madisonburkh", "lang": "en", "location": "null", "create_at": date("2014-03-18"), "description": "|-/ twenty øne piløts", "followers_count": 223, "friends_count": 386, "statues_count": 295 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington Heights, IL", "id": "cdf74b55fc0d4600", "name": "Arlington Heights", "place_type": "city", "bounding_box": rectangle("-88.026334,42.028374 -87.944618,42.162938") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1702154, "cityName": "Arlington Heights" } }
+{ "create_at": datetime("2016-01-03T10:00:22.000Z"), "id": 683709501573345280, "text": "I'm at Meijer in Lansing, MI https://t.co/HRwG6SFBTR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.54030991,42.66796894"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 58253068, "name": "The Xmas Spinnet", "screen_name": "PianoMarian", "lang": "en", "location": "Holt, MI USA", "create_at": date("2009-07-19"), "description": "A lil blind piano/keyboard playin' baseball-lovin' sut'm-or-other who could tweet about NEthing!", "followers_count": 140, "friends_count": 331, "statues_count": 4406 }, "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-03T10:00:22.000Z"), "id": 683709501854330880, "text": "you're welcome �� https://t.co/7PbGfGbLmv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3432090785, "name": "Deergod aka Jordan", "screen_name": "deergod20", "lang": "en", "location": "Ansonia, CT", "create_at": date("2015-08-19"), "description": "Ansonia High Varsity Soccer #20 // heavy metal // @FRONZ1LLA owns me //@Taylorcoffeyy has my heart", "followers_count": 232, "friends_count": 355, "statues_count": 5746 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ansonia, CT", "id": "b04794c3445e78cf", "name": "Ansonia", "place_type": "city", "bounding_box": rectangle("-73.102383,41.32376 -73.03819,41.363148") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 901150, "cityName": "Ansonia" } }
+{ "create_at": datetime("2016-01-03T10:00:22.000Z"), "id": 683709502194106375, "text": "all i want to do is travel 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": 2676442024, "name": "kayli", "screen_name": "tta_kay", "lang": "en", "location": "null", "create_at": date("2014-07-05"), "description": "tumble tech// elhs", "followers_count": 232, "friends_count": 268, "statues_count": 1425 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oldsmar, FL", "id": "61eefa163263bef0", "name": "Oldsmar", "place_type": "city", "bounding_box": rectangle("-82.710115,27.999141 -82.648349,28.066154") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1251350, "cityName": "Oldsmar" } }
+{ "create_at": datetime("2016-01-03T10:00:22.000Z"), "id": 683709502416273409, "text": "@peterkarlberg @aClassicLiberal \"provide many insights to the ongoing debate on austerity versus stimulus\"", "in_reply_to_status": 683708870125056006, "in_reply_to_user": 16088282, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16088282, 103375095 }}, "user": { "id": 799437895, "name": "Jordan Rice", "screen_name": "1jrice", "lang": "en", "location": "PDX (now) SEA (6yrs) NJ (from)", "create_at": date("2012-09-02"), "description": "Director Digital Sensing Innovation @Nike | ex-@Quanttus & ex-@Synapse | Wearable Technologist since before it was cool | Views mine | Sports geek: #EBFG #Cubs", "followers_count": 603, "friends_count": 866, "statues_count": 5823 }, "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-03T10:00:22.000Z"), "id": 683709502521237504, "text": "I another year I got to get thru without my baby ���������� RIP Kanah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 313639148, "name": "Ca. ❤️", "screen_name": "caaparker_", "lang": "en", "location": "null", "create_at": date("2011-06-08"), "description": "Class Of 2016 | Grinding For Me , Myself , & I | My Motivation Is My Cousins , RIP Shakanah J China & Thomas George Docher Jr | June 18th 2016!", "followers_count": 1734, "friends_count": 1286, "statues_count": 75015 }, "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-03T10:00:22.000Z"), "id": 683709502810632192, "text": "@jonathanhsinger @JoyAnnReid @jjmacnab\nThe term militia is completely inappropriate. This is an armed mob defying federal law. Disarm them.", "in_reply_to_status": 683707235801759746, "in_reply_to_user": 12811282, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 12811282, 49698134, 60610778 }}, "user": { "id": 230021244, "name": "Joe MacKrell", "screen_name": "sleepyjoe2", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2010-12-23"), "description": "null", "followers_count": 189, "friends_count": 828, "statues_count": 12096 }, "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-03T10:00:22.000Z"), "id": 683709502861000704, "text": "@WORLDSTARVlNE @nicoleschnnell you when we do that bean boozled challenge", "in_reply_to_status": 667483848712695808, "in_reply_to_user": 1204567627, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1204567627, 556467437 }}, "user": { "id": 394357575, "name": "Alex Corleone", "screen_name": "Papa_Gonzo", "lang": "en", "location": "City Island ⚓ New York", "create_at": date("2011-10-19"), "description": "NEW ROCHELLE FOOTBALL #FTS #NSG RIP Michael Mollin❤ Get excited about the process; the outcomes will take care of themselves - Coach Lou DiRienzo", "followers_count": 514, "friends_count": 326, "statues_count": 13064 }, "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-03T10:00:22.000Z"), "id": 683709502919696384, "text": "MINI-#SkylandsCyclingClub Weekend Ride @ Greene's Beans Cafe https://t.co/Ce6ndkEWWS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.635419,41.030754"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SkylandsCyclingClub" }}, "user": { "id": 1077536018, "name": "I Am Architect Ninja", "screen_name": "ArchitectNinja", "lang": "en", "location": "Steath Mode", "create_at": date("2013-01-10"), "description": "null", "followers_count": 893, "friends_count": 887, "statues_count": 5873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Mohawk, NJ", "id": "9d4974adf93654f4", "name": "Lake Mohawk", "place_type": "city", "bounding_box": rectangle("-74.726141,40.985186 -74.631213,41.058035") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34037, "countyName": "Sussex", "cityID": 3438040, "cityName": "Lake Mohawk" } }
+{ "create_at": datetime("2016-01-03T10:00:22.000Z"), "id": 683709503024553985, "text": "I hate seeing my puppy in so much pain ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 625279700, "name": "Teagan Ready", "screen_name": "TeaganReady", "lang": "en", "location": "null", "create_at": date("2012-07-02"), "description": "null", "followers_count": 292, "friends_count": 363, "statues_count": 2408 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spring Grove, IL", "id": "00adf35b11bba302", "name": "Spring Grove", "place_type": "city", "bounding_box": rectangle("-88.30152,42.415601 -88.198652,42.495082") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17111, "countyName": "McHenry", "cityID": 1772052, "cityName": "Spring Grove" } }
+{ "create_at": datetime("2016-01-03T10:00:22.000Z"), "id": 683709503137824769, "text": "@TakeThatHistory \nof course if u deny God, u deny moral giver, and atheism becomes a lot to do with morality https://t.co/n0QjaMDpSg", "in_reply_to_status": 683708774813724672, "in_reply_to_user": 3050123116, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3050123116 }}, "user": { "id": 3548142262, "name": "v. HELENO HENZO", "screen_name": "THISSBRAVA", "lang": "en", "location": "DJABRABA", "create_at": date("2015-09-04"), "description": "CHRISTIAN APOLOGIST. IF YOU ARE ON TWITTER, YOU ARE THEIST, ONLY INTELLIGENTLY CREATED BEING CAN TYPE AND READ", "followers_count": 482, "friends_count": 3964, "statues_count": 5708 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Smith Mills, MA", "id": "db0dcea533e078a2", "name": "Smith Mills", "place_type": "city", "bounding_box": rectangle("-71.047745,41.619498 -70.958622,41.680953") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25005, "countyName": "Bristol", "cityID": 2562300, "cityName": "Smith Mills" } }
+{ "create_at": datetime("2016-01-03T10:00:22.000Z"), "id": 683709503347425280, "text": "See our latest #BatonRouge, LA #job and click to apply: Retail Store Positions - https://t.co/Qy2Crcwc9I #Retail #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-91.0864763,30.4545545"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BatonRouge", "job", "Retail", "Hiring", "CareerArc" }}, "user": { "id": 59950484, "name": "TMJ-LA Retail Jobs", "screen_name": "tmj_la_retail", "lang": "en", "location": "Louisiana", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in Louisiana Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 402, "friends_count": 301, "statues_count": 1008 }, "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-03T10:00:22.000Z"), "id": 683709503368335360, "text": "@Jaileannnn I mean why would I not be up at 5:30 on a sunday�� Work loves me.", "in_reply_to_status": 683657870915731456, "in_reply_to_user": 712198262, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 712198262 }}, "user": { "id": 950860501, "name": "Heather.", "screen_name": "aliencourage", "lang": "en", "location": "California, USA", "create_at": date("2012-11-15"), "description": "What do you call a sphere full of idiots •Earth•", "followers_count": 302, "friends_count": 277, "statues_count": 1249 }, "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-03T10:00:22.000Z"), "id": 683709503372570625, "text": "Partly cloudy this afternoon, high 52 (11 C). Low 30 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119698223, "name": "Birmingham Weather", "screen_name": "BirminghamWX", "lang": "en", "location": "Birmingham, AL", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Birmingham, AL", "followers_count": 503, "friends_count": 68, "statues_count": 6065 }, "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-03T10:00:23.000Z"), "id": 683709503779377152, "text": "women gone have to start settling or be single forever, cause the days of boys actually becoming men and taking care of their woman is dying", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.747324,38.81699"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 194573948, "name": "Bobby Lee Swagger", "screen_name": "SirrBanks", "lang": "en", "location": "Lauderdale ", "create_at": date("2010-09-24"), "description": "father. soldier. husband.", "followers_count": 1115, "friends_count": 546, "statues_count": 121806 }, "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-03T10:00:23.000Z"), "id": 683709503779508224, "text": "I can see me going there just for cream spinach", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 130368672, "name": "K.I.L.L.A.", "screen_name": "K_Dot_Aleesha", "lang": "en", "location": "null", "create_at": date("2010-04-06"), "description": "null", "followers_count": 1132, "friends_count": 862, "statues_count": 104411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodlawn, MD", "id": "bf64a5dd9b5f1cf2", "name": "Woodlawn", "place_type": "city", "bounding_box": rectangle("-76.915396,38.938828 -76.711092,39.327477") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2486525, "cityName": "Woodlawn" } }
+{ "create_at": datetime("2016-01-03T10:00:23.000Z"), "id": 683709503783604224, "text": "I have an idea, let's watch all the High School Musicals", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1947301040, "name": "Mandito", "screen_name": "manditodude", "lang": "en", "location": "null", "create_at": date("2013-10-08"), "description": "Love your life and everyone in it.", "followers_count": 465, "friends_count": 379, "statues_count": 13436 }, "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-03T10:00:23.000Z"), "id": 683709504186388480, "text": "Reports out #lions plan to retain Jim Caldwell , but final decision will be with be new GM? #backwards", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "lions", "backwards" }}, "user": { "id": 22891320, "name": "Dan Leach", "screen_name": "dtmleach", "lang": "en", "location": "Farmington Hills, Michigan", "create_at": date("2009-03-04"), "description": "97.1 The Ticket. Voice of the Detroit Titans. Huge heart, undying passion for sports and life. Believes in the dream always:)", "followers_count": 8373, "friends_count": 2379, "statues_count": 25333 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Farmington Hills, MI", "id": "0b40afc99807b6ac", "name": "Farmington Hills", "place_type": "city", "bounding_box": rectangle("-83.437523,42.439001 -83.316839,42.529556") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2627440, "cityName": "Farmington Hills" } }
+{ "create_at": datetime("2016-01-03T10:00:23.000Z"), "id": 683709504194658308, "text": "@DJMervSmooth @TheShadeRoom that's why her neckass got lupus now. God don't like ugly.", "in_reply_to_status": 683708848708960256, "in_reply_to_user": 2564460612, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2564460612, 2426183605 }}, "user": { "id": 236156697, "name": "Kenny", "screen_name": "KennyWitTheKoke", "lang": "en", "location": "318225", "create_at": date("2011-01-09"), "description": "Excuse the Rant | The Last Cocaine Super Hero | #EnginEars | Inquirie$ : excusetherant@gmail.com |Come in my mentions sideways you getting Mutombo'd", "followers_count": 628, "friends_count": 311, "statues_count": 87534 }, "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-01-03T10:00:23.000Z"), "id": 683709504224104448, "text": "Public Service Announcement: He is not available. �� https://t.co/aRgDqqVd7d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1651146714, "name": "Lauren Lund", "screen_name": "LaurenBuLund", "lang": "en", "location": "Holland to Columbus", "create_at": date("2013-08-06"), "description": "All for God. Wings Model. GCT Model.", "followers_count": 671, "friends_count": 542, "statues_count": 8362 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chelyan, WV", "id": "013972e5177e9274", "name": "Chelyan", "place_type": "city", "bounding_box": rectangle("-81.504509,38.192553 -81.478532,38.199159") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54039, "countyName": "Kanawha", "cityID": 5414812, "cityName": "Chelyan" } }
+{ "create_at": datetime("2016-01-03T10:00:23.000Z"), "id": 683709504396001280, "text": "happs to this pretty girl!!!! hope you make the most of it!!! @laurenmay0103 https://t.co/FWTaFgZdBn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1621218704 }}, "user": { "id": 2263177410, "name": "fåith", "screen_name": "f_mcneill", "lang": "en", "location": "null", "create_at": date("2013-12-26"), "description": "Do all things with love", "followers_count": 436, "friends_count": 198, "statues_count": 698 }, "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-03T10:00:23.000Z"), "id": 683709504731623424, "text": "Super pissed I'm missing this game ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 264408714, "name": "Lexi❣", "screen_name": "lexi_rossey", "lang": "en", "location": "null", "create_at": date("2011-03-11"), "description": "♠️❤️ Rest in Peace Mommy ❤️ EUP ❤️ #FuckCancer", "followers_count": 1315, "friends_count": 991, "statues_count": 34431 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Homestead, PA", "id": "00f8dc268c71c619", "name": "Homestead", "place_type": "city", "bounding_box": rectangle("-79.919615,40.398579 -79.899802,40.415968") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4235424, "cityName": "Homestead" } }
+{ "create_at": datetime("2016-01-03T10:00:23.000Z"), "id": 683709504802959360, "text": "and to kick off the new year the BROWNS have beat the STEELERS and knocked them outta the playoffs #BrownsMania #PITvsCLE HAPPYNEWYEAR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BrownsMania", "PITvsCLE" }}, "user": { "id": 733620878, "name": "♛", "screen_name": "hannfiala", "lang": "en", "location": "CLE ", "create_at": date("2012-08-02"), "description": "12 | 13 | 15", "followers_count": 622, "friends_count": 1055, "statues_count": 27245 }, "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-03T10:00:23.000Z"), "id": 683709504920223744, "text": "Well the big diet starts tomorrow! #newyear", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.895268,30.458407"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "newyear" }}, "user": { "id": 1237373689, "name": "michael whitehead", "screen_name": "michael49752294", "lang": "en", "location": "Gulf Coast, Mississippi, USA", "create_at": date("2013-03-02"), "description": "Works at the Mississippi Dept. of Health. Believer. Married to Michele, father to Hayley and Hannah. Loves to sing!", "followers_count": 141, "friends_count": 191, "statues_count": 964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whataburger", "id": "07d9d27896c85004", "name": "Whataburger", "place_type": "poi", "bounding_box": rectangle("-88.8952681,30.4584069 -88.895268,30.458407") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28047, "countyName": "Harrison", "cityID": 2819180, "cityName": "D'Iberville" } }
+{ "create_at": datetime("2016-01-03T10:00:23.000Z"), "id": 683709505016758272, "text": "NOPE https://t.co/hRI2eoU6bE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2464970514, "name": "kelsey❥", "screen_name": "kels_maddi", "lang": "en", "location": "Kyle, TX", "create_at": date("2014-04-26"), "description": "i know things get hard, but girl you got it", "followers_count": 355, "friends_count": 190, "statues_count": 10079 }, "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-03T10:00:23.000Z"), "id": 683709505121693696, "text": "I've learned that being completely alone, is perfectly normal. It's normal to 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": 161016776, "name": "von. vondra.", "screen_name": "YourMajestyV_", "lang": "en", "location": "da ham.", "create_at": date("2010-06-29"), "description": "18 & Determined. | You either with me or against me. | ig: @YourMajestyV_", "followers_count": 3284, "friends_count": 2899, "statues_count": 57522 }, "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-01-03T10:00:23.000Z"), "id": 683709506170126336, "text": "Partly cloudy this afternoon, high 54 (12 C). Low 39 (4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119698549, "name": "Mobile Weather", "screen_name": "MobileWX", "lang": "en", "location": "Mobile, Al", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Mobile, Al", "followers_count": 403, "friends_count": 68, "statues_count": 6140 }, "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-03T10:00:23.000Z"), "id": 683709506438610945, "text": "Assistant Manager PT - dressbarn: (#Alabaster, AL) https://t.co/ase1vomGUJ #Retail #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.8163773,33.2442813"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Alabaster", "Retail", "Job", "Jobs", "Hiring" }}, "user": { "id": 2654767718, "name": "Jobs at Dressbarn", "screen_name": "dressbarnjobs", "lang": "en", "location": "null", "create_at": date("2014-07-17"), "description": "When you work at dressbarn, EVERY DAY is an opportunity to Inspire Women to Look and Feel Beautiful. Learn more @ http://dressbarncareers.com", "followers_count": 199, "friends_count": 15, "statues_count": 396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabaster, AL", "id": "8a6954309b085545", "name": "Alabaster", "place_type": "city", "bounding_box": rectangle("-86.878773,33.169625 -86.782514,33.271913") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1117, "countyName": "Shelby", "cityID": 100820, "cityName": "Alabaster" } }
+{ "create_at": datetime("2016-01-03T10:00:23.000Z"), "id": 683709506480521216, "text": "NYC is in good hands!!! @ Lox Lounge https://t.co/YDUziKCl6M", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9619827,40.6827965"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 45527036, "name": "WordSmiff", "screen_name": "_WordSmiff_", "lang": "en", "location": "NYC", "create_at": date("2009-06-08"), "description": "Wordsmiff on @AppleMusic.\nhttp://itun.es/us/u2CvF | Booking: HollywoodRomeTSL@gmail.com | Instagram: WordSmiffMusic\n#CanANiggaDream Coming March 2016", "followers_count": 7435, "friends_count": 2656, "statues_count": 71375 }, "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-03T10:00:23.000Z"), "id": 683709506782650368, "text": "Here at the Young Innovators Fair 2016 you can be part of the #SteamStars \n#projectsteamtv… https://t.co/9nNlTxij2v", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.45819961,40.12380216"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SteamStars", "projectsteamtv" }}, "user": { "id": 3089647052, "name": "Project S.T.E.A.M.TV", "screen_name": "ProjectSTEAMtv", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2015-03-16"), "description": "Media-based non profit supporting children in grades 6-12, encouraging them to know the importance of Science, Technology, Engineering, Art, & Mathematics.", "followers_count": 152, "friends_count": 306, "statues_count": 609 }, "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": 42029, "countyName": "Chester" } }
+{ "create_at": datetime("2016-01-03T10:00:23.000Z"), "id": 683709506816241666, "text": "@SkinnerLiber8ed You got it!", "in_reply_to_status": 683709339606081536, "in_reply_to_user": 262755165, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 262755165 }}, "user": { "id": 389589073, "name": "Stella*s PopKern", "screen_name": "stellaspopkern", "lang": "en", "location": "Washington, DC", "create_at": date("2011-10-12"), "description": "The country's first Gourmet Popcorn food truck. We're bringing the DC tri-state area innovative and creative flavors to delicious, NON-GMO popcorn!", "followers_count": 1986, "friends_count": 203, "statues_count": 4596 }, "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.022437") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2407125, "cityName": "Bethesda" } }
+{ "create_at": datetime("2016-01-03T10:00:23.000Z"), "id": 683709506992254976, "text": "Congrats Taylor! @tdank\n https://t.co/u9xLtDdWzh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 35295373 }}, "user": { "id": 2615757650, "name": "LA Mavs", "screen_name": "LA_Mavs", "lang": "en", "location": "Los Angeles ", "create_at": date("2014-07-10"), "description": "IG: @lamavs LA's Original Club - Since 2010 |HHH|", "followers_count": 222, "friends_count": 222, "statues_count": 545 }, "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-03T10:00:23.000Z"), "id": 683709507004952576, "text": "@archillect eee god make it stop", "in_reply_to_status": 683709431293566976, "in_reply_to_user": 2907774137, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2907774137 }}, "user": { "id": 1337194592, "name": "Ron Dahlgren", "screen_name": "ScaleItRon", "lang": "en", "location": "Mechanical Island", "create_at": date("2013-04-08"), "description": "Linux, D&D, SNES, PS4, JRPGs, SRPGs, Software, science news, pathetic attempts at humor. PGP - FB3E4542 | Not interested in activism | Svenska nybörjare", "followers_count": 229, "friends_count": 168, "statues_count": 7624 }, "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-01-03T10:00:23.000Z"), "id": 683709507034169346, "text": "https://t.co/YTPP7Vrlrl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 38110431, "name": "CrippleChris", "screen_name": "ReelNJShoreCris", "lang": "en", "location": "null", "create_at": date("2009-05-05"), "description": "null", "followers_count": 90, "friends_count": 415, "statues_count": 3325 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakhurst, NJ", "id": "329a8a8edd639e7d", "name": "Oakhurst", "place_type": "city", "bounding_box": rectangle("-74.065301,40.246324 -74.011188,40.274423") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3453790, "cityName": "Oakhurst" } }
+{ "create_at": datetime("2016-01-03T10:00:23.000Z"), "id": 683709507113910272, "text": "Yoga in DC! Will be in next weeks vlog ����don't forget to check out this weeks, link in bio �� @… https://t.co/ifJ2JcEuPS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.10222222,38.91111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 515513006, "name": "Ali", "screen_name": "alisonmacgomez", "lang": "en", "location": "null", "create_at": date("2012-03-05"), "description": "(910) I'm a lover, I'm a fighter, and I'm a cancer survivor.", "followers_count": 491, "friends_count": 115, "statues_count": 19225 }, "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-03T10:00:23.000Z"), "id": 683709507180990464, "text": "Join the Robert Half Finance & Accounting team! See our latest #InvestmentBanking #job opening here: https://t.co/7t7xU70eFv #Hampton, NH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.8389219,42.9375932"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "InvestmentBanking", "job", "Hampton" }}, "user": { "id": 207352965, "name": "NH Inv. Banking", "screen_name": "tmj_NH_invbank", "lang": "en", "location": "New Hampshire", "create_at": date("2010-10-24"), "description": "Follow this account for geo-targeted Investment Banking job tweets in New Hampshire Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 239, "friends_count": 235, "statues_count": 12 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hampton, NH", "id": "62a71dfc8f486745", "name": "Hampton", "place_type": "city", "bounding_box": rectangle("-70.877383,42.913667 -70.783882,42.960734") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33015, "countyName": "Rockingham", "cityID": 3332980, "cityName": "Hampton" } }
+{ "create_at": datetime("2016-01-03T10:00:23.000Z"), "id": 683709507218743296, "text": "@mixnmatch7 @BrandonBunnBun @DoryuuShepherd MINE!", "in_reply_to_status": 683709256219021312, "in_reply_to_user": 853403167, "favorite_count": 0, "retweet_count": 0, "lang": "et", "is_retweet": false, "user_mentions": {{ 853403167, 278168275, 28309724 }}, "user": { "id": 61121853, "name": "NOBITCHASSNESS", "screen_name": "Phor_dingo", "lang": "en", "location": "Seattle", "create_at": date("2009-07-28"), "description": "I'm foul mouthed, crude, anger pie, mechanic, junkyard dog, outrageous, an asshole, and I'll say some funny shit every now and then NSFW I make big urrplanes", "followers_count": 6254, "friends_count": 2918, "statues_count": 63161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Silver Firs, WA", "id": "003314b86e295845", "name": "Silver Firs", "place_type": "city", "bounding_box": rectangle("-122.187769,47.834183 -122.111025,47.891348") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5364380, "cityName": "Silver Firs" } }
+{ "create_at": datetime("2016-01-03T10:00:23.000Z"), "id": 683709507256455168, "text": "@Giants \nFinish strong! WIN THE CLOCK AND WIN THE GAME", "in_reply_to_status": -1, "in_reply_to_user": 240734425, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 240734425 }}, "user": { "id": 3798501852, "name": "Jeffrey Veilleux", "screen_name": "JeffreyVeilleux", "lang": "en", "location": "Carlsbad New Mexico", "create_at": date("2015-10-05"), "description": "Family man #1Cavemen fan", "followers_count": 7, "friends_count": 38, "statues_count": 73 }, "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-03T10:00:23.000Z"), "id": 683709507273281540, "text": "Hebrew Home #Nursing #Job: Geriatric Nursing Assistants (Long-term care GNA) (#Rockville, MD) https://t.co/AF4lkJPKMG #GNA #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.123972,39.0533434"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "Job", "Rockville", "GNA", "Jobs", "Hiring" }}, "user": { "id": 2532895171, "name": "CESLC Careers", "screen_name": "CESLCCareers", "lang": "en", "location": "Rockville, MD", "create_at": date("2014-05-29"), "description": "At CESLC we are committed to providing outstanding care for our seniors. Come see why over 200 staff members have been with us for more than 10 years!", "followers_count": 32, "friends_count": 3, "statues_count": 46 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Bethesda, MD", "id": "d3c4d514e13810f3", "name": "North Bethesda", "place_type": "city", "bounding_box": rectangle("-77.155886,39.012634 -77.084619,39.074304") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2456337, "cityName": "North Bethesda" } }
+{ "create_at": datetime("2016-01-03T10:00:23.000Z"), "id": 683709507336302592, "text": "@NerdJosh AFGC MLG SPLATOON EDITION?", "in_reply_to_status": 683707392924729344, "in_reply_to_user": 36711995, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 36711995 }}, "user": { "id": 735300073, "name": "Running Man", "screen_name": "Mahouko", "lang": "en", "location": "Queens, NY", "create_at": date("2012-08-03"), "description": "New York 真美亜美 Producer. Follow me on http://twitch.tv/mahouko08", "followers_count": 1429, "friends_count": 484, "statues_count": 51350 }, "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-03T10:00:24.000Z"), "id": 683709507952898048, "text": "HOW do I just wake up I think I was dead", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1387253718, "name": "Earlena", "screen_name": "CouchLena", "lang": "en", "location": "Medina, OH", "create_at": date("2013-04-28"), "description": "rest in peace Kat", "followers_count": 781, "friends_count": 549, "statues_count": 9806 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Medina, OH", "id": "00eb8de9015a3185", "name": "Medina", "place_type": "city", "bounding_box": rectangle("-81.907176,41.089056 -81.774633,41.215965") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39103, "countyName": "Medina", "cityID": 3948790, "cityName": "Medina" } }
+{ "create_at": datetime("2016-01-03T10:00:24.000Z"), "id": 683709508082864129, "text": ".@chrissysbridges \"When you talk, you are only repeating what you already know but if you listen, you may learn something new.\" Dalai Lama", "in_reply_to_status": 683693512680955904, "in_reply_to_user": 3603681256, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3603681256 }}, "user": { "id": 573111311, "name": "Susan Christine", "screen_name": "LoveOireachtas", "lang": "en", "location": "null", "create_at": date("2012-05-06"), "description": "Wherever you go, there you are.", "followers_count": 640, "friends_count": 2001, "statues_count": 14850 }, "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-03T10:00:24.000Z"), "id": 683709508175040512, "text": "New landmark created! Discover interesting places around: https://t.co/zo5nbgrjIC https://t.co/UK40OzlnfH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.098862,42.431664"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 255184064, "name": "GMS World", "screen_name": "geolocationms", "lang": "en", "location": "Warsaw, Poland", "create_at": date("2011-02-20"), "description": "Discover interesting places nearby!", "followers_count": 100, "friends_count": 1, "statues_count": 78593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Medford, MA", "id": "c8e7273a81fab7c0", "name": "Medford", "place_type": "city", "bounding_box": rectangle("-71.149807,42.396145 -71.071833,42.453759") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2539835, "cityName": "Medford" } }
+{ "create_at": datetime("2016-01-03T10:00:24.000Z"), "id": 683709508581863425, "text": "@killerOsprey I have never met him, but I have also never been seen in the same room with him so I will not comment further thank you", "in_reply_to_status": 683709322061164544, "in_reply_to_user": 1348466100, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1348466100 }}, "user": { "id": 26161350, "name": "Dr. Loomis Hamilton", "screen_name": "loomishamilton", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-03-23"), "description": "I'm a parody human. Mogul.", "followers_count": 1026, "friends_count": 832, "statues_count": 44539 }, "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-03T10:00:24.000Z"), "id": 683709508649025537, "text": "??? https://t.co/ASpcbCNPJF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1573465986, "name": "James Hope", "screen_name": "JamesHo69078419", "lang": "en", "location": "Hollywood Ca.", "create_at": date("2013-07-06"), "description": "Audio Engineer 4 Live Sound, & From 1985 Construction Electrician", "followers_count": 152, "friends_count": 618, "statues_count": 3300 }, "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-03T10:00:24.000Z"), "id": 683709508716109824, "text": "Morning clouds/afternoon sun this afternoon, high 33 (1 C). Low 24 (-4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119751854, "name": "Chicago Weather", "screen_name": "Chicago_WX", "lang": "en", "location": "Chicago, IL", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Chicago, IL", "followers_count": 1060, "friends_count": 68, "statues_count": 6422 }, "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-03T10:00:24.000Z"), "id": 683709508762251264, "text": "Wind 7 mph SSE. Barometer 1010.6 hPa, Falling. Temperature 90.3 °F. Rain today 0.00 in. Humidity 10%", "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": 87, "friends_count": 265, "statues_count": 140099 }, "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-03T10:00:24.000Z"), "id": 683709509169065985, "text": "@ashii03 sorry but we get win the prize for shittiest stadium. #RaiderNation", "in_reply_to_status": 683675131097530368, "in_reply_to_user": 260315410, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RaiderNation" }}, "user_mentions": {{ 260315410 }}, "user": { "id": 209892915, "name": "Leon Ollison", "screen_name": "leo7221978", "lang": "en", "location": "null", "create_at": date("2010-10-29"), "description": "null", "followers_count": 80, "friends_count": 445, "statues_count": 736 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-03T10:00:24.000Z"), "id": 683709509282459649, "text": "@mattobrien37 ..how? can you send me THAT list", "in_reply_to_status": 683709333675225088, "in_reply_to_user": 28176190, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28176190 }}, "user": { "id": 170011217, "name": "alina", "screen_name": "asrvtnik", "lang": "en", "location": "Western MA", "create_at": date("2010-07-23"), "description": "null", "followers_count": 342, "friends_count": 290, "statues_count": 27692 }, "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-01-03T10:00:24.000Z"), "id": 683709509458591745, "text": "Bdubs and Hawks game?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 473640145, "name": "Anthony Heaslip", "screen_name": "anthonyheaslip", "lang": "en", "location": "Rockford, Illinois", "create_at": date("2012-01-24"), "description": "#50in15.", "followers_count": 1192, "friends_count": 562, "statues_count": 43278 }, "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.964435,42.365885") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17201, "countyName": "Winnebago", "cityID": 1745031, "cityName": "Loves Park" } }
+{ "create_at": datetime("2016-01-03T10:00:24.000Z"), "id": 683709509789990912, "text": "Stop talking like #TamirRice had this gun out. Even #Loehmann admitted he never saw ANY weapon. Tamir executed with his hands in his pockets", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TamirRice", "Loehmann" }}, "user": { "id": 256352051, "name": "Magnificent2", "screen_name": "Indomitable69", "lang": "en", "location": "Whup Republican Ass Blvd.", "create_at": date("2011-02-22"), "description": "#Independent #Liberal #Leo #INTJ #RacistsBANNED4Life #Cowboys4Life #Obama4Life What is your deepest fear?", "followers_count": 5320, "friends_count": 5173, "statues_count": 112482 }, "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-03T10:00:24.000Z"), "id": 683709510335250432, "text": "Let's go @dallascowboys #WASvsDAL #CowboysNation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WASvsDAL", "CowboysNation" }}, "user_mentions": {{ 8824902 }}, "user": { "id": 29606947, "name": "Kyle Hinchman", "screen_name": "scwinchman27", "lang": "en", "location": "Lusby, MD", "create_at": date("2009-04-07"), "description": "Proud father of 2 and a huge fan of the Dallas Cowboys & rap/hip-hop music. #CowboysNation", "followers_count": 76, "friends_count": 201, "statues_count": 2056 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chesapeake Ranch Estates, MD", "id": "4dd2cff80a3e3636", "name": "Chesapeake Ranch Estates", "place_type": "city", "bounding_box": rectangle("-76.451763,38.319092 -76.380792,38.387749") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24009, "countyName": "Calvert", "cityID": 2416050, "cityName": "Chesapeake Ranch Estates" } }
+{ "create_at": datetime("2016-01-03T10:00:24.000Z"), "id": 683709510561599488, "text": "@32CharlieIV yeahh", "in_reply_to_status": 683706406034403328, "in_reply_to_user": 418270619, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 418270619 }}, "user": { "id": 18734999, "name": "Hannah Snyder", "screen_name": "Hannahbeatdown", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2009-01-07"), "description": "Graphic designer. Cat mom of 2. Running off of extra large cups of coffee and dreams.", "followers_count": 443, "friends_count": 592, "statues_count": 10572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Imperial, PA", "id": "01103da21c66b684", "name": "Imperial", "place_type": "city", "bounding_box": rectangle("-80.267456,40.428384 -80.169829,40.47999") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4236768, "cityName": "Imperial" } }
+{ "create_at": datetime("2016-01-03T10:00:24.000Z"), "id": 683709510695829504, "text": "NBA D-League\n\nVipers sign Guard Matt Carlino (Marquette/BYU)\n\nCarlino fills spot left by Jarvis Summers season ending injury.\n\n#KRGV #NBDL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "KRGV", "NBDL" }}, "user": { "id": 102158616, "name": "KRGV Sports", "screen_name": "KRGVSports", "lang": "en", "location": "Rio Grande Valley, Texas", "create_at": date("2010-01-05"), "description": "Follow the Rio Grande Valley's Sports Leader For Breaking Sports News & Stories!", "followers_count": 11750, "friends_count": 217, "statues_count": 26574 }, "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-03T10:00:24.000Z"), "id": 683709511035686912, "text": "seeking forever homes: Clown Fish, Celestial Fish, and Coral. All in porcelain by Franz. https://t.co/eGjqZSgISp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18558398, "name": "Carroll's Jewelers", "screen_name": "carrollsjeweler", "lang": "en", "location": "On Las Olas in Fort Lauderdale", "create_at": date("2009-01-02"), "description": "Custom Design Engagement Rings Fashion/Estate Jewelry Jewelry/Watch Repair Estate buying Certified Gemologist Appraisal", "followers_count": 1086, "friends_count": 679, "statues_count": 633 }, "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-03T10:00:24.000Z"), "id": 683709511052312577, "text": "We're #hiring! Read about our latest #job opening here: PT Security Officer - https://t.co/CqlPNOJHo7 #Bloomington, MN #security #Veterans", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.2982799,44.840798"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Bloomington", "security", "Veterans" }}, "user": { "id": 118914397, "name": "TMJ-MSP Security Job", "screen_name": "tmj_MSP_secure", "lang": "en", "location": "St. Paul, MN", "create_at": date("2010-03-01"), "description": "Follow this account for geo-targeted Security job tweets in St. Paul, MN from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 294, "friends_count": 266, "statues_count": 15 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomington, MN", "id": "080b8d8543aab399", "name": "Bloomington", "place_type": "city", "bounding_box": rectangle("-93.399443,44.78542 -93.203245,44.863519") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2706616, "cityName": "Bloomington" } }
+{ "create_at": datetime("2016-01-03T10:00:24.000Z"), "id": 683709511073447936, "text": "@BlazeHogan @BeauBilinsky it starts now ����������", "in_reply_to_status": -1, "in_reply_to_user": 1228839294, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1228839294, 1553513089 }}, "user": { "id": 395020505, "name": "D", "screen_name": "deannacoll_", "lang": "en", "location": "PITTSBURGH ♱ NYC DREAMIN'", "create_at": date("2011-10-20"), "description": "Mercyhurst University Fashion Merchandising • MUVB", "followers_count": 546, "friends_count": 533, "statues_count": 9168 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manor, PA", "id": "1d5a802a416313c7", "name": "Manor", "place_type": "city", "bounding_box": rectangle("-79.707516,40.311474 -79.644942,40.396018") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42129, "countyName": "Westmoreland", "cityID": 4247000, "cityName": "Manor" } }
+{ "create_at": datetime("2016-01-03T10:00:24.000Z"), "id": 683709511148924929, "text": "Hey Portland I think it's snowing, should have worn a warmer coat ❄️☃❄️ #PDX #PNW… https://t.co/lR9CEYatPY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.6792908,45.5188713"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PDX", "PNW" }}, "user": { "id": 28012949, "name": "Robin", "screen_name": "rlklein0234", "lang": "en", "location": "null", "create_at": date("2009-03-31"), "description": "Striving to live a more sustainable life & inspire others. I am St Croix/Lac Courte Oreilles Chippewa, a designer, muse, mom who loves art, music, movies & PDX", "followers_count": 684, "friends_count": 2000, "statues_count": 5902 }, "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-03T10:00:24.000Z"), "id": 683709511157202945, "text": "Finally, for the first time in my life, I got to witness snow fall.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 396144988, "name": "Alex Cusianovic", "screen_name": "Cusianovic", "lang": "en", "location": "California, United States", "create_at": date("2011-10-22"), "description": "If you have the opportunity to do something amazing, then you should.", "followers_count": 440, "friends_count": 441, "statues_count": 4002 }, "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-03T10:00:24.000Z"), "id": 683709511182479364, "text": "When you forget to spit game cuz u too busy trapppin at the function ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 244279115, "name": "Stanley Choi", "screen_name": "DoritozNKiloz", "lang": "en", "location": "(410) HoCo | Towson U '18", "create_at": date("2011-01-28"), "description": "A lil bit of henny and a few pulls of the sour, and she's ready to risk it all!", "followers_count": 292, "friends_count": 196, "statues_count": 15080 }, "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-03T10:00:24.000Z"), "id": 683709511215874048, "text": "@Shanna_Morrison tfj", "in_reply_to_status": 683709439791214592, "in_reply_to_user": 474382342, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 474382342 }}, "user": { "id": 20549679, "name": "Daniel LaForest", "screen_name": "yourmandaniel", "lang": "en", "location": "Memphis, TN", "create_at": date("2009-02-10"), "description": "May you live as long as you want and not want as long as you live. Results ≠ No Results + A Good story. Following, RTs, Mentions ≠ endorsement.", "followers_count": 772, "friends_count": 2015, "statues_count": 1181 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roland, OK", "id": "23bced1046032a28", "name": "Roland", "place_type": "city", "bounding_box": rectangle("-94.548873,35.401385 -94.491216,35.42767") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40135, "countyName": "Sequoyah", "cityID": 4063800, "cityName": "Roland" } }
+{ "create_at": datetime("2016-01-03T10:00:24.000Z"), "id": 683709511228456960, "text": "Morning clouds/afternoon sun this afternoon, high 55 (13 C). Low 34 (1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119753180, "name": "Jackson Weather", "screen_name": "JacksonWX", "lang": "en", "location": "Jackson, MS", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Jackson, MS", "followers_count": 295, "friends_count": 68, "statues_count": 6371 }, "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-03T10:00:24.000Z"), "id": 683709511266353153, "text": "Had so much fun last night ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 608705068, "name": "ℓєχ ☮", "screen_name": "alexis_myrick", "lang": "en", "location": "Bear country, Ga", "create_at": date("2012-06-14"), "description": "Bear Down & Go Dawgs | level 10 gymnast |", "followers_count": 1258, "friends_count": 729, "statues_count": 28537 }, "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-01-03T10:00:24.000Z"), "id": 683709511295709184, "text": "Win and we're in. J-E-T-S JETS JETS JETS!!!!!! #WinAndWereIn #L8GoesToBuffalo #JETUP ✈️✈️✈️ @… https://t.co/9riENQ383h", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.787,42.774"), "retweet_count": 0, "lang": "de", "is_retweet": false, "hashtags": {{ "WinAndWereIn", "L8GoesToBuffalo", "JETUP" }}, "user": { "id": 52783944, "name": "Bill", "screen_name": "bruuuuusca", "lang": "en", "location": "Queens, NY", "create_at": date("2009-07-01"), "description": "Rangers, Mets, Jets, St. John's....that's about it", "followers_count": 248, "friends_count": 316, "statues_count": 9569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orchard Park, NY", "id": "00f0112125d013dc", "name": "Orchard Park", "place_type": "city", "bounding_box": rectangle("-78.797966,42.724114 -78.696772,42.811289") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3655266, "cityName": "Orchard Park" } }
+{ "create_at": datetime("2016-01-03T10:00:24.000Z"), "id": 683709511438307329, "text": "Eventually �� https://t.co/MbcWiPEVQA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 786868418, "name": "Kaetlyn Rayne", "screen_name": "kaylorka", "lang": "en", "location": "Cincinnati, OH", "create_at": date("2012-08-28"), "description": "She acts like summer and walks like rain☔☀ 09.26.2015 || Lillian ❤", "followers_count": 793, "friends_count": 1568, "statues_count": 28344 }, "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-03T10:00:24.000Z"), "id": 683709511605948416, "text": "I want it all and i meant it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 344355665, "name": "WATCHDATBABYTIM ❗️", "screen_name": "Richasstim", "lang": "en", "location": "null", "create_at": date("2011-07-28"), "description": "null", "followers_count": 857, "friends_count": 571, "statues_count": 36015 }, "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-03T10:00:24.000Z"), "id": 683709511702425600, "text": "I look forward to the days of spray-on #neoprene. #CocoaBeach #ilovecocoabeach… https://t.co/hdIGeyI9XO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.60998992,28.32509093"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "neoprene", "CocoaBeach", "ilovecocoabeach" }}, "user": { "id": 9671232, "name": "octaviodiaz", "screen_name": "octaviodiaz", "lang": "en", "location": "Cocoa Beach, Florida", "create_at": date("2007-10-24"), "description": "I notice things.", "followers_count": 192, "friends_count": 361, "statues_count": 5733 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cocoa Beach, FL", "id": "376e6de722d9a691", "name": "Cocoa Beach", "place_type": "city", "bounding_box": rectangle("-80.64362,28.286581 -80.601732,28.372031") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12009, "countyName": "Brevard", "cityID": 1213175, "cityName": "Cocoa Beach" } }
+{ "create_at": datetime("2016-01-03T10:00:24.000Z"), "id": 683709511778045952, "text": "Life is Good\nDon't lock the way you believe @ Brooklyn Bridge https://t.co/6shLi2NdxM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.99243,40.70249"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 951776155, "name": "Ryunosuke Iwatsubo", "screen_name": "yes_ryu", "lang": "ja", "location": "null", "create_at": date("2012-11-16"), "description": "Fukuoka, Tokyo, San Francisco", "followers_count": 359, "friends_count": 329, "statues_count": 3235 }, "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-03T10:00:24.000Z"), "id": 683709511836676096, "text": "#Cheyenne, WY #Nursing : TRAVEL RN / REGISTERED NURSE / L&D RN / LABOR AND DELIVERY NURSE at Aureus Medical Group https://t.co/8AwAxNBJC5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.8202462,41.1399814"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Cheyenne", "Nursing" }}, "user": { "id": 49870147, "name": "Cheyenne Nursing", "screen_name": "tmj_wyc_nursing", "lang": "en", "location": "Cheyenne, WY", "create_at": date("2009-06-22"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Cheyenne, WY from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 333, "friends_count": 258, "statues_count": 44 }, "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-03T10:00:24.000Z"), "id": 683709511882960896, "text": "@CallieLC11 yes, our current one seemed sad without his friend.", "in_reply_to_status": 683709345356509184, "in_reply_to_user": 64778834, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 64778834 }}, "user": { "id": 2302211785, "name": "Ellie", "screen_name": "Ellie_H13", "lang": "en", "location": "Society XXI ", "create_at": date("2014-01-20"), "description": "We are not the opinions of people who don't know or care about us -Taylor Swift", "followers_count": 250, "friends_count": 502, "statues_count": 15959 }, "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-01-03T10:00:25.000Z"), "id": 683709512038150144, "text": "@hannieebrown if u wanna drive to idaho ill come", "in_reply_to_status": 683707209990168577, "in_reply_to_user": 1135384184, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1135384184 }}, "user": { "id": 1349507946, "name": "sum", "screen_name": "suuummaa", "lang": "en", "location": "757", "create_at": date("2013-04-13"), "description": "null", "followers_count": 1101, "friends_count": 545, "statues_count": 30075 }, "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-03T10:00:25.000Z"), "id": 683709512113631233, "text": "The road to London starts now. #Eagles", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Eagles" }}, "user": { "id": 387141109, "name": "Greg Frank", "screen_name": "g_frank6", "lang": "en", "location": "gregfrank6@yahoo.com", "create_at": date("2011-10-08"), "description": "@TempleUniv 2017, Writer @TheTempleNews, staff @OwlSportsUpdate, host @WHIPRADIO. Temple Journalism Major. Below average pickup basketball player.", "followers_count": 390, "friends_count": 1308, "statues_count": 13285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Barclay, NJ", "id": "00077b717cda9edf", "name": "Barclay", "place_type": "city", "bounding_box": rectangle("-75.017153,39.873108 -74.927195,39.929985") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34007, "countyName": "Camden", "cityID": 3402880, "cityName": "Barclay" } }
+{ "create_at": datetime("2016-01-03T10:00:25.000Z"), "id": 683709512126173187, "text": "Would rather be alone than unhappy.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.46839914,29.58554223"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35637234, "name": "queenBee", "screen_name": "GirlW_Tattoos", "lang": "en", "location": "null", "create_at": date("2009-04-26"), "description": "InstaGram: beescott_ SnapChat: bee_scott93 #BeYOU❗️", "followers_count": 1637, "friends_count": 969, "statues_count": 51822 }, "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-01-03T10:00:25.000Z"), "id": 683709512323215361, "text": "Want to work in #JerseyCity, NJ? View our latest opening: https://t.co/oWoku1rqcC #Accounting #ApplyNow #Job #Jobs #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": {{ "JerseyCity", "Accounting", "ApplyNow", "Job", "Jobs", "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": 22, "friends_count": 42, "statues_count": 22 }, "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-03T10:00:25.000Z"), "id": 683709512432390145, "text": "@JeanPierreRosa I def agree. I'm just preparing now for when I get there. Need a solid plan.", "in_reply_to_status": 683659602735542273, "in_reply_to_user": 246538057, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 246538057 }}, "user": { "id": 10788392, "name": "Allyn Hane", "screen_name": "AllynPaul", "lang": "en", "location": "St Petersburg, FL", "create_at": date("2007-12-02"), "description": "YouTuber :: Lawn Care Nut :: VP Digital Strategy @LaunchDigital :: Florida Native", "followers_count": 3096, "friends_count": 2900, "statues_count": 7375 }, "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-03T10:00:25.000Z"), "id": 683709512499462145, "text": "bruh I remember when everyone wanted the middle finger emoji & now no one uses it ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2611639389, "name": "yung $and", "screen_name": "mckenziesandraa", "lang": "en", "location": "null", "create_at": date("2014-06-15"), "description": "null", "followers_count": 345, "friends_count": 279, "statues_count": 8496 }, "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-01-03T10:00:25.000Z"), "id": 683709512625172481, "text": "@realest_jlaw I haven't started haha", "in_reply_to_status": 683708111333490688, "in_reply_to_user": 3067696157, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3067696157 }}, "user": { "id": 950648384, "name": "brandon", "screen_name": "ItsUnRiel", "lang": "en", "location": "RVA", "create_at": date("2012-11-15"), "description": "lil ol me", "followers_count": 2018, "friends_count": 1921, "statues_count": 15573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakeside, VA", "id": "5c1e04e89b26b3fb", "name": "Lakeside", "place_type": "city", "bounding_box": rectangle("-77.500713,37.58881 -77.455218,37.633827") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51087, "countyName": "Henrico", "cityID": 5143464, "cityName": "Lakeside" } }
+{ "create_at": datetime("2016-01-03T10:00:25.000Z"), "id": 683709512755183616, "text": "Interested in a #Hospitality #job near #Monroe, LA? This could be a great fit: https://t.co/MuTNiDKSWL #SONIC #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.1134958,32.5239256"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Monroe", "SONIC", "Hiring", "CareerArc" }}, "user": { "id": 132057683, "name": "TMJ-LAM HRTA Jobs", "screen_name": "tmj_LAM_HRTA", "lang": "en", "location": "Monroe, LA", "create_at": date("2010-04-11"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Monroe, LA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 294, "friends_count": 272, "statues_count": 128 }, "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-03T10:00:25.000Z"), "id": 683709512935673856, "text": "Temp 42.3°F RH 51% Wind 4.1 WNW Gust 10.0 WNW SLP 30.028 in Falling Rain 0.00 in Solar 533 UV 2.4 #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": 102, "friends_count": 65, "statues_count": 32257 }, "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-03T10:00:25.000Z"), "id": 683709512956559360, "text": "Don't you tease me... https://t.co/WOyhW5riDI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.72392153,41.42907663"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2341010750, "name": "Larry Pompeo", "screen_name": "Larrycp_104", "lang": "en", "location": "null", "create_at": date("2014-02-12"), "description": "I like me and me", "followers_count": 83, "friends_count": 138, "statues_count": 208 }, "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-03T10:00:25.000Z"), "id": 683709513107521536, "text": "Can you recommend anyone for this #BusinessMgmt #job? https://t.co/zPFwythzS2 #JacksonvilleUS, FL #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.655651,30.3321838"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BusinessMgmt", "job", "JacksonvilleUS", "Hiring", "CareerArc" }}, "user": { "id": 22030524, "name": "TMJ-FL Mgmt. Jobs", "screen_name": "tmj_fl_mgmt", "lang": "en", "location": "Florida", "create_at": date("2009-02-26"), "description": "Follow this account for geo-targeted Business/Mgmt. job tweets in Florida Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 383, "friends_count": 248, "statues_count": 476 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downtown Jacksonville, Jacksonville", "id": "3496512730330cc3", "name": "Downtown Jacksonville", "place_type": "neighborhood", "bounding_box": rectangle("-81.667342,30.320695 -81.6461,30.33406") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12031, "countyName": "Duval", "cityID": 1235000, "cityName": "Jacksonville" } }
+{ "create_at": datetime("2016-01-03T10:00:25.000Z"), "id": 683709513858351104, "text": "Sunny this afternoon, high 57 (14 C). Low 31 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119754993, "name": "Waco Weather", "screen_name": "WacoWeather", "lang": "en", "location": "Waco, TX", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Waco, TX", "followers_count": 816, "friends_count": 68, "statues_count": 6352 }, "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.069786,31.639828") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4876000, "cityName": "Waco" } }
+{ "create_at": datetime("2016-01-03T10:00:25.000Z"), "id": 683709514055548928, "text": "@theark27 ���� Bet", "in_reply_to_status": 683709298111811585, "in_reply_to_user": 22513464, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 22513464 }}, "user": { "id": 26696875, "name": "PR1DE", "screen_name": "Optimus_Priide", "lang": "en", "location": "MKE", "create_at": date("2009-03-25"), "description": "If rhymes were Valiums, I'd be comfortably numb.", "followers_count": 74, "friends_count": 128, "statues_count": 1145 }, "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-03T10:00:25.000Z"), "id": 683709514575511552, "text": "I need to find a ride home ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1220162372, "name": "CLICHE➰", "screen_name": "Chekardash", "lang": "en", "location": "null", "create_at": date("2013-02-25"), "description": "sc: chekardash", "followers_count": 2583, "friends_count": 817, "statues_count": 65785 }, "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-03T10:00:25.000Z"), "id": 683709514647007238, "text": "That was so uncomfortable ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 395739405, "name": "Shay Little", "screen_name": "Shay_M_96", "lang": "en", "location": "Springfield - Edwardsville, IL", "create_at": date("2011-10-21"), "description": "null", "followers_count": 349, "friends_count": 269, "statues_count": 13641 }, "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-01-03T10:00:25.000Z"), "id": 683709514701340672, "text": "@InaMaziarcz @robspillman This is the reason why guns don't belong in the hands of society, it makes people's ambition for power real on us.", "in_reply_to_status": 683689386823839744, "in_reply_to_user": 703983715, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 703983715, 56420636 }}, "user": { "id": 16672155, "name": "Ashley Barnett Smith", "screen_name": "Ashlybsmith", "lang": "en", "location": "US", "create_at": date("2008-10-09"), "description": "I'm a Mystery Novel Author. I've written four books. The Columnist's Blog, A Soldier's Fight, Moscow Connection & The Journal Check it at http://amazon.com", "followers_count": 666, "friends_count": 2003, "statues_count": 38075 }, "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-03T10:00:25.000Z"), "id": 683709514734907396, "text": "See our latest #Indianapolis, IN #job and click to apply: Crisis Mental Health Therapist - https://t.co/MkqlhllQuN #Healthcare #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.1580423,39.7683765"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Indianapolis", "job", "Healthcare", "Hiring" }}, "user": { "id": 21725584, "name": "TMJ- IND Health Jobs", "screen_name": "tmj_ind_health", "lang": "en", "location": "Indianapolis, IN", "create_at": date("2009-02-23"), "description": "Follow this account for geo-targeted Healthcare job tweets in Indianapolis, IN. Need help? Tweet us at @CareerArc!", "followers_count": 408, "friends_count": 303, "statues_count": 773 }, "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-01-03T10:00:25.000Z"), "id": 683709514747482112, "text": "mfs ain't real nomore", "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": "chante.", "screen_name": "achant3_", "lang": "en", "location": "null", "create_at": date("2014-04-12"), "description": "just trying to make it...", "followers_count": 1176, "friends_count": 694, "statues_count": 11319 }, "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-03T10:00:25.000Z"), "id": 683709514781192192, "text": "@RandyDunnit \"dark teddy\" ��", "in_reply_to_status": 683584829645025280, "in_reply_to_user": 473254235, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 473254235 }}, "user": { "id": 816408242, "name": "Omega Richards", "screen_name": "OFWGTanMan", "lang": "en", "location": "The CommonWealth", "create_at": date("2012-09-10"), "description": "There's a million things I haven't done, but just you wait.", "followers_count": 269, "friends_count": 88, "statues_count": 5910 }, "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-03T10:00:25.000Z"), "id": 683709515351470081, "text": "\"Texans after open carry\" �� https://t.co/pGQH6utITz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1602695912, "name": "double steez burger", "screen_name": "silentalb", "lang": "en", "location": "dddddddddallas", "create_at": date("2013-07-17"), "description": "I'm awake", "followers_count": 551, "friends_count": 410, "statues_count": 25170 }, "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-03T10:00:25.000Z"), "id": 683709515762499584, "text": "@laurenwiliams8 @Starbucks that is my own hand you are so rude", "in_reply_to_status": 683709153794195457, "in_reply_to_user": 2191385298, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2191385298, 30973 }}, "user": { "id": 2799163676, "name": "Malorie Jatnieks", "screen_name": "maloriejatnieks", "lang": "en", "location": "null", "create_at": date("2014-09-08"), "description": "null", "followers_count": 214, "friends_count": 209, "statues_count": 1342 }, "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-01-03T10:00:25.000Z"), "id": 683709515901079553, "text": "If I gotta question whether or not you're trust worthy, ima just ✂️ you off right then and there.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 423858207, "name": "Alma Nicole Dillon", "screen_name": "missalmanicole", "lang": "en", "location": "Charlotte, NC", "create_at": date("2011-11-28"), "description": "Feeling reckless, living it up; just because.", "followers_count": 251, "friends_count": 210, "statues_count": 1948 }, "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-03T10:00:25.000Z"), "id": 683709515959681025, "text": "CVS Health: CoordinatorRx Call Center (#Medley, FL) https://t.co/F6bk6lPgrG #CustomerService #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.3264404,25.8406526"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Medley", "CustomerService", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 71968216, "name": "FL Cust. Srv. Jobs", "screen_name": "tmj_FL_cstsrv", "lang": "en", "location": "Florida", "create_at": date("2009-09-05"), "description": "Follow this account for geo-targeted Customer Service job tweets in Florida Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 339, "friends_count": 275, "statues_count": 389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Medley, FL", "id": "0195c7e129eda2ae", "name": "Medley", "place_type": "city", "bounding_box": rectangle("-80.388384,25.826149 -80.306145,25.900034") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1243900, "cityName": "Medley" } }
+{ "create_at": datetime("2016-01-03T10:00:25.000Z"), "id": 683709516001755136, "text": "My freezer is taking forever to defrost.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26409238, "name": "Таб", "screen_name": "amerimatryoshka", "lang": "en", "location": "NYC & St. Petersburg, Russia", "create_at": date("2009-03-24"), "description": "New Yorker, Traveler, Hustler of Culture, Russophile, History Buff, Book Worm, Sugar Junkie. Challenged the British Nationality Act...and Won!", "followers_count": 681, "friends_count": 459, "statues_count": 40287 }, "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-03T10:00:26.000Z"), "id": 683709516316147713, "text": "I would love to show you my #listing at 638 Thornbird Circle #BoilingSprings #SC https://t.co/4EIlHsYko6 #realestate https://t.co/TcJsaPskIa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.961641,35.029616"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "BoilingSprings", "SC", "realestate" }}, "user": { "id": 35979233, "name": "LISA EBERSOLE", "screen_name": "lisasells", "lang": "en", "location": "Greenville-Spartanburg", "create_at": date("2009-04-27"), "description": "Wife, Mom, Mimi, Top Producing Realtor®, Piano Player, Singer, Coffee Lover! Call me today for all your real estate needs! For with God ALL things are possible!", "followers_count": 437, "friends_count": 574, "statues_count": 7764 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boiling Springs, SC", "id": "07e90b95fd0aa9d9", "name": "Boiling Springs", "place_type": "city", "bounding_box": rectangle("-82.036159,35.016441 -81.876408,35.101499") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45083, "countyName": "Spartanburg", "cityID": 4507345, "cityName": "Boiling Springs" } }
+{ "create_at": datetime("2016-01-03T10:00:26.000Z"), "id": 683709516433625089, "text": "Can you recommend anyone for this #job? Restaurant Team Member (Crew) - https://t.co/liRrhdZh1z #SONIC #Zebulon, NC #Hospitality #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.322874,35.837823"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "SONIC", "Zebulon", "Hospitality", "Hiring" }}, "user": { "id": 22452620, "name": "TMJ-RDU HRTA Jobs", "screen_name": "tmj_rdu_hrta", "lang": "en", "location": "RDU, NC", "create_at": date("2009-03-01"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Raleigh/Durham, NC. Need help? Tweet us at @CareerArc!", "followers_count": 414, "friends_count": 292, "statues_count": 477 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Zebulon, NC", "id": "24b9361a2c3c4dba", "name": "Zebulon", "place_type": "city", "bounding_box": rectangle("-78.351034,35.811927 -78.29703,35.846225") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3776220, "cityName": "Zebulon" } }
+{ "create_at": datetime("2016-01-03T10:00:26.000Z"), "id": 683709516756553728, "text": "Partly cloudy this afternoon, high 44 (7 C). Low 28 (-2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119755728, "name": "Louisville Weather", "screen_name": "Louisville_WX", "lang": "en", "location": "Louisville, KY", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Louisville, KY", "followers_count": 1308, "friends_count": 68, "statues_count": 6378 }, "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-01-03T10:00:26.000Z"), "id": 683709516781846528, "text": "@Browns yeah let's go to work... Lol", "in_reply_to_status": 683709307607715840, "in_reply_to_user": 40358743, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 40358743 }}, "user": { "id": 25958286, "name": "Jared Guitar", "screen_name": "JaredGuitar00", "lang": "en", "location": "Cleveland Browns ♡", "create_at": date("2009-03-22"), "description": "My name is Jared. I have a pretty good credit score. Sooooo Yeah.", "followers_count": 152, "friends_count": 100, "statues_count": 4326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wellington, OH", "id": "a3af9899b32c4fa7", "name": "Wellington", "place_type": "city", "bounding_box": rectangle("-82.246927,41.147951 -82.206474,41.184212") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39093, "countyName": "Lorain", "cityID": 3982642, "cityName": "Wellington" } }
+{ "create_at": datetime("2016-01-03T10:00:26.000Z"), "id": 683709516865748996, "text": "Let's get it @Patriots", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 31126587 }}, "user": { "id": 2224250272, "name": "Jon Riordan", "screen_name": "Riordan5J", "lang": "en", "location": "|MA|", "create_at": date("2013-12-13"), "description": "null", "followers_count": 308, "friends_count": 195, "statues_count": 2306 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plymouth, MA", "id": "01bbfc630e681dda", "name": "Plymouth", "place_type": "city", "bounding_box": rectangle("-70.758703,41.789998 -70.525503,41.972125") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25023, "countyName": "Plymouth", "cityID": 2554275, "cityName": "Plymouth" } }
+{ "create_at": datetime("2016-01-03T10:00:26.000Z"), "id": 683709517016662016, "text": "Tykni https://t.co/7I2Y2V9dpk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pl", "is_retweet": false, "user": { "id": 732953395, "name": "knight☄", "screen_name": "TyLatrice", "lang": "en", "location": "Flint, MI ✈️ Houston, TX ", "create_at": date("2012-08-02"), "description": "free jdill❣snapchat tykni", "followers_count": 1883, "friends_count": 1528, "statues_count": 159049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-03T10:00:26.000Z"), "id": 683709517138272256, "text": "@ohiobigcat 2 more I think. Yea, Id be smiling too.", "in_reply_to_status": 683709244563156992, "in_reply_to_user": 89088537, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 89088537 }}, "user": { "id": 382701558, "name": "Go Cavs", "screen_name": "JimWalker13", "lang": "en", "location": "#TheLand ", "create_at": date("2011-09-30"), "description": "You know me. Still the same old G. #Cavs #Buckeyes #TheLand #OHIOvsTheWorld #Indians #Akron #Browns", "followers_count": 1550, "friends_count": 1157, "statues_count": 78639 }, "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-01-03T10:00:26.000Z"), "id": 683709517146619904, "text": "So many notifications when I woke up but still none from the person I want it 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": 286346208, "name": "Rebecca", "screen_name": "__MixedPrincess", "lang": "en", "location": "Cole World ", "create_at": date("2011-04-22"), "description": "•Love me•• SJG•• I just want to adventure•", "followers_count": 2433, "friends_count": 1415, "statues_count": 161805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marin City, CA", "id": "01410b71117b08fa", "name": "Marin City", "place_type": "city", "bounding_box": rectangle("-122.521009,37.863235 -122.507135,37.879578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6041, "countyName": "Marin", "cityID": 645820, "cityName": "Marin City" } }
+{ "create_at": datetime("2016-01-03T10:00:26.000Z"), "id": 683709517159350272, "text": "Um. @JerSchrand, we need Hulu. @HappyEndingsVH1 is now streaming! �� I need @adampally @HappyElishas @caseyrosewilson back in my life. Now.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 247507635, 161128687, 26582656, 1130156892, 80977873 }}, "user": { "id": 811714992, "name": "Angelina Schrand", "screen_name": "acschrand", "lang": "en", "location": "Northern Kentucky", "create_at": date("2012-09-08"), "description": "Living the sweet, happy life with my adorable hubs & son. My legal name is too long to fit in the name box. Sicilian and it shows. #theluckiest", "followers_count": 166, "friends_count": 396, "statues_count": 3166 }, "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-01-03T10:00:26.000Z"), "id": 683709517192925184, "text": "Parents are ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1003979672, "name": "Ash ♕", "screen_name": "Marie_ashxoxo", "lang": "en", "location": "null", "create_at": date("2012-12-11"), "description": "Be the reason someone smiles today ☀️", "followers_count": 531, "friends_count": 573, "statues_count": 22870 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wayne, MI", "id": "6f7df6cb0ecd58ad", "name": "Wayne", "place_type": "city", "bounding_box": rectangle("-83.427427,42.265498 -83.348061,42.28933") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2684940, "cityName": "Wayne" } }
+{ "create_at": datetime("2016-01-03T10:00:26.000Z"), "id": 683709517289271296, "text": "Back 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": 2492188784, "name": "Alejandra Contreras", "screen_name": "ale_33contreras", "lang": "en", "location": "Oakland, CA", "create_at": date("2014-05-12"), "description": "Ahi que darle gusto al gusto. La vida pronto se acaba -Ramon Ayala", "followers_count": 180, "friends_count": 151, "statues_count": 2384 }, "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-03T10:00:26.000Z"), "id": 683709517360705539, "text": "#PITvsCLE I got my Browns today just hope they don't get embarrassed to much����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PITvsCLE" }}, "user": { "id": 1223381011, "name": "Domenic Martucci Jr", "screen_name": "domenicmartucci", "lang": "en", "location": "@Logan_mason1734", "create_at": date("2013-02-26"), "description": "Mentor '18 | Runningback #22 |@Logan_mason1734", "followers_count": 446, "friends_count": 1156, "statues_count": 1907 }, "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-03T10:00:26.000Z"), "id": 683709517566226437, "text": "@jittsista_kari Ight bet", "in_reply_to_status": 683708538204590081, "in_reply_to_user": 179920634, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 179920634 }}, "user": { "id": 2499792430, "name": "JittAintShit#FDG4️⃣", "screen_name": "PacoGang_Jitt", "lang": "en", "location": "#PacoGang", "create_at": date("2014-04-21"), "description": "#Famu18 305✈️850| RIP PacoGotti 2.13.15 ‼️RIP Stevie ‼️RIP Pickle ‼️RIP Auntie Cheryl‼️... David Living Through Me‼️", "followers_count": 2163, "friends_count": 1850, "statues_count": 11805 }, "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-01-03T10:00:26.000Z"), "id": 683709518161678336, "text": "Watch Bradford tear his ACL today", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43031197, "name": "#BleedGreen", "screen_name": "mhuntone", "lang": "en", "location": "Rochester, NY", "create_at": date("2009-05-27"), "description": "Father of 2, terrible guitarist, just twiddling my thumbs for another yr on the Philadelphia Eagles Season Ticket Waiting List #DezDidntCatchIt", "followers_count": 399, "friends_count": 1363, "statues_count": 9891 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Victor, NY", "id": "00201e4c78552ccb", "name": "Victor", "place_type": "city", "bounding_box": rectangle("-77.48445,42.965213 -77.370621,43.034903") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36069, "countyName": "Ontario", "cityID": 3677376, "cityName": "Victor" } }
+{ "create_at": datetime("2016-01-03T10:00:26.000Z"), "id": 683709518476251136, "text": "You can't lead a transition without a transformation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2468502127, "name": "JoMcc", "screen_name": "jordannn_m_", "lang": "en", "location": "null", "create_at": date("2014-04-28"), "description": "quit that out", "followers_count": 142, "friends_count": 239, "statues_count": 1649 }, "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-01-03T10:00:26.000Z"), "id": 683709518644129797, "text": "Sorry I said your baby has resting-Patrick-Ewing-face", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 95928470, "name": "Doug Bies", "screen_name": "dougbies", "lang": "en", "location": "Long Island, New York", "create_at": date("2009-12-10"), "description": " Snob, Beer Snob, Chicago native, Candy fanatic, Proud Dad of fish (1), Hipster-friendly, Basic intolerant, Larry David idol; IG, Snapchat, & Untappd @dougbies", "followers_count": 7134, "friends_count": 5953, "statues_count": 28358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Huntington, NY", "id": "5946966f54d17dbb", "name": "South Huntington", "place_type": "city", "bounding_box": rectangle("-73.422179,40.809853 -73.364496,40.835664") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3669254, "cityName": "South Huntington" } }
+{ "create_at": datetime("2016-01-03T10:00:26.000Z"), "id": 683709519503986690, "text": "SOLD! #Handmade #beach themed handstamped #giftbag & #bookmarks by @sosartgallery… https://t.co/m40GnUo37X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4501877,28.4670963"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Handmade", "beach", "giftbag", "bookmarks" }}, "user_mentions": {{ 1271659207 }}, "user": { "id": 1271659207, "name": "SOS Art Gallery", "screen_name": "SOSArtGallery", "lang": "en", "location": "Orlando, FL (USA)", "create_at": date("2013-03-16"), "description": "Studio Orlando Services (SOS) Art Gallery = Artwork & Handmade Cards / Gifts Sold Globally * TOP 10 FL BLOG 2015: Biz / Money / Marketing / Tech", "followers_count": 978, "friends_count": 804, "statues_count": 9539 }, "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-03T10:00:26.000Z"), "id": 683709519667408896, "text": "I'm fucking weak �� https://t.co/QLqAcVXMec", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 161733069, "name": "alejandrea", "screen_name": "Lollieandrea", "lang": "en", "location": "null", "create_at": date("2010-07-01"), "description": "JFK", "followers_count": 465, "friends_count": 493, "statues_count": 18347 }, "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-03T10:00:26.000Z"), "id": 683709519881318400, "text": "I done messed up the boy sleeping schedule ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2431026626, "name": "Ⓝ", "screen_name": "_lachaun", "lang": "en", "location": "918", "create_at": date("2014-04-06"), "description": "null", "followers_count": 1364, "friends_count": 1342, "statues_count": 7388 }, "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-01-03T10:00:26.000Z"), "id": 683709519910711297, "text": "@RealBenCarson Go back so sleep Ben.", "in_reply_to_status": 683701049073504258, "in_reply_to_user": 1180379185, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1180379185 }}, "user": { "id": 1365441361, "name": "Mechadave", "screen_name": "Anumbr1", "lang": "en", "location": "null", "create_at": date("2013-04-19"), "description": "Retired", "followers_count": 61, "friends_count": 78, "statues_count": 5725 }, "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-03T10:00:26.000Z"), "id": 683709520002940932, "text": "And another friendly nudge reminding me l how much I hate people https://t.co/BWca5EiFzM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 327802439, "name": "M@GICTROPHSKY", "screen_name": "MAGICMIKE1212", "lang": "en", "location": "Tarzana, Los Angeles", "create_at": date("2011-07-01"), "description": "I'm thinking of a number between .1 & 1", "followers_count": 67, "friends_count": 220, "statues_count": 1678 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-03T10:00:26.000Z"), "id": 683709520187531264, "text": "come home, i miss you :( @AyeeItssArii", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2187580382 }}, "user": { "id": 2188787972, "name": "mads", "screen_name": "Maddylinnn", "lang": "en", "location": "null", "create_at": date("2013-11-11"), "description": "null", "followers_count": 207, "friends_count": 208, "statues_count": 1620 }, "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-01-03T10:00:27.000Z"), "id": 683709520464470016, "text": "Next Friday Jan 8th 11pm: First #ibomba of the year at @cmoneverybodybk and it's our 4th… https://t.co/MZZoOTkvYx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.95681,40.6882782"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ibomba" }}, "user_mentions": {{ 2881865128 }}, "user": { "id": 41916954, "name": "Ushka (Thanu)", "screen_name": "ty_ushka", "lang": "en", "location": "Brooklyn and the Internet", "create_at": date("2009-05-22"), "description": "global south l music culture l media l sri lanka l race l (im)migration l advocate l dj l vast interests l own tweets", "followers_count": 1841, "friends_count": 1440, "statues_count": 11849 }, "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-03T10:00:27.000Z"), "id": 683709520477057025, "text": "Cheguei Orlando!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 258044754, "name": "Nathália", "screen_name": "nathaliaruzicka", "lang": "pt", "location": "null", "create_at": date("2011-02-26"), "description": "ame-se.", "followers_count": 197, "friends_count": 176, "statues_count": 24305 }, "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-03T10:00:27.000Z"), "id": 683709520581931008, "text": "I'm at Chipotle Mexican Grill in Atlanta, GA https://t.co/NWi2WLrsrd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.37966704,33.91644048"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 133417627, "name": ".", "screen_name": "_amberann", "lang": "en", "location": ". NJ.GA .", "create_at": date("2010-04-15"), "description": "Welcome to the party.", "followers_count": 740, "friends_count": 388, "statues_count": 34103 }, "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-01-03T10:00:27.000Z"), "id": 683709520661594112, "text": "Wanna make performing 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": 3380357593, "name": "ヽ’", "screen_name": "rxghteous", "lang": "en", "location": "null", "create_at": date("2015-08-29"), "description": "righteous aka. mihji / artist", "followers_count": 124, "friends_count": 179, "statues_count": 37 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Decatur, GA", "id": "7af0fb6f137530df", "name": "North Decatur", "place_type": "city", "bounding_box": rectangle("-84.310051,33.776522 -84.262886,33.829099") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1356112, "cityName": "North Decatur" } }
+{ "create_at": datetime("2016-01-03T10:00:27.000Z"), "id": 683709520762277888, "text": "No doubt! Lol!!!! https://t.co/s7VwfxcnC4", "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": "Paradise, FL", "create_at": date("2009-04-30"), "description": "❤️, CEO, self made Millionaire, Type A, Christian, unafraid, Honoring Vets, I love ppl who ❤️.", "followers_count": 875, "friends_count": 1012, "statues_count": 5458 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Janesville, WI", "id": "1b210235050e5e85", "name": "Janesville", "place_type": "city", "bounding_box": rectangle("-89.080479,42.613559 -88.943512,42.759238") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55105, "countyName": "Rock", "cityID": 5537825, "cityName": "Janesville" } }
+{ "create_at": datetime("2016-01-03T10:00:27.000Z"), "id": 683709520774729728, "text": "Yo if any of my followers are good with design drop me a dm pls and thx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 393119276, "name": "Jar Lar Binks", "screen_name": "Jar_Lar", "lang": "en", "location": "Ames, IA//The jive in the 515", "create_at": date("2011-10-17"), "description": "Catholic||just a tall angry intersectional feminist||cyclONEnation||Isaiah 40:29-31 #BlackLivesMatter #TreverRyenForHeisman #LANRAM", "followers_count": 1072, "friends_count": 1047, "statues_count": 27881 }, "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-03T10:00:27.000Z"), "id": 683709520892174337, "text": "you can look at it, as long as you don't grab it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1323473220, "name": "nolimitlex", "screen_name": "alexpowellx3", "lang": "en", "location": "St.Croix Virgin Islands ", "create_at": date("2013-04-02"), "description": "attitude like Kanye, feelings like Drake |201-717| unbothered|", "followers_count": 823, "friends_count": 271, "statues_count": 23068 }, "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-03T10:00:27.000Z"), "id": 683709520934256640, "text": "Start the day out right with some Daith action. #UnHolyInkTattooandPiercingStudio… https://t.co/hg7rpvYb8c", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8685303,32.2073517"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "UnHolyInkTattooandPiercingStudio" }}, "user": { "id": 2856045262, "name": "Gabriel Guerrero", "screen_name": "sparky2braidz", "lang": "en", "location": "null", "create_at": date("2014-11-01"), "description": "I'm a Piercer in Tucson, AZ @UnHolyInkTattoo. I play bass. Need more? Ask.", "followers_count": 79, "friends_count": 252, "statues_count": 149 }, "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-03T10:00:27.000Z"), "id": 683709521026482177, "text": "If you're healthy and not playing for the patriots today you aren't very good. Simple as that.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 214935229, "name": "J.R.", "screen_name": "jricci_rich77", "lang": "en", "location": "603 -- 203 QU '17", "create_at": date("2010-11-12"), "description": "Family First. Once won the World Cup with the Ivory Coast in FIFA 16.", "followers_count": 381, "friends_count": 326, "statues_count": 41036 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portsmouth, NH", "id": "fb3aafdc1555efcf", "name": "Portsmouth", "place_type": "city", "bounding_box": rectangle("-70.820242,43.013673 -70.737324,43.098556") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33015, "countyName": "Rockingham", "cityID": 3362900, "cityName": "Portsmouth" } }
+{ "create_at": datetime("2016-01-03T10:00:27.000Z"), "id": 683709521156530178, "text": "St Paul, MN: Only in St. Paul, MN would it be 28ºF and cloudy right now.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.1,44.95"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3195747619, "name": "Weather By Brian", "screen_name": "WeatherByBrian", "lang": "en", "location": "null", "create_at": date("2015-05-14"), "description": "Tweets the weather about wherever Brian last tweets.\n\nRun by @bman4789", "followers_count": 11, "friends_count": 1, "statues_count": 1595 }, "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.89076 -93.003514,44.992279") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27123, "countyName": "Ramsey", "cityID": 2758000, "cityName": "St. Paul" } }
+{ "create_at": datetime("2016-01-03T10:00:27.000Z"), "id": 683709521374482433, "text": "This #Hospitality #job might be a great fit for you: Barista (US) - https://t.co/it7SvhFOfN #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.2642831,32.3539443"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 22454785, "name": "TMJ-MGM HRTA Jobs", "screen_name": "tmj_mgm_hrta", "lang": "en", "location": "Montgomery, AL", "create_at": date("2009-03-01"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Montgomery, AL. Need help? Tweet us at @CareerArc!", "followers_count": 322, "friends_count": 290, "statues_count": 129 }, "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-03T10:00:27.000Z"), "id": 683709521664016386, "text": "sometimes you just have to fight for what you want and it will 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": 1293468907, "name": "alex mashon", "screen_name": "alex_lynn4", "lang": "en", "location": "Hammond, LA", "create_at": date("2013-03-23"), "description": "hhms junior. cheerleader", "followers_count": 538, "friends_count": 530, "statues_count": 2765 }, "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-03T10:00:27.000Z"), "id": 683709521672310784, "text": "@ryanbader5 he makes me sad:/ look at his hair����", "in_reply_to_status": 683709153425133568, "in_reply_to_user": 2364972428, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2364972428 }}, "user": { "id": 603269704, "name": "Fe Fe the Traplord", "screen_name": "Toddwalsh35", "lang": "en", "location": "Brazil", "create_at": date("2012-06-08"), "description": "I'm just a squirrel tryna nut, bless up. I'm out of highschool fags", "followers_count": 504, "friends_count": 240, "statues_count": 31662 }, "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-01-03T10:00:27.000Z"), "id": 683709521995272192, "text": "This #Nursing #job might be a great fit for you: RN Float Pool (Internal Registry) Tele-Med-Surg-Onc - https://t.co/o4B8INtn3Z #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.0697937,26.5047866"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "Hiring" }}, "user": { "id": 2615559954, "name": "Bethesda Health Jobs", "screen_name": "BH_Jobs", "lang": "en", "location": "null", "create_at": date("2014-07-10"), "description": "Whether you’re a nurse looking for a more challenging position or a clerical prof. looking for great benefits, you’ll find a rewarding career @ Bethesda Health.", "followers_count": 341, "friends_count": 672, "statues_count": 1763 }, "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-01-03T10:00:27.000Z"), "id": 683709521999478784, "text": "Last game of the season. Eagles-Giants. Yes I want the Eagles to lose to get a higher draft pick. Just saying. #PHIvsNYG #FlyEaglesFly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PHIvsNYG", "FlyEaglesFly" }}, "user": { "id": 35994976, "name": "Anthony Miller", "screen_name": "theatrain40", "lang": "en", "location": "Canyon, TX", "create_at": date("2009-04-27"), "description": "Sr. WTAMU Student, KWTS General Manager, WT NBS President, Sports Fanatic, Film Buff, Music Lover (except country lol).", "followers_count": 213, "friends_count": 588, "statues_count": 2980 }, "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-03T10:00:27.000Z"), "id": 683709522016219138, "text": "@TheRakitic according to liberal blacks & msm they are. I respectfully disagree .", "in_reply_to_status": 683705544113139713, "in_reply_to_user": 2764051057, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2764051057 }}, "user": { "id": 1120641384, "name": "Chad Clement", "screen_name": "cclement873", "lang": "en", "location": "Houston, TX.", "create_at": date("2013-01-25"), "description": "Texas conservative ,southern baptist christian, #CruzCrew #Cruz2016 #2A #CCOT #TCOT #NRA #PJNET #StandwithIsrael", "followers_count": 1583, "friends_count": 2247, "statues_count": 4345 }, "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-01-03T10:00:27.000Z"), "id": 683709522100285441, "text": "I need new friends mine are too mean", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 718534723, "name": "The Ashhole", "screen_name": "ash_williamson_", "lang": "en", "location": "Muncie, IN", "create_at": date("2012-07-26"), "description": "Speech Pathology BSU '19", "followers_count": 455, "friends_count": 642, "statues_count": 11612 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbia City, IN", "id": "11438867a3b3fd50", "name": "Columbia City", "place_type": "city", "bounding_box": rectangle("-85.542706,41.127206 -85.415056,41.17957") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18183, "countyName": "Whitley", "cityID": 1814716, "cityName": "Columbia City" } }
+{ "create_at": datetime("2016-01-03T10:00:27.000Z"), "id": 683709522116907008, "text": "I voted Adam Lambert - Another Lonely Night @B104! https://t.co/E5A50wp04G Pwd by @B104's Mike & Steph", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21006970, 21006970 }}, "user": { "id": 1927572523, "name": "MWestwood15", "screen_name": "mjbrn924", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-10-02"), "description": "null", "followers_count": 94, "friends_count": 189, "statues_count": 11640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-03T10:00:27.000Z"), "id": 683709522146283520, "text": "@putemupcdukes ME TOO...from my sick bed", "in_reply_to_status": 683700658109833217, "in_reply_to_user": 69971825, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 69971825 }}, "user": { "id": 132220466, "name": "Oh Mighty Cool One", "screen_name": "BTrustyNO2ATL", "lang": "en", "location": "New Orleans, LA", "create_at": date("2010-04-12"), "description": "Back in the 504 after 10 years in the 404. Host of The Shoutout Show. I gotta dope life, my friends are dope, and I do dope things.", "followers_count": 626, "friends_count": 575, "statues_count": 39580 }, "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-03T10:00:27.000Z"), "id": 683709522716803072, "text": "I'm still waiting on what \"sisters\" I left in the cold. I ll wait... https://t.co/FiIEzPZBkk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 252578331, "name": "Walter L. Hudson Sr.", "screen_name": "Activist30", "lang": "en", "location": "Penns Grove, NJ", "create_at": date("2011-02-15"), "description": "Motivational Speaker & Social Justice Activist. Chairman/Founder of National Awareness Alliance. Leading South Jersey Civil Rights Advocacy & Social Justice.", "followers_count": 1020, "friends_count": 770, "statues_count": 18908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Penns Grove, NJ", "id": "e9c9cf4cc853d786", "name": "Penns Grove", "place_type": "city", "bounding_box": rectangle("-75.477251,39.718202 -75.460398,39.73934") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34033, "countyName": "Salem", "cityID": 3457750, "cityName": "Penns Grove" } }
+{ "create_at": datetime("2016-01-03T10:00:27.000Z"), "id": 683709522720915456, "text": "Can you recommend anyone for this #job? General Utility/Floor Tech - https://t.co/qEGpY5MScI #NewAlbany, IN #Healthcare #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.8241312,38.2856247"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "NewAlbany", "Healthcare", "Hiring", "CareerArc" }}, "user": { "id": 21730633, "name": "LouisvilleHealthJobs", "screen_name": "tmj_sdf_health", "lang": "en", "location": "Louisville, KY", "create_at": date("2009-02-23"), "description": "Follow this account for geo-targeted Healthcare job tweets in Louisville, KY. Need help? Tweet us at @CareerArc!", "followers_count": 468, "friends_count": 322, "statues_count": 217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Albany, IN", "id": "ec5febdaca1534b0", "name": "New Albany", "place_type": "city", "bounding_box": rectangle("-85.884579,38.25708 -85.761144,38.39041") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18043, "countyName": "Floyd", "cityID": 1852326, "cityName": "New Albany" } }
+{ "create_at": datetime("2016-01-03T10:00:27.000Z"), "id": 683709522746068992, "text": "I already pre ordered it https://t.co/qNrMlhNHcG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 60419054, "name": "Ryan", "screen_name": "PresidentRyan", "lang": "en", "location": "null", "create_at": date("2009-07-26"), "description": "I never know what to put here.", "followers_count": 1285, "friends_count": 2036, "statues_count": 113467 }, "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-01-03T10:00:27.000Z"), "id": 683709522930614272, "text": "We're #hiring! Click to apply: PA Hosp Care Spec - https://t.co/mBqo2H27qa #Healthcare #Cleveland, OH #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.6205953,41.5034433"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Healthcare", "Cleveland", "Job", "Jobs" }}, "user": { "id": 2843895727, "name": "ClevelandClinic Jobs", "screen_name": "CleClinicJobs", "lang": "en", "location": "Northeast Ohio", "create_at": date("2014-10-07"), "description": "As one of the nation’s premier hospitals, we offer superior opportunities to those ready for a vital role in the future of medicine providing world class care.", "followers_count": 426, "friends_count": 10, "statues_count": 1448 }, "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-03T10:00:27.000Z"), "id": 683709523089960961, "text": "COUNTDOWN!!! :)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.31973975,34.26004661"), "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": 384, "friends_count": 346, "statues_count": 61466 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-03T10:00:27.000Z"), "id": 683709524125982720, "text": "#WhitePrivilege @ #OregonUnderAttack for #WhiteNationalist #DomesticTerrorist & #TheMedia #remain #silent\n#EPICFAIL https://t.co/K2uAOnyxKN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "WhitePrivilege", "OregonUnderAttack", "WhiteNationalist", "DomesticTerrorist", "TheMedia", "remain", "silent", "EPICFAIL" }}, "user": { "id": 712967096, "name": "MzPeachezCobbLa", "screen_name": "ericsean467", "lang": "en", "location": "null", "create_at": date("2012-07-23"), "description": "ADULTS OnLY! UnLeSS a MuthAFuCKa IS FeeDiN' U, FuckiN' U 0r FINANCIN' U, WHO CAhrEZ what THEY say bout U or me, ShhhhhhhyT...", "followers_count": 481, "friends_count": 1978, "statues_count": 12217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pinole, CA", "id": "2ecc2108e9d5d658", "name": "Pinole", "place_type": "city", "bounding_box": rectangle("-122.318091,37.973198 -122.245122,38.015067") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 657288, "cityName": "Pinole" } }
+{ "create_at": datetime("2016-01-03T10:00:27.000Z"), "id": 683709524155469824, "text": "@PropaneMay nah.", "in_reply_to_status": 683709484745752576, "in_reply_to_user": 257730455, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 257730455 }}, "user": { "id": 277194355, "name": "khalil.", "screen_name": "FlySince95_", "lang": "en", "location": "Atlanta, GA / Miami, FL", "create_at": date("2011-04-04"), "description": "every day is a day closer to a million.. http://flysince95.tumblr.com snapchat: fly_since95", "followers_count": 1341, "friends_count": 765, "statues_count": 141303 }, "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-01-03T10:00:27.000Z"), "id": 683709524394418176, "text": "Music is the way to women's hearts ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 214203517, "name": "Jay Sanders Petate", "screen_name": "jaysanders_p", "lang": "en", "location": "Stockton to the Bay, SFC", "create_at": date("2010-11-10"), "description": "you feat. , ain't shit.", "followers_count": 293, "friends_count": 211, "statues_count": 13981 }, "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-01-03T10:00:28.000Z"), "id": 683709524537036800, "text": "#WISCONSINRAPIDS, WI #Retail #Job: Store Counter Sales at O'Reilly Auto Parts https://t.co/1ki20I092d #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.817695,44.37216"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WISCONSINRAPIDS", "Retail", "Job", "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": 364, "friends_count": 309, "statues_count": 521 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wisconsin Rapids, WI", "id": "25f4e3714f48e177", "name": "Wisconsin Rapids", "place_type": "city", "bounding_box": rectangle("-89.872931,44.3367 -89.744889,44.423607") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55141, "countyName": "Wood", "cityID": 5588200, "cityName": "Wisconsin Rapids" } }
+{ "create_at": datetime("2016-01-03T10:00:28.000Z"), "id": 683709524604223490, "text": "#Eagles #NoPhlyZone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Eagles", "NoPhlyZone" }}, "user": { "id": 39607776, "name": "Michael Bish", "screen_name": "MBish7", "lang": "en", "location": "Richmond, VA", "create_at": date("2009-05-12"), "description": "God, Family, Humbled | 22 | Writer Covering the Philadelphia Eagles for http://Midnightandwhite.com | #Eagles #Flyers #Phillies #Sixers #UVA #NoPhlyZone", "followers_count": 1478, "friends_count": 491, "statues_count": 62611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chester, VA", "id": "8a8fb2a0babe3b8a", "name": "Chester", "place_type": "city", "bounding_box": rectangle("-77.480516,37.307356 -77.400126,37.391516") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51041, "countyName": "Chesterfield", "cityID": 5116096, "cityName": "Chester" } }
+{ "create_at": datetime("2016-01-03T10:00:28.000Z"), "id": 683709524625199104, "text": "Lazy ass https://t.co/HeqFUjDgaf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 399104970, "name": "Pootie Tang", "screen_name": "RonovanMcNabb", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2011-10-26"), "description": "Be a slave at first, or free at last • Jamaican • I like anime • #EaglesNation • #ThunderUp • 12/2❤️", "followers_count": 505, "friends_count": 207, "statues_count": 67811 }, "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-03T10:00:28.000Z"), "id": 683709524813955072, "text": "@craigsturgis or Thursday for that matter", "in_reply_to_status": 683709020469723136, "in_reply_to_user": 14170404, "favorite_count": 0, "coordinate": point("-86.09393606,39.98799301"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14170404 }}, "user": { "id": 75007709, "name": "Heath Matthias", "screen_name": "HMATTHIAS", "lang": "en", "location": "null", "create_at": date("2009-09-17"), "description": "null", "followers_count": 472, "friends_count": 271, "statues_count": 115208 }, "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-01-03T10:00:28.000Z"), "id": 683709524889481216, "text": "#wokeuplikethis https://t.co/QFbCY2ISec", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "wokeuplikethis" }}, "user": { "id": 91415032, "name": "Big Daddy Beefcake", "screen_name": "IAmHaleyMcGuire", "lang": "en", "location": "Portland, ME or Austin, TX", "create_at": date("2009-11-20"), "description": "I like naps and Beyonce.", "followers_count": 566, "friends_count": 117, "statues_count": 6948 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Portland, ME", "id": "2438e517cb050ced", "name": "South Portland", "place_type": "city", "bounding_box": rectangle("-70.349558,43.595844 -70.220983,43.655565") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23005, "countyName": "Cumberland", "cityID": 2371990, "cityName": "South Portland" } }
+{ "create_at": datetime("2016-01-03T10:00:28.000Z"), "id": 683709525094830080, "text": "Can you recommend anyone for this #job? Registered Nurse (RN) - Emergency - https://t.co/UxK7n1WbL6 #Rochester, MI https://t.co/fO7oFCO5D6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.1492553,42.6807334"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Rochester" }}, "user": { "id": 1076984880, "name": "Crittenton Jobs", "screen_name": "CrittentonJobs", "lang": "en", "location": "Rochester, Michigan", "create_at": date("2013-01-10"), "description": "Working for Crittenton Hospital Medical Center is more than just a job. It’s a career. Follow us for current opportunities.", "followers_count": 228, "friends_count": 249, "statues_count": 3317 }, "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-01-03T10:00:28.000Z"), "id": 683709525451341824, "text": "@TahniiniiM booyyyy������������ I'm so so so so dead���� Jesus bless us������������������", "in_reply_to_status": 683709061028642817, "in_reply_to_user": 557437382, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 557437382 }}, "user": { "id": 1123195604, "name": "Christinnaa❤️", "screen_name": "ChrissyCherrye", "lang": "en", "location": "Coldspring Tx", "create_at": date("2013-01-26"), "description": "Snapchat? chrissy_cherrye InstagramChristina.cherrye", "followers_count": 1299, "friends_count": 723, "statues_count": 18953 }, "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-03T10:00:28.000Z"), "id": 683709525459922949, "text": "I'm at H-Mart in Tigard, OR https://t.co/s3aBQ9mRwJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.783049,45.421198"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25140952, "name": "@mariafikany", "screen_name": "mariafikany", "lang": "en", "location": "null", "create_at": date("2009-03-18"), "description": "null", "followers_count": 155, "friends_count": 144, "statues_count": 9614 }, "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-03T10:00:28.000Z"), "id": 683709525476683777, "text": "I Will Tell You The Real But Don't Ask Me About Your Chick Because I Owe U No Loyalty.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 453636714, "name": "Rondo", "screen_name": "Jad_Young", "lang": "en", "location": "Hammond, LA", "create_at": date("2012-01-02"), "description": "23, SC: Jadguar", "followers_count": 940, "friends_count": 493, "statues_count": 21767 }, "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-03T10:00:28.000Z"), "id": 683709525606723585, "text": "#AQUARIUS @ mechanicville-zone3 https://t.co/isyGUZ6MYd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.39687194,33.74198722"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "AQUARIUS" }}, "user": { "id": 242015438, "name": "JTLady", "screen_name": "HOLLYWOODGIRL4", "lang": "en", "location": "Atlanta", "create_at": date("2011-01-23"), "description": "I am a advanced Pure Romance Consultant so book your party with me today and get 10% off and earn up to a 100 in free products", "followers_count": 154, "friends_count": 229, "statues_count": 2906 }, "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-03T10:00:28.000Z"), "id": 683709525623443458, "text": "just accidentally drank nyquil i am so over myself", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 60157064, "name": "mae", "screen_name": "maecagalingan", "lang": "en", "location": "Midtown, Detroit", "create_at": date("2009-07-25"), "description": ":-)-: IG-maecagalingan//SC-maezedong", "followers_count": 1392, "friends_count": 542, "statues_count": 33558 }, "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-03T10:00:28.000Z"), "id": 683709525719814144, "text": "We're #hiring! Click to apply: Staff Accountant- Corporate Accounting - https://t.co/jqjZxVGXKk #Finance #SantaClarita, CA #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.542586,34.3916641"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Finance", "SantaClarita", "Job", "Jobs" }}, "user": { "id": 2237934078, "name": "Princess Cruises", "screen_name": "JobsAtPrincess", "lang": "en", "location": "Santa Clarita, CA ", "create_at": date("2013-12-09"), "description": "Set sail on an exciting new career course with the cruise line that helps you #comebacknew! For non-career-related news and assistance, follow @PrincessCruises.", "followers_count": 271, "friends_count": 249, "statues_count": 155 }, "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-03T10:00:28.000Z"), "id": 683709525765967872, "text": "I would love to show you my #listing at 213 Wolfe Lane #Inman #SC https://t.co/T1IcVTR33m #realestate https://t.co/Fp3LizL0Mz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.080234,35.035648"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "Inman", "SC", "realestate" }}, "user": { "id": 35979233, "name": "LISA EBERSOLE", "screen_name": "lisasells", "lang": "en", "location": "Greenville-Spartanburg", "create_at": date("2009-04-27"), "description": "Wife, Mom, Mimi, Top Producing Realtor®, Piano Player, Singer, Coffee Lover! Call me today for all your real estate needs! For with God ALL things are possible!", "followers_count": 437, "friends_count": 574, "statues_count": 7766 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inman, SC", "id": "1eef22e52ca02fb5", "name": "Inman", "place_type": "city", "bounding_box": rectangle("-82.116735,35.025359 -82.056912,35.071599") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45083, "countyName": "Spartanburg", "cityID": 4535755, "cityName": "Inman" } }
+{ "create_at": datetime("2016-01-03T10:00:28.000Z"), "id": 683709525900132353, "text": "#Sales #Job in #NewYork, NY: Oral Care Consultant - Uptown Manhattan & Bronx at Colgate-Palmolive https://t.co/MJfckVLZ00 #Jobs #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": {{ "Sales", "Job", "NewYork", "Jobs", "Hiring" }}, "user": { "id": 3004246198, "name": "Colgate Jobs - U.S.", "screen_name": "ColgateJobsUS", "lang": "en", "location": "United States", "create_at": date("2015-01-29"), "description": "null", "followers_count": 1142, "friends_count": 30, "statues_count": 113 }, "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-03T10:00:28.000Z"), "id": 683709525946429440, "text": "@SamuelForstved hey sam ik how it feels im exactlly like that believe me but we gotta keep living and remember theres others thisngs than", "in_reply_to_status": 683709043018301440, "in_reply_to_user": 246610959, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 246610959 }}, "user": { "id": 2570010444, "name": "Adriana!!", "screen_name": "JUSTINTEAMO100", "lang": "en", "location": "Guaynabo, Puerto Rico", "create_at": date("2014-06-15"), "description": "@justinbieber followed 20/0ct/15 1:04 what do you mean ? & sorry & ill show you buyPurposeOnItunes get @shots & @falho justinbieber lets make the world better", "followers_count": 2711, "friends_count": 2929, "statues_count": 67287 }, "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-03T10:00:28.000Z"), "id": 683709526009229313, "text": "Watching Jesse Jane break it down. Porn: Business of Pleasure �� #CNBCW #OKC #JesseJane #FortheLoveofMoney ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CNBCW", "OKC", "JesseJane", "FortheLoveofMoney" }}, "user": { "id": 105360853, "name": "David Rambo", "screen_name": "dave_rambo", "lang": "en", "location": "Wichita, KS", "create_at": date("2010-01-15"), "description": "null", "followers_count": 134, "friends_count": 610, "statues_count": 1849 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-03T10:00:28.000Z"), "id": 683709526047100928, "text": "Hey Steelers, you want to hurt the Browns? Lose today #DraftPosition #CominForThatNumberOnePick", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DraftPosition", "CominForThatNumberOnePick" }}, "user": { "id": 248861280, "name": "Matt McDougal", "screen_name": "Maverick__Matt", "lang": "en", "location": "Perry, Canton, OH, USA, Earth.", "create_at": date("2011-02-07"), "description": "Ezekiel 25:17", "followers_count": 538, "friends_count": 268, "statues_count": 19231 }, "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-01-03T10:00:28.000Z"), "id": 683709526399410176, "text": "it's official: I have signed my contract and will be joining NASCAR. I would like to thank my… https://t.co/zG9iVRwwzF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4143372,28.4273472"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 374864101, "name": "Morgan Luibrand", "screen_name": "morganluibrand4", "lang": "en", "location": "null", "create_at": date("2011-09-16"), "description": "ya win some ya lose some", "followers_count": 586, "friends_count": 760, "statues_count": 13652 }, "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-01-03T10:00:28.000Z"), "id": 683709526667870208, "text": "Bless up #BillsMafia #theyneedhelp https://t.co/aY3k31TVGa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BillsMafia", "theyneedhelp" }}, "user": { "id": 480093218, "name": "Luke Pereira", "screen_name": "Luke_Pereira_", "lang": "en", "location": "null", "create_at": date("2012-01-31"), "description": "Current employee of NBC Sports - Bachelor of Arts in Public Relations from Quinnipiac University, minor in Business - Ready For Anything", "followers_count": 502, "friends_count": 613, "statues_count": 5352 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ossining, NY", "id": "462c2d50fa4b9bb9", "name": "Ossining", "place_type": "city", "bounding_box": rectangle("-73.876223,41.136627 -73.813587,41.199919") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3655530, "cityName": "Ossining" } }
+{ "create_at": datetime("2016-01-03T10:00:28.000Z"), "id": 683709526957252608, "text": "@Dkerperin_00 @BReinkemeyer @skewis70 @Cforck10 upper decker?", "in_reply_to_status": 683698143612039173, "in_reply_to_user": 518673937, "favorite_count": 0, "retweet_count": 0, "lang": "sv", "is_retweet": false, "user_mentions": {{ 518673937, 1045975698, 14057402, 162961453 }}, "user": { "id": 482499362, "name": "Trent VanLoo", "screen_name": "shocker2169", "lang": "en", "location": "south Taos", "create_at": date("2012-02-03"), "description": "Everyday is hump day", "followers_count": 174, "friends_count": 538, "statues_count": 12479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jefferson City, MO", "id": "01cace72701072d0", "name": "Jefferson City", "place_type": "city", "bounding_box": rectangle("-92.312792,38.484463 -92.042823,38.620738") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29051, "countyName": "Cole", "cityID": 2937000, "cityName": "Jefferson City" } }
+{ "create_at": datetime("2016-01-03T10:00:28.000Z"), "id": 683709527036801024, "text": "living the life! ������ https://t.co/g38KsATW1L", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2233686324, "name": "kale", "screen_name": "kaylirayy", "lang": "en", "location": "Livonia, MI", "create_at": date("2013-12-06"), "description": "shs '16", "followers_count": 561, "friends_count": 302, "statues_count": 15521 }, "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-01-03T10:00:28.000Z"), "id": 683709527473127424, "text": "I don't get why Calvin Johnson's decline in production isn't being blamed on Joe Lombardi's horrible offense. Dude is still elite", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 57511796, "name": "Kunj Goenka", "screen_name": "GrandmaKunj", "lang": "en", "location": "Number 6", "create_at": date("2009-07-16"), "description": "You can always think of a better Twitter bio than the one you have right now.", "followers_count": 226, "friends_count": 144, "statues_count": 3567 }, "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-01-03T10:00:28.000Z"), "id": 683709527586385920, "text": "One more DDD establishment on the way outta town! #DuckClubSammie (@ Tattooed Moose in Charleston, SC) https://t.co/Xq7bcfcTWJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.94926661,32.81140655"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DuckClubSammie" }}, "user": { "id": 74016038, "name": "Chris Angel", "screen_name": "chrisangel02", "lang": "en", "location": "Fletcher, NC", "create_at": date("2009-09-13"), "description": "I have an amazing wife & two sweet little girls! I am the Business Development Director at @MountainCU in NC & a CUNA CULAC Trustee. Tweets are mine.", "followers_count": 692, "friends_count": 1061, "statues_count": 4550 }, "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-03T10:00:28.000Z"), "id": 683709527632547840, "text": "what's good 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": 78463526, "name": "Cristina Campero", "screen_name": "CamperoCristina", "lang": "en", "location": "null", "create_at": date("2009-09-29"), "description": "null", "followers_count": 680, "friends_count": 709, "statues_count": 12937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Naples, FL", "id": "a3f971b69fefb699", "name": "Naples", "place_type": "city", "bounding_box": rectangle("-81.817216,26.093538 -81.766738,26.211277") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12021, "countyName": "Collier", "cityID": 1247625, "cityName": "Naples" } }
+{ "create_at": datetime("2016-01-03T10:00:28.000Z"), "id": 683709527724666880, "text": "Time you enjoyed wasting was not wasted, right?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 616675970, "name": "Meghan", "screen_name": "MinnesotaMeghan", "lang": "en", "location": "Minnesota", "create_at": date("2012-06-23"), "description": "Zootopia is where I belong. Ginnifer Goodwin is my spirit animal. Adele speaks to my soul. Pure Minnesotan.", "followers_count": 58, "friends_count": 164, "statues_count": 5120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Savage, MN", "id": "aab09ade8f67410a", "name": "Savage", "place_type": "city", "bounding_box": rectangle("-93.39978,44.694941 -93.328217,44.790981") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27139, "countyName": "Scott", "cityID": 2758738, "cityName": "Savage" } }
+{ "create_at": datetime("2016-01-03T10:00:28.000Z"), "id": 683709527745761280, "text": "Wind 4.0 mph WNW. Barometer 30.034 in, Steady. Temperature 30.7 °F. Rain today 0.00 in. Humidity 85%", "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": 287, "statues_count": 7226 }, "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-03T10:00:28.000Z"), "id": 683709527875829760, "text": "I have the best waitress ever. Grew up in Texas and is wearing a Redskins jersey and trashing the Cowboys. I see a big tip!!\n\n#GoSteelers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.8727,34.235685"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GoSteelers" }}, "user": { "id": 1920961190, "name": "Jeff Thomas", "screen_name": "JeffThomas03", "lang": "en", "location": "Pickerington, OH", "create_at": date("2013-09-30"), "description": "Investing and believing in yourself is the most important decision you can make.\n\n#ChicagoCubs \n#Steelers \n#Michigan \n#UNCW\n#CBJ\n\nPsalm 23:1-6", "followers_count": 596, "friends_count": 701, "statues_count": 4080 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carolina Ale House", "id": "07d9ec9a64083000", "name": "Carolina Ale House", "place_type": "poi", "bounding_box": rectangle("-77.87270009999999,34.235684899999995 -77.8727,34.235685") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37129, "countyName": "New Hanover", "cityID": 3774440, "cityName": "Wilmington" } }
+{ "create_at": datetime("2016-01-03T10:00:28.000Z"), "id": 683709527892602885, "text": "That last RedZone countdown was lit.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 227934746, "name": "ESPN_CGeezy", "screen_name": "cgarcia__75", "lang": "en", "location": "Elgin, IL", "create_at": date("2010-12-17"), "description": "#SpeedKills Go @Panthers #KeepPounding Go @LAKings #GoKingsGo Die Hard @Cubs fan", "followers_count": 704, "friends_count": 659, "statues_count": 154169 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elgin, IL", "id": "7c4ae4537997a58f", "name": "Elgin", "place_type": "city", "bounding_box": rectangle("-88.431293,41.986293 -88.189602,42.093586") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17089, "countyName": "Kane", "cityID": 1723074, "cityName": "Elgin" } }
+{ "create_at": datetime("2016-01-03T10:00:28.000Z"), "id": 683709527963734017, "text": "Posting pictures later on ����for you looking ass mfkas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 797196541, "name": "KEYWEEZY✨", "screen_name": "imALWAYS_key", "lang": "en", "location": "null", "create_at": date("2012-09-01"), "description": "living and doing just FINE baby✨ #STUDIOTHRIFT ☺️ #IMPRINT ☺️ •Insta : ksg.__ •Snapchat : Keyionna", "followers_count": 1843, "friends_count": 1185, "statues_count": 65536 }, "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-03T10:00:28.000Z"), "id": 683709528093757440, "text": "We're #hiring! Click to apply: Carhop/Skating Carhop (Server) - https://t.co/yuXnb3vHhq #Hospitality #SONIC #GreersFerry, AR #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.1764986,35.5767108"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Hospitality", "SONIC", "GreersFerry", "Job", "Jobs" }}, "user": { "id": 88003429, "name": "TMJ-AR HRTA Jobs", "screen_name": "tmj_ar_hrta", "lang": "en", "location": "Arkansas", "create_at": date("2009-11-06"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Arkansas Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 329, "friends_count": 281, "statues_count": 2055 }, "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": 5023, "countyName": "Cleburne", "cityID": 528810, "cityName": "Greers Ferry" } }
+{ "create_at": datetime("2016-01-03T10:00:28.000Z"), "id": 683709528374788096, "text": "If you're a #Sales professional in #Westchester, IL, check out this #job: https://t.co/3DEZnBEipk #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.8806738,41.8498339"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "Westchester", "job", "Hiring", "CareerArc" }}, "user": { "id": 22037938, "name": "TMJ-IL-USA Sales Job", "screen_name": "tmj_IL_sales", "lang": "en", "location": "Illinois", "create_at": date("2009-02-26"), "description": "Follow this account for geo-targeted Sales job tweets in Illinois Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 451, "friends_count": 308, "statues_count": 634 }, "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-01-03T10:00:28.000Z"), "id": 683709528446111744, "text": "@AADaddario stay strong, don't let @united get down my the force be with you. #TheForceAwakens", "in_reply_to_status": 683707444137181184, "in_reply_to_user": 87318558, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TheForceAwakens" }}, "user_mentions": {{ 87318558, 260907612 }}, "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": 193, "friends_count": 548, "statues_count": 50920 }, "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-03T10:00:28.000Z"), "id": 683709528471277568, "text": "Today's Sermon is as follows \"How Did We Make It\" Lamentation 3:22-23 @ Eighth Street Missionary Baptist Church", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54107430, "name": "James Curley", "screen_name": "jms_curley", "lang": "en", "location": "Dallas, Tx. - Little Rock, Ar.", "create_at": date("2009-07-05"), "description": "5'8..Single..Philander Smith College '15..Mommas Boy..I love money..I love my cars..Favorite Color=Blue Blacc Burgundy Maroon..#team Lakers..#team elite noel #", "followers_count": 191, "friends_count": 369, "statues_count": 1962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Little Rock, AR", "id": "31bb014b56203c53", "name": "North Little Rock", "place_type": "city", "bounding_box": rectangle("-92.352506,34.735212 -92.161794,34.840962") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 550450, "cityName": "North Little Rock" } }
+{ "create_at": datetime("2016-01-03T10:00:28.000Z"), "id": 683709528664309761, "text": "@SweetSydney108 https://t.co/Xzz62qOQPw", "in_reply_to_status": 683707346208583680, "in_reply_to_user": 278278051, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 278278051 }}, "user": { "id": 278278051, "name": "ՏγԀ Տʟɑʊɢһɫєʀ.", "screen_name": "SweetSydney108", "lang": "en", "location": "Auburn, AL", "create_at": date("2011-04-06"), "description": "are you living, or just existing? #AU19. ★ॐ instagram:@sydneyelycenoel_", "followers_count": 1330, "friends_count": 887, "statues_count": 16611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Harvest, AL", "id": "00fdde687bcae39b", "name": "Harvest", "place_type": "city", "bounding_box": rectangle("-86.766736,34.830651 -86.712516,34.870416") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1089, "countyName": "Madison", "cityID": 133472, "cityName": "Harvest" } }
+{ "create_at": datetime("2016-01-03T10:00:29.000Z"), "id": 683709528857292800, "text": "I will always find the beauty in the most damaged flower... @ Santa Monica Beach https://t.co/9vK7VcSvwX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.50166667,34.015"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 300982502, "name": "Ralph Soto", "screen_name": "RalphisNice", "lang": "en", "location": "Santa Monica, CA", "create_at": date("2011-05-18"), "description": "Officer of the Law, Hates to be Tickled, Ice Cream lover, undefeated freestyle rapper, Lover of females, Hard worker, Nerd, button pusher and I am not a jerk ;)", "followers_count": 56, "friends_count": 231, "statues_count": 5344 }, "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-03T10:00:29.000Z"), "id": 683709528966303745, "text": "2015 Bayside Coach of the Year: James McCormick, North Caroline.\n\nLed Bulldogs to 10-0 regular season and first 2 playoff wins. Congrats!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 733491744, "name": "Spenser Tilus", "screen_name": "STilus47ABC", "lang": "en", "location": "Salisbury, Maryland", "create_at": date("2012-08-02"), "description": "They call me shirt pants. I rock J's. KCCO. I cover sports for 47ABC.", "followers_count": 2144, "friends_count": 594, "statues_count": 23688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salisbury, MD", "id": "00caf39d503a84e9", "name": "Salisbury", "place_type": "city", "bounding_box": rectangle("-75.71412,38.28923 -75.487032,38.431613") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24045, "countyName": "Wicomico", "cityID": 2469925, "cityName": "Salisbury" } }
+{ "create_at": datetime("2016-01-03T10:00:29.000Z"), "id": 683709528974725120, "text": "I spent $300 at kohls last month ohhhh ywahhhh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 860695658, "name": "squid✽", "screen_name": "kohlsyd", "lang": "en", "location": "Kohls ", "create_at": date("2012-10-03"), "description": "As hard as I've tried, I don't know how to not be adorable.", "followers_count": 1686, "friends_count": 897, "statues_count": 30575 }, "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-03T10:00:29.000Z"), "id": 683709529293340673, "text": "@fuckkruben happy birthday!!! ������", "in_reply_to_status": 683707883410735104, "in_reply_to_user": 278724537, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 278724537 }}, "user": { "id": 293061095, "name": "CocoHosie", "screen_name": "M00nface_", "lang": "en", "location": "Greg's Place ", "create_at": date("2011-05-04"), "description": "your favorite counter hoe at High Priestess Piercing & Tattoo", "followers_count": 1937, "friends_count": 368, "statues_count": 46989 }, "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-03T10:00:29.000Z"), "id": 683709529633222660, "text": "@goldenXchange see see and going to the gym doing that you gone have a nice fatty ����", "in_reply_to_status": 683709300867489792, "in_reply_to_user": 321787378, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 321787378 }}, "user": { "id": 972872178, "name": "Blu.", "screen_name": "hey_imbluee", "lang": "en", "location": "Norfolk, VA", "create_at": date("2012-11-26"), "description": "VSU. @emteeee_", "followers_count": 1878, "friends_count": 1583, "statues_count": 24330 }, "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-03T10:00:29.000Z"), "id": 683709529645711360, "text": "Check out my #listing in #Clermont #FL https://t.co/rrUvBNUv19 #realestate #realtor https://t.co/91HHdR9nhd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.7654078,28.4818091"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "Clermont", "FL", "realestate", "realtor" }}, "user": { "id": 314002579, "name": "Patti-Jo Jungreis", "screen_name": "PJJUNGREIS", "lang": "en", "location": "Clermont, Florida", "create_at": date("2011-06-09"), "description": "null", "followers_count": 208, "friends_count": 227, "statues_count": 3841 }, "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": 12069, "countyName": "Lake" } }
+{ "create_at": datetime("2016-01-03T10:00:29.000Z"), "id": 683709529696030721, "text": "So in love with @white_delano", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2524723275 }}, "user": { "id": 496102614, "name": "zayum, zender", "screen_name": "thelittlezender", "lang": "en", "location": "with angelli", "create_at": date("2012-02-18"), "description": "behind every great man is a woman rolling her eyes beauty school wya??? #LiveLukesLegacy @white_delano is the love of my life", "followers_count": 392, "friends_count": 293, "statues_count": 20298 }, "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-03T10:00:29.000Z"), "id": 683709529750650882, "text": "You bitches talk to much!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6447014,41.7945563"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1082047549, "name": "✨BossCbaybee✨", "screen_name": "Cedes_Benzz", "lang": "en", "location": "moetown ", "create_at": date("2013-01-11"), "description": "In Endia We Trust ....... MissingDbaybee like crazy ❤", "followers_count": 931, "friends_count": 841, "statues_count": 16925 }, "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-03T10:00:29.000Z"), "id": 683709529918279680, "text": "Grace mentioned the Tree of Death breaking. I am glad you were not killed or injured by that wayward stage prop that has been troublesome.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 250248914, "name": "Robert Haviland", "screen_name": "LambysPoet", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-02-10"), "description": "Personal poet to the world's most beautiful and talented woman, Brit. Poetry Book http://tinyurl.com/q59vsyd", "followers_count": 300, "friends_count": 445, "statues_count": 187344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-03T10:00:29.000Z"), "id": 683709529956192261, "text": ". https://t.co/Sjwpj3mK4i", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 163474462, "name": "Jeremiah", "screen_name": "SwaggMeOut__", "lang": "en", "location": "Columbus, OH", "create_at": date("2010-07-06"), "description": "SC- swaggme0ut", "followers_count": 646, "friends_count": 135, "statues_count": 44212 }, "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-03T10:00:29.000Z"), "id": 683709529964462080, "text": "This #Healthcare #job might be a great fit for you: Clinical Case Manager I (270) - https://t.co/lFuHDteIq3 #Hiring https://t.co/ekkppn7r0x", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.9355793,39.7350359"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "Hiring" }}, "user": { "id": 4036025892, "name": "MHCD Careers", "screen_name": "MHCD_Careers", "lang": "en", "location": "Denver, CO", "create_at": date("2015-10-27"), "description": "We believe that people can and do recover from mental illness and should have that chance. We hire people dedicated to working toward that goal. Join us!", "followers_count": 10, "friends_count": 2, "statues_count": 48 }, "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-03T10:00:29.000Z"), "id": 683709530035851264, "text": "Yo nunca me kedo atras", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 1210217358, "name": "∀♖", "screen_name": "PardonMyAURA", "lang": "en", "location": "NYC", "create_at": date("2013-02-22"), "description": "⚗ Instagram: @illestauraalive", "followers_count": 503, "friends_count": 165, "statues_count": 10327 }, "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-03T10:00:29.000Z"), "id": 683709530098774017, "text": "Day 2, good luck Rosemont! @ Johns Hopkins University https://t.co/sNYYINPMnP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.61967786,39.3293878"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 41875256, "name": "Carolyn Hughes", "screen_name": "HereComesTreb1e", "lang": "en", "location": "null", "create_at": date("2009-05-22"), "description": "“Sometimes you gotta work a little, so you can ball a lot.” - Tom Haverford #ParksandRec", "followers_count": 571, "friends_count": 446, "statues_count": 23027 }, "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-03T10:00:29.000Z"), "id": 683709530144813057, "text": "Northside Hospital #Nursing #Job: LPN, INFUSION, *ATLANTA SOUTH AREAS (#Atlanta, GA) https://t.co/OiQPoKmRa8 #LPN #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.352759,33.909187"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "Job", "Atlanta", "LPN", "Jobs", "Hiring" }}, "user": { "id": 3161440380, "name": "Northside Careers", "screen_name": "NorthsideCareer", "lang": "en", "location": "null", "create_at": date("2015-04-17"), "description": "Dedicated to providing clinical excellence balanced with compassionate care. Learn how you can make a difference and continue growing your career.", "followers_count": 16, "friends_count": 1, "statues_count": 438 }, "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-01-03T10:00:29.000Z"), "id": 683709530329493504, "text": "Hoy se gana! Let's go #Colts\n\n#nfl #coltsnation #sunday #football #indianapolis #gocolts… https://t.co/hy676JOVZH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.8367,41.5821"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "Colts", "nfl", "coltsnation", "sunday", "football", "indianapolis", "gocolts" }}, "user": { "id": 61742567, "name": "Abel García", "screen_name": "abelgarcialavoz", "lang": "es", "location": "Indiana, United States.", "create_at": date("2009-07-31"), "description": "29, Mexican, Saggitarius. Morning Show Host, Social Media, Marketing & Promotions Director @larazaindiana", "followers_count": 2265, "friends_count": 2000, "statues_count": 27992 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Goshen, IN", "id": "4fbddb5bf6f19735", "name": "Goshen", "place_type": "city", "bounding_box": rectangle("-85.91235,41.515957 -85.770099,41.629644") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18039, "countyName": "Elkhart", "cityID": 1828386, "cityName": "Goshen" } }
+{ "create_at": datetime("2016-01-03T10:00:29.000Z"), "id": 683709530392375296, "text": "Knowing we have school tomorrow is ruining my my day", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1067355505, "name": "K.", "screen_name": "thatshit__KrAY", "lang": "en", "location": "delco", "create_at": date("2013-01-06"), "description": "volleyball. B17CHES. Ridley 17'.", "followers_count": 700, "friends_count": 1105, "statues_count": 21499 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Folsom, PA", "id": "c8d23c1f5d6736ea", "name": "Folsom", "place_type": "city", "bounding_box": rectangle("-75.341968,39.882641 -75.29943,39.916604") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42045, "countyName": "Delaware", "cityID": 4226432, "cityName": "Folsom" } }
+{ "create_at": datetime("2016-01-03T10:00:29.000Z"), "id": 683709530526466048, "text": "https://t.co/HurfwwEKW8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 315468021, "name": "Barbie Doll", "screen_name": "Tripleenaee", "lang": "en", "location": "DFW", "create_at": date("2011-06-11"), "description": "Positive vibes my friends , One with the wind C.mason", "followers_count": 681, "friends_count": 801, "statues_count": 52964 }, "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-03T10:00:29.000Z"), "id": 683709530627149824, "text": "I got on @Deadspin!!! https://t.co/rygZCrkyx8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 13213122 }}, "user": { "id": 322876206, "name": "Mike Wood", "screen_name": "DixnWood", "lang": "en", "location": "Villanova", "create_at": date("2011-06-23"), "description": "Slightly Above Above Average. No one will ever doubt my GRIT or COMPETE. I am the King in the North.", "followers_count": 272, "friends_count": 418, "statues_count": 5503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orchard Park, NY", "id": "00f0112125d013dc", "name": "Orchard Park", "place_type": "city", "bounding_box": rectangle("-78.797966,42.724114 -78.696772,42.811289") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3655266, "cityName": "Orchard Park" } }
+{ "create_at": datetime("2016-01-03T10:00:29.000Z"), "id": 683709530631487489, "text": "\"Relax & Ride\" �� @haleyelizabethh @csteinruck15 @leeshdatg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 472362889, 421997642, 131707602 }}, "user": { "id": 2472419500, "name": "Ⓜalt", "screen_name": "malt_shakee", "lang": "en", "location": "snapchat: malt_shakee", "create_at": date("2014-04-09"), "description": "Hi, my name is Junie B Jones! The B stands for BACK TO THIS BITCH THAT HAD A LOT TO SAY ABOUT ME THE OTHER DAY IN THE PRESS MILEY WHAT'S GOOD", "followers_count": 464, "friends_count": 312, "statues_count": 6511 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seneca Falls, NY", "id": "76042dbd41998d7a", "name": "Seneca Falls", "place_type": "city", "bounding_box": rectangle("-76.820054,42.887235 -76.776333,42.929662") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36099, "countyName": "Seneca", "cityID": 3666322, "cityName": "Seneca Falls" } }
+{ "create_at": datetime("2016-01-03T10:00:29.000Z"), "id": 683709530752987137, "text": "When it snows take cool photos. #BRCB https://t.co/0o5O7C8fFK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BRCB" }}, "user": { "id": 2211966072, "name": "Gavin Johnson™", "screen_name": "GavinJohnson245", "lang": "en", "location": "La Center, WA", "create_at": date("2013-11-23"), "description": "21. Probably making your coffee somewhere in the woods. Black Rock Coffee Bar Employee SC: gavin245 IG: gavin_johnson245", "followers_count": 741, "friends_count": 545, "statues_count": 9322 }, "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-01-03T10:00:29.000Z"), "id": 683709530820182016, "text": "Towels are ready! https://t.co/6oyTOCd2Mc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2865626862, "name": "Kelly Webster", "screen_name": "kellyweb99", "lang": "en", "location": "Rochester, NY", "create_at": date("2014-10-19"), "description": "Die Hard Steelers fan, Love every thing Pittsburgh! can't wait for the day to move to the Burgh!", "followers_count": 236, "friends_count": 352, "statues_count": 1571 }, "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-03T10:00:29.000Z"), "id": 683709530903941120, "text": "@Eli_Pizza_Queen crush is gonna be awesome����", "in_reply_to_status": 683707929359323136, "in_reply_to_user": 2327949895, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2327949895 }}, "user": { "id": 392327777, "name": "Skyler Cisco", "screen_name": "Skyler_Cisco", "lang": "en", "location": "Basspod", "create_at": date("2011-10-16"), "description": "CO♊️ @Eli_Pizza_Queen I like how she headbangs", "followers_count": 692, "friends_count": 373, "statues_count": 9089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Silverthorne, CO", "id": "01c30b4e60422ea0", "name": "Silverthorne", "place_type": "city", "bounding_box": rectangle("-106.111197,39.610311 -106.058128,39.674734") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8117, "countyName": "Summit", "cityID": 870525, "cityName": "Silverthorne" } }
+{ "create_at": datetime("2016-01-03T10:00:29.000Z"), "id": 683709531025707008, "text": "Lunch time! Smoked salmon with cream cheese #sushi https://t.co/HWCxcBnkN3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sushi" }}, "user": { "id": 349034018, "name": "Shel Burkes", "screen_name": "Shelvasha", "lang": "en", "location": "Charlotte, NC", "create_at": date("2011-08-05"), "description": "null", "followers_count": 68, "friends_count": 77, "statues_count": 567 }, "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-03T10:00:29.000Z"), "id": 683709531570962432, "text": "If you look at my banner photo, between Ennis and Eichel is a grumpy man����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2202001548, "name": "Chakaoofka", "screen_name": "AllisonTaylorr", "lang": "en", "location": "Shia LaBeoufs Heart", "create_at": date("2013-11-18"), "description": "Hi, My name is Chakaoofka and my dad knows God.", "followers_count": 277, "friends_count": 268, "statues_count": 3516 }, "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-01-03T10:00:29.000Z"), "id": 683709531625410561, "text": "@joshgriggs000 but maybe it won't. I hope it doesn't melt", "in_reply_to_status": 683708955252543488, "in_reply_to_user": 2824018913, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2824018913 }}, "user": { "id": 807763219, "name": "Mitchell Smith", "screen_name": "MitchSmith24", "lang": "en", "location": "Washington, USA", "create_at": date("2012-09-06"), "description": "| P | | CF | Skyview Varsity Baseball #24 senior @ Skyview. @EmmaYasson. ❤️", "followers_count": 612, "friends_count": 1390, "statues_count": 5731 }, "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-01-03T10:00:29.000Z"), "id": 683709531667460096, "text": "Drinking a Bud Light Lime by @AnheuserBusch @ Boca Grove Country Club — https://t.co/igry1HYvcO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.1633,26.3609"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1435710336 }}, "user": { "id": 227697746, "name": "Black Spanish", "screen_name": "BlackSpanishT", "lang": "en", "location": "null", "create_at": date("2010-12-17"), "description": "null", "followers_count": 88, "friends_count": 114, "statues_count": 2007 }, "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-03T10:00:29.000Z"), "id": 683709532355358720, "text": "Driving so slow but bk is from Texas ~", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 922230127, "name": "falcao", "screen_name": "Kevinfalcao_NYR", "lang": "en", "location": "null", "create_at": date("2012-11-02"), "description": "21/ NYR", "followers_count": 826, "friends_count": 538, "statues_count": 19616 }, "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-03T10:00:29.000Z"), "id": 683709532707487744, "text": "I kinda want to fo 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": 172809674, "name": "Eleanor Guerra", "screen_name": "EleanorGuerra", "lang": "en", "location": "null", "create_at": date("2010-07-30"), "description": "somebody save me.", "followers_count": 119, "friends_count": 523, "statues_count": 7961 }, "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-03T10:00:29.000Z"), "id": 683709532816674821, "text": "Made it back in time for more friends, more football, more beer, more cigars (@ American Legion - Milan) https://t.co/ZbZX7yv0Po", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.12386492,39.13407968"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 95810893, "name": "Matt Henrich", "screen_name": "SleepingRust", "lang": "en", "location": "ÜT: 39.161546,-84.608612", "create_at": date("2009-12-09"), "description": "Changing the world with nothing more than a cigar and an attitude", "followers_count": 195, "friends_count": 276, "statues_count": 7013 }, "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": 18137, "countyName": "Ripley", "cityID": 1849266, "cityName": "Milan" } }
+{ "create_at": datetime("2016-01-03T10:00:30.000Z"), "id": 683709532963487745, "text": "this looks heavenly https://t.co/jmsc4CMYlJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 261817713, "name": "graceland", "screen_name": "_graceramsey", "lang": "en", "location": "w/ jordan", "create_at": date("2011-03-06"), "description": "mw sabres | fhs volleyball", "followers_count": 724, "friends_count": 327, "statues_count": 8275 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Findlay, OH", "id": "20a30e29568f7ddc", "name": "Findlay", "place_type": "city", "bounding_box": rectangle("-83.699081,40.993401 -83.568726,41.116626") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39063, "countyName": "Hancock", "cityID": 3927048, "cityName": "Findlay" } }
+{ "create_at": datetime("2016-01-03T10:00:30.000Z"), "id": 683709533353410561, "text": "Interested in a #Hospitality #job near #Riverside, MO? This could be a great fit: https://t.co/kSuoBsi10m #security #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.6243649,39.162493"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Riverside", "security", "Hiring" }}, "user": { "id": 1725772980, "name": "Argosy Casino KC", "screen_name": "ArgosyKC_Jobs", "lang": "en", "location": "null", "create_at": date("2013-09-03"), "description": "A Penn National Gaming property, Argosy Casino Hotel & Spa offers gaming excitement, luxury hotel rooms, fine dining and a rejuvenating spa.", "followers_count": 179, "friends_count": 137, "statues_count": 77 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, MO", "id": "f8e9521355ca84c2", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-94.659895,39.154401 -94.601419,39.189295") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29165, "countyName": "Platte", "cityID": 2962156, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-01-03T10:00:30.000Z"), "id": 683709533714251779, "text": "@SeanMonaghanPDX I'll NEVER defend that flag. But always love my home.", "in_reply_to_status": 683708771898507265, "in_reply_to_user": 34104619, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 34104619 }}, "user": { "id": 18110050, "name": "rubyc⭐️kes", "screen_name": "pjolstead", "lang": "en", "location": "45.424313,-122.649029", "create_at": date("2008-12-13"), "description": "Dad, Husband, Timbers, Ducks. #RCTID", "followers_count": 338, "friends_count": 822, "statues_count": 11537 }, "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-03T10:00:30.000Z"), "id": 683709533722525696, "text": "Always been the type to keep 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": 177331285, "name": "Bri Baby ❤️", "screen_name": "iLoveBriauna", "lang": "en", "location": "UA '18", "create_at": date("2010-08-11"), "description": "RIP K'wan ✨", "followers_count": 572, "friends_count": 485, "statues_count": 11753 }, "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-01-03T10:00:30.000Z"), "id": 683709533760405504, "text": "�������� https://t.co/PrJjHQKzK1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 153653592, "name": "TyeeJiggyy", "screen_name": "SlimmJiggyy", "lang": "en", "location": "Frostburg '16", "create_at": date("2010-06-08"), "description": "Grinding.", "followers_count": 1730, "friends_count": 1249, "statues_count": 103527 }, "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-03T10:00:30.000Z"), "id": 683709534137876480, "text": "@itssbrookee .... ��", "in_reply_to_status": 683709237877452800, "in_reply_to_user": 538785807, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 538785807 }}, "user": { "id": 485219034, "name": "White Thugga ✊", "screen_name": "JayJayKing_", "lang": "en", "location": "19 • #RIPScottMoos", "create_at": date("2012-02-06"), "description": "welcome to Gangster 101", "followers_count": 1068, "friends_count": 522, "statues_count": 123257 }, "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-01-03T10:00:30.000Z"), "id": 683709534443995136, "text": "Dab https://t.co/IJrRLXUj6K", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 281096773, "name": "Alex Lange", "screen_name": "Lange_17", "lang": "en", "location": "KC↔️BR", "create_at": date("2011-04-12"), "description": "Joshua 1:9 LSU Baseball #35", "followers_count": 5235, "friends_count": 387, "statues_count": 6951 }, "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-03T10:00:30.000Z"), "id": 683709534649491456, "text": "Quick list - 7 Habits of Highly Emotionally Intelligent People https://t.co/xtDFXfACRc by Harvey Deutschendorf via @FastCompany", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2735591 }}, "user": { "id": 2284147364, "name": "Shonna Dorsey", "screen_name": "shonna_dorsey", "lang": "en", "location": "Omaha, NE", "create_at": date("2014-01-09"), "description": "Omaha native, Interface Web School Managing Dir and cofounder. I've made it both my personal and business mission to help change lives for the better.", "followers_count": 978, "friends_count": 681, "statues_count": 888 }, "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-03T10:00:30.000Z"), "id": 683709534779478016, "text": "might dye my hair today but I'm scared it won't turn out good ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011079797, "name": "michelle.", "screen_name": "michelle___30", "lang": "en", "location": "null", "create_at": date("2015-02-01"), "description": "Gerardo Rivera❤️", "followers_count": 153, "friends_count": 206, "statues_count": 1663 }, "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-01-03T10:00:30.000Z"), "id": 683709534796316672, "text": "And it's stillllll #CowboysNation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CowboysNation" }}, "user": { "id": 606393935, "name": "Elmer Moreno", "screen_name": "elmerrbamf", "lang": "en", "location": "DTX//SMTX", "create_at": date("2012-06-12"), "description": "null", "followers_count": 227, "friends_count": 205, "statues_count": 12726 }, "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-03T10:00:30.000Z"), "id": 683709534817222656, "text": "This #ProjectMgmt #job might be a great fit for you: Lead Programmer Analyst - https://t.co/qOig8g7yZR #KingofPrussia, PA #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.3960211,40.0892746"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ProjectMgmt", "job", "KingofPrussia", "Hiring" }}, "user": { "id": 22393668, "name": "TMJ-PHL IT PM Jobs", "screen_name": "tmj_phl_itpm", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2009-03-01"), "description": "Follow this account for geo-targeted IT-PM/BA job tweets in Philadelphia, PA. Need help? Tweet us at @CareerArc!", "followers_count": 248, "friends_count": 201, "statues_count": 46 }, "place": { "country": "United States", "country_code": "United States", "full_name": "King of Prussia, PA", "id": "23b12afb6fe7507a", "name": "King of Prussia", "place_type": "city", "bounding_box": rectangle("-75.420016,40.071936 -75.343479,40.117633") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42091, "countyName": "Montgomery", "cityID": 4239736, "cityName": "King of Prussia" } }
+{ "create_at": datetime("2016-01-03T10:00:30.000Z"), "id": 683709534834049026, "text": "#TexansAppreciation https://t.co/x6FKLPCmMF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "TexansAppreciation" }}, "user": { "id": 756441366, "name": "ashley michelle", "screen_name": "gypsy_ash", "lang": "en", "location": "LC/Stephenville, TX", "create_at": date("2012-08-13"), "description": "[TSU '16 alpha gam]", "followers_count": 512, "friends_count": 412, "statues_count": 15130 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-03T10:00:30.000Z"), "id": 683709534850830336, "text": "CHECK OUT THIS DOPE MERCH! https://t.co/8392Lp03p4 https://t.co/24EUDrtW8I", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157141380, "name": "Skyhousemusic", "screen_name": "Skyhousemusic", "lang": "en", "location": "Portland, OR", "create_at": date("2010-06-18"), "description": "The musical mothership of Harris Da Rockin' MC & Bets Pott, The co-captains of Starship Skyhouse. Let's cybercruise through Alternative R&B heaven... PEACE.", "followers_count": 3628, "friends_count": 3602, "statues_count": 55187 }, "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-03T10:00:30.000Z"), "id": 683709534964178945, "text": "@aguierra ah ok verdade to aqui passa tudo atrasado", "in_reply_to_status": 683709402361245697, "in_reply_to_user": 1656476010, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user_mentions": {{ 1656476010 }}, "user": { "id": 167205262, "name": "Morena Tropicana™", "screen_name": "lulliromero", "lang": "pt", "location": "toca gando e andando", "create_at": date("2010-07-15"), "description": "babuinos bobocas balbuciando em bando", "followers_count": 2885, "friends_count": 411, "statues_count": 137165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland Beach, FL", "id": "4a29c5d001f991a6", "name": "Highland Beach", "place_type": "city", "bounding_box": rectangle("-80.070171,26.39072 -80.062653,26.426347") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1230200, "cityName": "Highland Beach" } }
+{ "create_at": datetime("2016-01-03T10:00:30.000Z"), "id": 683709535010226176, "text": "@jacobsartorius oh 5 OMG UH UHH ��", "in_reply_to_status": -1, "in_reply_to_user": 3579012568, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3579012568 }}, "user": { "id": 2790514979, "name": "sofia is happy af", "screen_name": "idgafConnors", "lang": "en", "location": "Arkansas || 3/5Quad", "create_at": date("2014-09-28"), "description": "trc:)", "followers_count": 1093, "friends_count": 312, "statues_count": 26583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Farmington, AR", "id": "da25141980d997de", "name": "Farmington", "place_type": "city", "bounding_box": rectangle("-94.285526,36.000016 -94.222787,36.05422") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5143, "countyName": "Washington", "cityID": 523170, "cityName": "Farmington" } }
+{ "create_at": datetime("2016-01-03T10:00:30.000Z"), "id": 683709535110955008, "text": "Just posted a photo @ Battle Creek Paintball and Airsoft Fields https://t.co/6Q3N0ffkEX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.38402211,41.03792773"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 590541871, "name": "WoLFJoNeS", "screen_name": "WoLFpAcLES", "lang": "en", "location": "L.E.S (lower east side ) ", "create_at": date("2012-05-25"), "description": "null", "followers_count": 79, "friends_count": 317, "statues_count": 1477 }, "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": 34027, "countyName": "Morris" } }
+{ "create_at": datetime("2016-01-03T10:00:30.000Z"), "id": 683709535266066432, "text": "Cintas Corporation #Sales : Assistant Route Service Sales Representative... (#Normal, IL) https://t.co/1z3NZaxord https://t.co/OAv5IYSGhg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.9906312,40.5142026"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "Sales", "Normal" }}, "user": { "id": 78995297, "name": "Cintas Careers", "screen_name": "CintasCareers", "lang": "en", "location": "null", "create_at": date("2009-10-01"), "description": "Looking for a career, and not just another job? Consider Cintas, where the spirit is the difference. Follow us for our latest career opportunities.", "followers_count": 261, "friends_count": 2, "statues_count": 9505 }, "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-03T10:00:30.000Z"), "id": 683709535433928704, "text": "цветы,лето https://t.co/RBIYtHKTjy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-123.01200803,37.6994837"), "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": 4, "friends_count": 0, "statues_count": 9857 }, "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-03T10:00:30.000Z"), "id": 683709535576440832, "text": "@chefroncec they robbed Tony Romo last year league MVP facts u can't change that facts I give u facts", "in_reply_to_status": 683708914790084608, "in_reply_to_user": 249405405, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 249405405 }}, "user": { "id": 461459323, "name": "Gary Borror", "screen_name": "gborror1972", "lang": "en", "location": "null", "create_at": date("2012-01-11"), "description": "null", "followers_count": 65, "friends_count": 95, "statues_count": 14336 }, "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-03T10:00:30.000Z"), "id": 683709535622688768, "text": "@claynmiddleton @Pamela_O_Plays I agree that we should do more to secure borders but we have 17 Intel angencies costing 8.5 bill a year.", "in_reply_to_status": 674618901976731648, "in_reply_to_user": 283499967, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 283499967, 389768752 }}, "user": { "id": 2998164052, "name": "Michael Bottella Jr", "screen_name": "MWBoJr", "lang": "en", "location": "Tampa, FL", "create_at": date("2015-01-25"), "description": "Singer dude/tenor vocalist, musician man, seer of sights, atheist, wrestling fan, and all around cool cat", "followers_count": 215, "friends_count": 288, "statues_count": 2158 }, "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-03T10:00:30.000Z"), "id": 683709535714963458, "text": "Browns win toss, defers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 161802070, "name": "Chris Bradford", "screen_name": "BCTBradford", "lang": "en", "location": "Pittsburgh-ish, Pa.", "create_at": date("2010-07-01"), "description": "Official Twitter account of NFL/Steelers writer at Beaver County Times. It's not the size of the Twitter following, it's how you use it.", "followers_count": 1362, "friends_count": 558, "statues_count": 15056 }, "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-03T10:00:30.000Z"), "id": 683709535803031553, "text": "Making a Murderer is taking over 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": 393557669, "name": "»» kels ««", "screen_name": "kcrispyy", "lang": "en", "location": "null", "create_at": date("2011-10-18"), "description": "gotta enjoy the little things", "followers_count": 688, "friends_count": 344, "statues_count": 18024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgetown, KY", "id": "000574562a5d9f4f", "name": "Georgetown", "place_type": "city", "bounding_box": rectangle("-84.599917,38.176888 -84.514815,38.273257") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21209, "countyName": "Scott", "cityID": 2130700, "cityName": "Georgetown" } }
+{ "create_at": datetime("2016-01-03T10:00:30.000Z"), "id": 683709536000028674, "text": "@Miss_BellaBooo you're welcome��������", "in_reply_to_status": 683563730299305984, "in_reply_to_user": 2339549058, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2339549058 }}, "user": { "id": 492882637, "name": "Adrianna", "screen_name": "Adrianna_Rhena_", "lang": "en", "location": "null", "create_at": date("2012-02-14"), "description": "snap: adriannagrimes", "followers_count": 572, "friends_count": 463, "statues_count": 7686 }, "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-03T10:00:30.000Z"), "id": 683709536067268609, "text": "Still can't get over my excitement for my wonderful friend @katiederees and her new FIANCÉ @Eric_Attard. Lots of love to you both! ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 101099703, 1107877292 }}, "user": { "id": 30085673, "name": "Marissa McConnell", "screen_name": "MarissaMcConn", "lang": "en", "location": "Ferndale, MI", "create_at": date("2009-04-09"), "description": "24. Student at Douglas J Aveda Institute. Catholic. Music. Foodie. Fitness. A life lived for art is never a life wasted.", "followers_count": 616, "friends_count": 1054, "statues_count": 18379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Royal Oak, MI", "id": "47cfe29f57708110", "name": "Royal Oak", "place_type": "city", "bounding_box": rectangle("-83.204418,42.474131 -83.111076,42.540305") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2670040, "cityName": "Royal Oak" } }
+{ "create_at": datetime("2016-01-03T10:00:30.000Z"), "id": 683709536146857984, "text": "@2keyz_init @madeyamiss2 @iHurdle21 @SuperHotDaddy @isaiah_ripst3r @WrightNoLeft @highsavage so what time? 1?", "in_reply_to_status": 683701496114855936, "in_reply_to_user": 728861628, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 728861628, 422073363, 448007224, 391003723, 1253647039, 331491465, 1643156256 }}, "user": { "id": 387774109, "name": "6'5©ertifiedJumpGod", "screen_name": "TripleJumpGod", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-10-09"), "description": "SA✈️TAMUK✈️MT.SAC#82", "followers_count": 1641, "friends_count": 2026, "statues_count": 44086 }, "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-03T10:00:30.000Z"), "id": 683709536386023424, "text": "2898 : 10", "in_reply_to_status": 683706745374568448, "in_reply_to_user": 2507576581, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2507576581, "name": "Ziyad Abdullah", "screen_name": "ziyadxd", "lang": "en", "location": "استغفرالله", "create_at": date("2014-05-19"), "description": "فَلا نَامَتْ أعْيُنُ الجُبَنَاء", "followers_count": 454, "friends_count": 178, "statues_count": 39134 }, "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-03T10:00:30.000Z"), "id": 683709536604143616, "text": "@PanthersDiehard @StrikeTwiceTB and MSL did request one in like 09 but stayed until he requested again", "in_reply_to_status": 683709324527403008, "in_reply_to_user": 1892640048, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1892640048, 2259407592 }}, "user": { "id": 2413244882, "name": "Santi", "screen_name": "TBLDiehard", "lang": "en", "location": "Gainesville, FL", "create_at": date("2014-03-26"), "description": "Bucs Rays Bolts Magic Gators Arsenal", "followers_count": 1357, "friends_count": 1146, "statues_count": 38196 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tarpon Springs, FL", "id": "4ad609c5e10fc033", "name": "Tarpon Springs", "place_type": "city", "bounding_box": rectangle("-82.806995,28.118339 -82.718876,28.173326") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1271150, "cityName": "Tarpon Springs" } }
+{ "create_at": datetime("2016-01-03T10:00:30.000Z"), "id": 683709536646119424, "text": "10/10 YouTube video. Would recommend https://t.co/HfWwak0VAn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 80974571, "name": "emily", "screen_name": "intergalacticem", "lang": "en", "location": " ☽", "create_at": date("2009-10-08"), "description": "just wanna see what spring is like on Jupiter and Mars @archambaullt", "followers_count": 618, "friends_count": 345, "statues_count": 9134 }, "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-03T10:00:30.000Z"), "id": 683709536696418305, "text": "@CodyPrell_ @elizabethcooI scratch an sniff stickers /:", "in_reply_to_status": 683695483722678272, "in_reply_to_user": 1416937046, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1416937046, 3180012522 }}, "user": { "id": 2891345921, "name": "✞", "screen_name": "bananarayne", "lang": "en", "location": "null", "create_at": date("2014-11-24"), "description": "null", "followers_count": 335, "friends_count": 105, "statues_count": 9896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westlake, OH", "id": "abcede5a546a90a3", "name": "Westlake", "place_type": "city", "bounding_box": rectangle("-81.970255,41.41104 -81.872904,41.478131") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3983622, "cityName": "Westlake" } }
+{ "create_at": datetime("2016-01-03T10:00:30.000Z"), "id": 683709536851636224, "text": "@HernandezPaaola yo todos los dias����", "in_reply_to_status": 683590091374080000, "in_reply_to_user": 2459390580, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 2459390580 }}, "user": { "id": 142923968, "name": "BarbiePortillo", "screen_name": "barbie_portillo", "lang": "es", "location": "San Antonio, TX", "create_at": date("2010-05-11"), "description": "Together we stand • Reina del Club Pumas 2015", "followers_count": 1079, "friends_count": 871, "statues_count": 14120 }, "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-01-03T10:00:30.000Z"), "id": 683709536922767360, "text": "https://t.co/1jYMyB3EFm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3184374859, "name": "Perry Fedorow", "screen_name": "FedorowPerry", "lang": "en", "location": "null", "create_at": date("2015-05-03"), "description": "Men Anything goes Photography. I love muscular men and I really like a mans chest and nipples oh and a nice butt. The AV picture is me. Adult content be over 18", "followers_count": 7657, "friends_count": 6992, "statues_count": 3859 }, "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-03T10:00:30.000Z"), "id": 683709536927150080, "text": "Last game of the season n I'm still cheering for my boys! Let's do this! #wrapitup… https://t.co/7ZRbAC4egA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-106.423,31.7902"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wrapitup" }}, "user": { "id": 1216597771, "name": "Cynthia Bernal", "screen_name": "cynber2013", "lang": "en", "location": "null", "create_at": date("2013-02-24"), "description": "null", "followers_count": 21, "friends_count": 130, "statues_count": 583 }, "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-03T10:00:30.000Z"), "id": 683709537027674113, "text": "#tweetavideothatyoucantexplain https://t.co/3mDQkcILSY", "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": 3058931028, "name": "Hayden Guttu", "screen_name": "HaydenGuttu", "lang": "en", "location": "Kuna, ID", "create_at": date("2015-03-02"), "description": "Alex McKenzie 07-25-14 Baseball is life! Snapchat: gutty08", "followers_count": 481, "friends_count": 745, "statues_count": 3888 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kuna, ID", "id": "e96d79ee36026a02", "name": "Kuna", "place_type": "city", "bounding_box": rectangle("-116.453164,43.473902 -116.398828,43.521123") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16001, "countyName": "Ada", "cityID": 1644290, "cityName": "Kuna" } }
+{ "create_at": datetime("2016-01-03T10:00:31.000Z"), "id": 683709537115766784, "text": "@procraftinator @tedder42 I took Jimmy out and set him down on the sidewalk. He mewed pitifully a few times and went for the door.", "in_reply_to_status": 683707607199027200, "in_reply_to_user": 18278452, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18278452, 14667502 }}, "user": { "id": 56893, "name": "(╯°□°)╯︵ uǝɥɔʇıʞ@", "screen_name": "kitchen", "lang": "en", "location": "Portland", "create_at": date("2006-12-10"), "description": "cats, corgis, bikes, yak barbering", "followers_count": 997, "friends_count": 274, "statues_count": 13862 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kerns, Portland", "id": "1b425015f1e0377d", "name": "Kerns", "place_type": "neighborhood", "bounding_box": rectangle("-122.669599,45.519297 -122.62988,45.534629") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-03T10:00:31.000Z"), "id": 683709537291874306, "text": "#Hospitality in #Seattle, WA: Cook II – Nutrition Services, Swedish Cherry Hill (0.5... at Swedish Medical Center https://t.co/BDIDqdFVMl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.33,47.61"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "Seattle" }}, "user": { "id": 117387498, "name": "Swedish Careers", "screen_name": "SwedishCareers", "lang": "en", "location": "Seattle, Washington, USA", "create_at": date("2010-02-25"), "description": "Where you work matters. \r\nFor 100 years, employees at Swedish have provided the area's best and most comprehensive health care.", "followers_count": 419, "friends_count": 260, "statues_count": 1284 }, "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-03T10:00:31.000Z"), "id": 683709537921019904, "text": "I'm having a bad hair day", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 447659568, "name": "Baby✨", "screen_name": "Lanttee_", "lang": "en", "location": "null", "create_at": date("2011-12-26"), "description": "I'm like a man smoking at a gas station", "followers_count": 741, "friends_count": 654, "statues_count": 60272 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-03T10:00:31.000Z"), "id": 683709537967292416, "text": "thank you!! https://t.co/Wkt497vuM2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 504032666, "name": "aℓ", "screen_name": "allieleigh25", "lang": "en", "location": "wardamneagle", "create_at": date("2012-02-25"), "description": "if you don't like Kevin Gates, chances are I don't like you", "followers_count": 655, "friends_count": 160, "statues_count": 21319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dalton, GA", "id": "85ee217f155e3ad8", "name": "Dalton", "place_type": "city", "bounding_box": rectangle("-85.037178,34.701611 -84.914986,34.833507") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13313, "countyName": "Whitfield", "cityID": 1321380, "cityName": "Dalton" } }
+{ "create_at": datetime("2016-01-03T10:00:31.000Z"), "id": 683709538000879616, "text": "Made Instagram private because all these XXX people kept liking posts. Now they all send me follow requests", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165879451, "name": "Cap Carey", "screen_name": "CapCareyWDT", "lang": "en", "location": "44.593951,-75.164961", "create_at": date("2010-07-12"), "description": "Cover Clarkson men's hockey and Clarkson/St. Lawrence women's hockey for Watertown Daily Times. University of Arizona alum, Southern California native.", "followers_count": 942, "friends_count": 1331, "statues_count": 40732 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Canton, NY", "id": "08db31d47a830ab4", "name": "Canton", "place_type": "city", "bounding_box": rectangle("-75.193366,44.583353 -75.117515,44.61458") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36089, "countyName": "St. Lawrence", "cityID": 3612331, "cityName": "Canton" } }
+{ "create_at": datetime("2016-01-03T10:00:31.000Z"), "id": 683709538055417862, "text": "it feels so good to be home ❤️ #LoveAmerica #NoMoreIndianFood ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LoveAmerica", "NoMoreIndianFood" }}, "user": { "id": 3319656085, "name": "Nida Mandani", "screen_name": "mandani_nida", "lang": "en", "location": "null", "create_at": date("2015-08-18"), "description": "null", "followers_count": 265, "friends_count": 316, "statues_count": 281 }, "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-03T10:00:31.000Z"), "id": 683709538264977408, "text": "Jets trash", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 276183689, "name": "Haydan Sawyer", "screen_name": "ILsuperstar4", "lang": "en", "location": "704 charlotte born and raised ", "create_at": date("2011-04-02"), "description": "paper.chasing.always.", "followers_count": 1370, "friends_count": 971, "statues_count": 98265 }, "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-03T10:00:31.000Z"), "id": 683709538701324289, "text": "Can't wait to see Allison tomorrow ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 256273211, "name": "☯ Cass ☯", "screen_name": "cassidybloom", "lang": "en", "location": "null", "create_at": date("2011-02-22"), "description": "Μην τα παρατάτε. IX.XXIII.MMXV", "followers_count": 941, "friends_count": 790, "statues_count": 38340 }, "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-03T10:00:31.000Z"), "id": 683709538814431232, "text": "Aye and everything is as it should be.��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4063859714, "name": "YungSavvHippie", "screen_name": "savvhippie", "lang": "en", "location": "LA↔️BAY", "create_at": date("2015-10-29"), "description": "You're living a lie & I'm dying to fly. ✨♋️", "followers_count": 287, "friends_count": 471, "statues_count": 1347 }, "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-03T10:00:31.000Z"), "id": 683709538827124737, "text": "Jets are getting into the playoffs mark my words", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 311039521, "name": "Victoria", "screen_name": "imnotickyvicky", "lang": "en", "location": "Bronx, NY", "create_at": date("2011-06-04"), "description": "Trust in the Lord with all your heart and lean not on your own understanding; in all your ways submit to him, and he will make your paths straight ✞", "followers_count": 506, "friends_count": 271, "statues_count": 30268 }, "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-03T10:00:31.000Z"), "id": 683709538889908224, "text": "Want to work in #NASHVILLE, GA? View our latest opening: https://t.co/ee1vvBn2fa #Retail #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.2172113,31.1955121"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NASHVILLE", "Retail", "Job", "Jobs", "Hiring" }}, "user": { "id": 2706847724, "name": "Harveys Jobs", "screen_name": "HarveysJobs", "lang": "en", "location": "null", "create_at": date("2014-08-04"), "description": "null", "followers_count": 1, "friends_count": 0, "statues_count": 3108 }, "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": 13019, "countyName": "Berrien" } }
+{ "create_at": datetime("2016-01-03T10:00:31.000Z"), "id": 683709539183538176, "text": "Game Time at the NRG #GoTexans", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GoTexans" }}, "user": { "id": 972266546, "name": "$AUCIN'", "screen_name": "Francisco_txs80", "lang": "en", "location": "Houston, TX", "create_at": date("2012-11-26"), "description": "null", "followers_count": 582, "friends_count": 1561, "statues_count": 4592 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-03T10:00:31.000Z"), "id": 683709539191934977, "text": "We're #hiring! Click to apply: Pharmacy Technician - https://t.co/TKilh5XqwQ #Healthcare #pharmacy #Oakhurst, CA #Veterans #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.6493154,37.3279997"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Healthcare", "pharmacy", "Oakhurst", "Veterans", "Job", "Jobs" }}, "user": { "id": 2786536903, "name": "Raley's Jobs", "screen_name": "raleysjobs", "lang": "en", "location": "null", "create_at": date("2014-09-02"), "description": "Feed your passion at Raley’s; join our team! Job opportunities you can grow your career with. Get news, recipes and more @raleys.", "followers_count": 74, "friends_count": 1, "statues_count": 1538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakhurst, CA", "id": "6ed2a772760538c5", "name": "Oakhurst", "place_type": "city", "bounding_box": rectangle("-119.671846,37.300488 -119.603613,37.370568") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 652764, "cityName": "Oakhurst" } }
+{ "create_at": datetime("2016-01-03T10:00:31.000Z"), "id": 683709539921756160, "text": "#YearOfTheCakes2016", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "YearOfTheCakes2016" }}, "user": { "id": 46305981, "name": "Jack of DemBoy$", "screen_name": "ImDatNigga_Jack", "lang": "en", "location": "PVAMU", "create_at": date("2009-06-10"), "description": "I Don't Tweet No Lies Dem Boy$ IG:imdatnigga_jack #PVNation #HighLifePromos #LongLiveChopo Free @JoeSwaggerRight #LongLiveClyde", "followers_count": 5530, "friends_count": 3927, "statues_count": 318336 }, "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-03T10:00:31.000Z"), "id": 683709540026695681, "text": "I need a nice blow out but I keep getting gelled off ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 303923593, "name": "♛", "screen_name": "najahxo__", "lang": "en", "location": "somewhere sittin' pretty", "create_at": date("2011-05-23"), "description": "sc : najahox || @izzymcfly_❣", "followers_count": 1994, "friends_count": 551, "statues_count": 73295 }, "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-01-03T10:00:31.000Z"), "id": 683709540068540416, "text": "Focused and determined", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 55122887, "name": "NevettaKaye", "screen_name": "NevettaKaye", "lang": "en", "location": "San Antonio, Texas", "create_at": date("2009-07-08"), "description": "God is first ! #enoughsaid", "followers_count": 230, "friends_count": 209, "statues_count": 8449 }, "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-03T10:00:31.000Z"), "id": 683709540144148480, "text": "On a scale of one to ten I'm at 100", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 546432573, "name": "Satans baby girl™", "screen_name": "Princess_Roura", "lang": "en", "location": "null", "create_at": date("2012-04-05"), "description": "centrarse en mí y me encantaría más difícil", "followers_count": 866, "friends_count": 939, "statues_count": 46205 }, "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-03T10:00:31.000Z"), "id": 683709540336992256, "text": "Happy Birthday @EsteboJaylin I love you so much! ���� https://t.co/NNwmz2Bzz7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4006532714 }}, "user": { "id": 2615854994, "name": "Gween Queen", "screen_name": "SpadyGwen", "lang": "en", "location": "null", "create_at": date("2014-07-10"), "description": "chances are I'm too BLUNTLY ARGUMENTATIVE to notice that I hurt your feelings. Don't take it personal.", "followers_count": 112, "friends_count": 176, "statues_count": 1081 }, "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-03T10:00:31.000Z"), "id": 683709540374724608, "text": "@AceAKAYoungn Thats Love Family\nWe Need To Make Some Waves For Texas 1 Time ��", "in_reply_to_status": 683700151219654656, "in_reply_to_user": 371341426, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 371341426 }}, "user": { "id": 47054031, "name": "#TRAPMUSIC2", "screen_name": "OfficialMDuKe", "lang": "en", "location": "HeadAss, TX", "create_at": date("2009-06-14"), "description": "N E W . W A V E S . O N L Y", "followers_count": 1940, "friends_count": 1, "statues_count": 24097 }, "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-03T10:00:31.000Z"), "id": 683709540496326656, "text": "baby it's been that way...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 632444012, "name": "ملكة", "screen_name": "Charsi__", "lang": "en", "location": "3|1|8", "create_at": date("2012-07-10"), "description": "teja ♡ •Respect The Queen•", "followers_count": 732, "friends_count": 752, "statues_count": 57147 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jena, LA", "id": "fca70390d005bc49", "name": "Jena", "place_type": "city", "bounding_box": rectangle("-92.14465,31.667524 -92.105647,31.709415") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22059, "countyName": "La Salle", "cityID": 2238285, "cityName": "Jena" } }
+{ "create_at": datetime("2016-01-03T10:00:31.000Z"), "id": 683709540521607168, "text": "@Tiffany_Huffman happy birthday Tiff!!!! Have a great day, ily ��", "in_reply_to_status": -1, "in_reply_to_user": 278343633, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 278343633 }}, "user": { "id": 484326345, "name": "Ally", "screen_name": "AllyDennler", "lang": "en", "location": "nky", "create_at": date("2012-02-05"), "description": "conner varsity cheer // matt ♡ // 2014 state champion", "followers_count": 922, "friends_count": 502, "statues_count": 17178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Francisville, KY", "id": "00c548f506b07b9e", "name": "Francisville", "place_type": "city", "bounding_box": rectangle("-84.753305,39.07942 -84.693643,39.133115") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21015, "countyName": "Boone", "cityID": 2128882, "cityName": "Francisville" } }
+{ "create_at": datetime("2016-01-03T10:00:31.000Z"), "id": 683709540542447616, "text": "This #Healthcare #job might be a great fit for you: Nurse Practitioner, Urgent Care - https://t.co/Yr1vqpN3iw #nurse #Cabot, AR #Veterans", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.0165336,34.974532"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "nurse", "Cabot", "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": 173, "friends_count": 32, "statues_count": 851 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cabot, AR", "id": "7c96add4b12f4768", "name": "Cabot", "place_type": "city", "bounding_box": rectangle("-92.088415,34.928053 -91.96429,35.037344") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5085, "countyName": "Lonoke", "cityID": 510300, "cityName": "Cabot" } }
+{ "create_at": datetime("2016-01-03T10:00:31.000Z"), "id": 683709540689297408, "text": "not pictured: all the tacos (������) @ Venice Beach https://t.co/OCuSULKd7E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.47195779,33.98538962"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1729536842, "name": "Leora Yashari", "screen_name": "Leoramonay", "lang": "en", "location": "NY | LA | BOS ", "create_at": date("2013-09-04"), "description": "Production // Editorial @VanityFair | probably not making eye contact unless you have a cute dog.", "followers_count": 109, "friends_count": 198, "statues_count": 153 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-03T10:00:31.000Z"), "id": 683709540714414081, "text": "https://t.co/wx9Bd9RpW3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 36672719, "name": "Jason Parker", "screen_name": "mrjasonparker", "lang": "en", "location": "New York City", "create_at": date("2009-04-30"), "description": "Doing what I love the BEST! Special Events + Celebrity Wrangling (Are you CONNECTED!?) @CTD_Ent_Group", "followers_count": 592, "friends_count": 359, "statues_count": 6212 }, "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-03T10:00:31.000Z"), "id": 683709540831858688, "text": "what's the point of an Oreo if it isn't double stuffed?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1409121841, "name": "max grew yawn", "screen_name": "MaxGrullon_", "lang": "en", "location": "null", "create_at": date("2013-05-06"), "description": "CCHS⚾️ SA,TX", "followers_count": 452, "friends_count": 356, "statues_count": 8117 }, "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-03T10:00:31.000Z"), "id": 683709540836192257, "text": "2 more hrs til I can take my ass home", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 128590338, "name": "$upah Jefe", "screen_name": "DJSupahstar_", "lang": "en", "location": "617-978", "create_at": date("2010-04-01"), "description": "#RIPSmalls #VybeTribe IG & Snapchat-DJSupahstar | PSN-DJSupahstar_ | http://blogdjsupahstar.tumblr.com | djsupahstar@gmail.com", "followers_count": 3882, "friends_count": 2057, "statues_count": 158585 }, "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-03T10:00:31.000Z"), "id": 683709540852862977, "text": "need the 12th to hurry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 448448106, "name": "kelly", "screen_name": "kmrbobba", "lang": "en", "location": "Louisiana, USA", "create_at": date("2011-12-27"), "description": "null", "followers_count": 329, "friends_count": 309, "statues_count": 8483 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Patterson, LA", "id": "27bab96869e9bfc1", "name": "Patterson", "place_type": "city", "bounding_box": rectangle("-91.350534,29.67399 -91.291746,29.725939") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22101, "countyName": "St. Mary", "cityID": 2259340, "cityName": "Patterson" } }
+{ "create_at": datetime("2016-01-03T10:00:31.000Z"), "id": 683709540852994049, "text": "@R_Habes @kathleen_liebs @NanosauR @Target @azizansari I just said all of them in his voice", "in_reply_to_status": 683707360234311681, "in_reply_to_user": 550646716, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 550646716, 315263652, 22716072, 89084561, 6480682 }}, "user": { "id": 357157178, "name": "Matheson", "screen_name": "amanda_ragusa", "lang": "en", "location": "Westchester|Scranton|Galway", "create_at": date("2011-08-17"), "description": "live every week like it's shark week", "followers_count": 382, "friends_count": 286, "statues_count": 32669 }, "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-03T10:00:31.000Z"), "id": 683709541184212993, "text": "The chargers are playing in Denver today and I'll be in Denver... At an airport. Dammit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 333772850, "name": "Tyler Crispen⚡", "screen_name": "Tcrispen2", "lang": "en", "location": "Ohio, USA", "create_at": date("2011-07-11"), "description": "Beach Bum / Hilton Head Island Lifeguard", "followers_count": 643, "friends_count": 499, "statues_count": 8486 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Breckenridge, CO", "id": "00401a907e12e1b1", "name": "Breckenridge", "place_type": "city", "bounding_box": rectangle("-106.0747,39.460879 -105.997415,39.550435") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8117, "countyName": "Summit", "cityID": 808400, "cityName": "Breckenridge" } }
+{ "create_at": datetime("2016-01-03T10:00:32.000Z"), "id": 683709541385543680, "text": "\"Girl gimme those socks if they too big, you know I got LaCienaga feet..\" ������ - @daddylongglegss", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2898655318 }}, "user": { "id": 2879742936, "name": "Jade Maria", "screen_name": "Ja_De15", "lang": "en", "location": "Bay Area, CA", "create_at": date("2014-10-27"), "description": "God is ALWAYS on time❤️", "followers_count": 204, "friends_count": 337, "statues_count": 6657 }, "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-03T10:00:32.000Z"), "id": 683709541419069440, "text": "Giving so much of your self it seems the only way ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2169658092, "name": "#️⃣Jan.20‼️", "screen_name": "Dmoe_4Fss", "lang": "en", "location": "Dallas, TX", "create_at": date("2013-11-01"), "description": "Keep it 8 more than 92 with me‼️ #God #OTF #Ballislife #4️⃣ #FleeGang yall know my pops @Dame_Lillard. Racine WI ✈️ Dallas TX ig:dmoe_bucketteam35SC: ll_dmoe4fs", "followers_count": 1483, "friends_count": 1151, "statues_count": 30696 }, "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-03T10:00:32.000Z"), "id": 683709541427445762, "text": "Bye bye Brooklyn ��✌️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 2493097277, "name": "Tim$", "screen_name": "PNDs_Girl", "lang": "en", "location": "Houston, TX", "create_at": date("2014-04-18"), "description": "i got to feel alive even if it kills me #PARTYNEXTDOOR|(HTX)|SaloneTiTi|Aquarius| Godly child", "followers_count": 518, "friends_count": 511, "statues_count": 7613 }, "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-03T10:00:32.000Z"), "id": 683709541452759040, "text": "Real mvp �� https://t.co/RZKI4vLrdL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 191481848, "name": "Erky♠️", "screen_name": "EGotti1", "lang": "en", "location": "Baltimore", "create_at": date("2010-09-16"), "description": "Stuntin on the old me❗ Snapchat: Egotti1 ig: @E_gottiii", "followers_count": 733, "friends_count": 362, "statues_count": 15612 }, "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-03T10:00:32.000Z"), "id": 683709541947695105, "text": "I literally got out of the shower a hour and a half ago and still have yet to get dressed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 566875159, "name": "Em", "screen_name": "EmilyStevens98", "lang": "en", "location": "Stillwater, NY", "create_at": date("2012-04-29"), "description": "snapchat: heyitsemilyjo", "followers_count": 716, "friends_count": 1491, "statues_count": 4118 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stillwater, NY", "id": "01d62f216bc2c949", "name": "Stillwater", "place_type": "city", "bounding_box": rectangle("-73.698693,42.909194 -73.630371,42.959595") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36091, "countyName": "Saratoga", "cityID": 3671322, "cityName": "Stillwater" } }
+{ "create_at": datetime("2016-01-03T10:00:32.000Z"), "id": 683709542006439937, "text": "We in here! https://t.co/EIReP7VZiH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.40106,33.757617"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3432459995, "name": "Logan Marshall", "screen_name": "LoganDaDon", "lang": "en", "location": "null", "create_at": date("2015-08-19"), "description": "Atlanta sports fan. BROTHER OF ALPHA SIGMA SIGMA AT GEORGIA SOUTHERN UNIVERSITY.", "followers_count": 366, "friends_count": 364, "statues_count": 1269 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia Dome", "id": "07d9e38ea3087002", "name": "Georgia Dome", "place_type": "poi", "bounding_box": rectangle("-84.4010601,33.7576169 -84.40106,33.757617") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-01-03T10:00:32.000Z"), "id": 683709542023172096, "text": "@Erwan_Tao bah si c légal c sque tu fais dessus qui est illégal", "in_reply_to_status": 683708964396253186, "in_reply_to_user": 2656660034, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user_mentions": {{ 2656660034 }}, "user": { "id": 2723759169, "name": "DucBilal", "screen_name": "Givefuuck", "lang": "fr", "location": "los angeles ", "create_at": date("2014-07-24"), "description": "Snap : theetalon", "followers_count": 17916, "friends_count": 90, "statues_count": 30991 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-03T10:00:32.000Z"), "id": 683709542098538496, "text": "Is it too late to say I'm sorry now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 988093242, "name": "Ellen Gonzalez", "screen_name": "elleng04", "lang": "en", "location": "null", "create_at": date("2012-12-03"), "description": "null", "followers_count": 663, "friends_count": 414, "statues_count": 6491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Smithville, NJ", "id": "00b285d96fc8425d", "name": "Smithville", "place_type": "city", "bounding_box": rectangle("-74.508829,39.47126 -74.43359,39.510641") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34001, "countyName": "Atlantic", "cityID": 3468190, "cityName": "Smithville" } }
+{ "create_at": datetime("2016-01-03T10:00:32.000Z"), "id": 683709542249721856, "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": 577583058, "name": "Melinda⚓", "screen_name": "MindySoccer13", "lang": "en", "location": "null", "create_at": date("2012-05-11"), "description": "Love life Carefree Stay Trippy For Life ✌️✨Chris Moyer has my heart", "followers_count": 705, "friends_count": 579, "statues_count": 3513 }, "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-03T10:00:32.000Z"), "id": 683709542643949569, "text": "������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2995212520, "name": "Sugar", "screen_name": "shreyyya_", "lang": "en", "location": "null", "create_at": date("2015-01-23"), "description": "VCU", "followers_count": 155, "friends_count": 132, "statues_count": 1226 }, "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-03T10:00:32.000Z"), "id": 683709543235239937, "text": "#Hodgkins, IL #Transportation #Job: Driver - $2,000 HIRING BONUS - Class A CDL at DPI Specialty Foods https://t.co/JN3M8uE2AU #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.857835,41.7689207"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hodgkins", "Transportation", "Job", "Jobs", "Hiring" }}, "user": { "id": 3143485394, "name": "DPI Careers", "screen_name": "DPICareers", "lang": "en", "location": "Nationwide", "create_at": date("2015-04-07"), "description": "We are on an exciting growth journey and need exceptional talent to deliver our vision. Come and join the winning team, apply today!", "followers_count": 8, "friends_count": 0, "statues_count": 49 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hodgkins, IL", "id": "f8834b23cf9ae9f3", "name": "Hodgkins", "place_type": "city", "bounding_box": rectangle("-87.886644,41.738726 -87.831892,41.783996") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1735385, "cityName": "Hodgkins" } }
+{ "create_at": datetime("2016-01-03T10:00:32.000Z"), "id": 683709543411367936, "text": "We're #hiring! Read about our latest #job opening here: Staff RN - Good Samaritan Hospital ICU - https://t.co/TFqqT7PdUY #Puyallup, WA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.2928974,47.1853785"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Puyallup" }}, "user": { "id": 22868585, "name": "WA Nursing Jobs", "screen_name": "tmj_wa_nursing", "lang": "en", "location": "Washington", "create_at": date("2009-03-04"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Washington Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 721, "friends_count": 305, "statues_count": 529 }, "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-03T10:00:32.000Z"), "id": 683709543474282496, "text": "@BeeHeintz8517 @PatDoneyNBC5 @StubHub It's pretty bad.", "in_reply_to_status": 683709311843872769, "in_reply_to_user": 4521252040, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4521252040, 36746142, 14419089 }}, "user": { "id": 2564249876, "name": "Mark Ranstrom", "screen_name": "Mark_Ranstrom", "lang": "en", "location": "Minnesota, USA", "create_at": date("2014-06-12"), "description": "Father. Fishing. Sports. #CowboysNation #ThrowUpTheX #LosLynx #MNWild #MNTwins", "followers_count": 353, "friends_count": 639, "statues_count": 33135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Otsego, MN", "id": "0100513d0b8c6d73", "name": "Otsego", "place_type": "city", "bounding_box": rectangle("-93.615401,45.221955 -93.519511,45.301215") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27171, "countyName": "Wright", "cityID": 2749138, "cityName": "Otsego" } }
+{ "create_at": datetime("2016-01-03T10:00:32.000Z"), "id": 683709543566585856, "text": "We're #hiring! Read about our latest #job opening here: Auto Tire/Lube Technician - FT $300 BONUS! - https://t.co/9jefzhcFBe #NewUlm, MN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.461114,44.312225"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "NewUlm" }}, "user": { "id": 2963298858, "name": "Royal Tire Jobs", "screen_name": "RoyalTireJobs", "lang": "en", "location": "St. Cloud, MN", "create_at": date("2015-01-07"), "description": "Since 1948, Royal Tire has been providing customers with quality transportation care, products, and services.", "followers_count": 71, "friends_count": 279, "statues_count": 31 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Ulm, MN", "id": "40aef2dae5f2e9cd", "name": "New Ulm", "place_type": "city", "bounding_box": rectangle("-94.507217,44.280315 -94.414288,44.344475") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27015, "countyName": "Brown", "cityID": 2746042, "cityName": "New Ulm" } }
+{ "create_at": datetime("2016-01-03T10:00:32.000Z"), "id": 683709543650594817, "text": "\"I'm leaving now\" https://t.co/peNUCAvPWA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1116509455, "name": "ä.❁", "screen_name": "enalajjalane", "lang": "en", "location": "876", "create_at": date("2013-01-24"), "description": "eleese.(: // sc : nylletak", "followers_count": 1667, "friends_count": 874, "statues_count": 29940 }, "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-03T10:00:32.000Z"), "id": 683709543918866432, "text": "Oh my what a wonderful time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1651659241, "name": "☪Drizzle", "screen_name": "7claraaa", "lang": "en", "location": "california", "create_at": date("2013-08-06"), "description": "19 | sjcc hoop #24", "followers_count": 513, "friends_count": 295, "statues_count": 33284 }, "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-03T10:00:32.000Z"), "id": 683709543986020352, "text": "Music and football give me life ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3167683110, "name": "Dakota Culley", "screen_name": "assemblagebrood", "lang": "en", "location": "null", "create_at": date("2015-04-22"), "description": "Native American|Poet|It's Funny How Artistic We Become When Our Hearts Are Broken -H.B. RIP~Victoria | Aggressive Intellectual Giant", "followers_count": 1721, "friends_count": 1765, "statues_count": 6922 }, "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-03T10:00:32.000Z"), "id": 683709543994363904, "text": "PATIENT CARE TECH I - South Georgia Medical Center: (#Valdosta, GA) https://t.co/P275rWYCTo #Nursing #RN #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.2863,30.8625"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Valdosta", "Nursing", "RN", "Job", "Jobs", "Hiring" }}, "user": { "id": 2842711225, "name": "SGMC Careers", "screen_name": "SGMCCareers", "lang": "en", "location": "Valdosta, Georgia", "create_at": date("2014-10-06"), "description": "SGMC offers employees a warm and welcoming environment in a rapidly growing, dynamic organization. Follow us to learn more about our openings!", "followers_count": 65, "friends_count": 33, "statues_count": 358 }, "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-01-03T10:00:32.000Z"), "id": 683709544128757760, "text": "@RealTeamGetNude Where #RavenSwallows #Cums 2 #GetNude wit #TeamGetNude cuz dey like #pussy n #BadGirls RT&F ifu do2 https://t.co/2t7eircm4h", "in_reply_to_status": -1, "in_reply_to_user": 4316900363, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RavenSwallows", "Cums", "GetNude", "TeamGetNude", "pussy", "BadGirls" }}, "user_mentions": {{ 4316900363 }}, "user": { "id": 1070092106, "name": "RavenSwallows", "screen_name": "sexyohgirl", "lang": "en", "location": "US", "create_at": date("2013-01-07"), "description": "#HappilyMarried #OSUBuckeye #BikiniModel #Cosplayfan #AdultModel https://t.co/C3hSRdB0Sm https://t.co/nHunE0JPaz https://t.co/Hh0Mtfr660\nhttps://t.co/yPYWfSKdf2", "followers_count": 39104, "friends_count": 550, "statues_count": 8324 }, "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-03T10:00:32.000Z"), "id": 683709544309088256, "text": "���������� https://t.co/WDeLndhXeu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1542057960, "name": "quonyquon.", "screen_name": "onlyonequon", "lang": "en", "location": "null", "create_at": date("2013-06-23"), "description": "the godson .", "followers_count": 1648, "friends_count": 823, "statues_count": 42763 }, "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-03T10:00:32.000Z"), "id": 683709544313176064, "text": "@tedcruz Make sure you sign up so he can slam your inbox with donation requests.", "in_reply_to_status": 683700599037153280, "in_reply_to_user": 23022687, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23022687 }}, "user": { "id": 1689920359, "name": "Robyn Ryan", "screen_name": "Reryan08", "lang": "en", "location": "null", "create_at": date("2013-08-21"), "description": "American Patriot", "followers_count": 574, "friends_count": 297, "statues_count": 15713 }, "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-03T10:00:32.000Z"), "id": 683709544392859649, "text": "We're #hiring! Read about our latest #job opening here: Web Designer - https://t.co/J2OIBU5rGm #holidayjobs #BocaRaton, FL #Retail", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.0830984,26.3586885"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "holidayjobs", "BocaRaton", "Retail" }}, "user": { "id": 2798260664, "name": "Boston Proper Jobs", "screen_name": "BostonProperJob", "lang": "en", "location": "null", "create_at": date("2014-09-08"), "description": "Boston Proper provides fashions for women who are fearlessly feminine, enviably chic and who possess the poise and confidence to “wear it like no one else“.", "followers_count": 30, "friends_count": 0, "statues_count": 460 }, "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-03T10:00:32.000Z"), "id": 683709544577380353, "text": "cant raise a man", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 99545472, "name": "Mykell Beyince", "screen_name": "iLoveMYKELL", "lang": "en", "location": "St. Louis", "create_at": date("2009-12-26"), "description": "welcome to the black parade. IG/snap ilovemykell booking: @MeanGirlzMedia or meangirlzmedia@gmail.com or 3143756775", "followers_count": 1303, "friends_count": 590, "statues_count": 146779 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Black Jack, MO", "id": "51ec1aa7df16aa68", "name": "Black Jack", "place_type": "city", "bounding_box": rectangle("-90.279715,38.778714 -90.227967,38.818977") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2906004, "cityName": "Black Jack" } }
+{ "create_at": datetime("2016-01-03T10:00:32.000Z"), "id": 683709544585916421, "text": "@just_britt13 you always told me even when you put us on sc which I still have����", "in_reply_to_status": 683701810289328128, "in_reply_to_user": 2774214963, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2774214963 }}, "user": { "id": 3300963784, "name": "Red Velvet❤", "screen_name": "abbsm17", "lang": "en", "location": "the 334✊", "create_at": date("2015-05-27"), "description": "Dessert doesn't go to the stomach. Dessert goes to the heart❤", "followers_count": 388, "friends_count": 449, "statues_count": 12911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clay, AL", "id": "6232fcc6971703ec", "name": "Clay", "place_type": "city", "bounding_box": rectangle("-86.656883,33.653873 -86.570275,33.728871") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1073, "countyName": "Jefferson", "cityID": 115256, "cityName": "Clay" } }
+{ "create_at": datetime("2016-01-03T10:00:32.000Z"), "id": 683709544606863360, "text": "@nateinmilwaukee Hi, Nate!", "in_reply_to_status": 683708968028475392, "in_reply_to_user": 488931863, "favorite_count": 0, "coordinate": point("-87.99169357,42.16871289"), "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 488931863 }}, "user": { "id": 1395826254, "name": "DickInMyPants", "screen_name": "dickinmypants", "lang": "en", "location": "null", "create_at": date("2013-05-01"), "description": "I'm an appendage that hangs out in shorts or trousers usually and gets shoved into places for sexual purposes. Naturally, I produce insightful commentary", "followers_count": 9, "friends_count": 34, "statues_count": 234 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buffalo Grove, IL", "id": "11e5f4a486b64a57", "name": "Buffalo Grove", "place_type": "city", "bounding_box": rectangle("-87.995118,42.129664 -87.897352,42.210544") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1709447, "cityName": "Buffalo Grove" } }
+{ "create_at": datetime("2016-01-03T10:00:32.000Z"), "id": 683709544682237953, "text": "UNLESS SHE ONLY CURVING YOU LOL THATS LIFE THO https://t.co/m8lmNA65r9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3247833884, "name": "daddy", "screen_name": "dvshxun_", "lang": "en", "location": "texa$", "create_at": date("2015-06-17"), "description": "bitch i got the sauce", "followers_count": 203, "friends_count": 120, "statues_count": 3881 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Poteet, TX", "id": "0e4060f232480ec9", "name": "Poteet", "place_type": "city", "bounding_box": rectangle("-98.58807,29.025442 -98.561054,29.050464") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48013, "countyName": "Atascosa", "cityID": 4859084, "cityName": "Poteet" } }
+{ "create_at": datetime("2016-01-03T10:00:32.000Z"), "id": 683709544741138432, "text": "Just posted a photo @ Elevationchurch https://t.co/1Bx4hrqkO5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.78890381,35.29872786"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 159156062, "name": "Tamara Ward", "screen_name": "mihi78", "lang": "en", "location": "Alexandria, VA", "create_at": date("2010-06-24"), "description": "null", "followers_count": 220, "friends_count": 679, "statues_count": 3297 }, "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-03T10:00:32.000Z"), "id": 683709544770342913, "text": "making memories �� https://t.co/be5mnqlwPf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.918295,33.805213"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1064322272, "name": "lyss", "screen_name": "AlyssaPalmer99", "lang": "en", "location": "null", "create_at": date("2013-01-05"), "description": "†", "followers_count": 780, "friends_count": 446, "statues_count": 7837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Monica Pier", "id": "07d9ea8c53487000", "name": "Santa Monica Pier", "place_type": "poi", "bounding_box": rectangle("-117.9182951,33.8052129 -117.918295,33.805213") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 629000, "cityName": "Garden Grove" } }
+{ "create_at": datetime("2016-01-03T10:00:32.000Z"), "id": 683709544959062018, "text": "@lbozoo lol", "in_reply_to_status": 683709476428353537, "in_reply_to_user": 1125348871, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 62374282 }}, "user": { "id": 1125348871, "name": "CJ", "screen_name": "JustSayBi", "lang": "en", "location": "Miami", "create_at": date("2013-01-27"), "description": "I'm a married guy trying to have fun in crazy Miami looking to connect with other cool people and couples see whats out there in Miami", "followers_count": 653, "friends_count": 1432, "statues_count": 4181 }, "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.446969,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-01-03T10:00:32.000Z"), "id": 683709545009549312, "text": "Wind 2 mph N. Barometer 30.06 in, Steady. Temperature 59.9 °F. Rain today 0.15 in. Humidity 71%", "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": 51, "friends_count": 70, "statues_count": 24705 }, "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-03T10:00:32.000Z"), "id": 683709545131032576, "text": "Mom made fun of me having trouble talking to her in Tagalog �� smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 499545329, "name": "Ramon T.", "screen_name": "Ramsgotchuu", "lang": "en", "location": "Los Angeles", "create_at": date("2012-02-21"), "description": "Professional Snapchat Singer", "followers_count": 259, "friends_count": 225, "statues_count": 5321 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-03T10:00:33.000Z"), "id": 683709545902813184, "text": "Happy birthday pretty girl!!�� @Lindsey_araujo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1384612554 }}, "user": { "id": 1450364822, "name": "emmy k", "screen_name": "emilykerfoot", "lang": "en", "location": "UTSA20", "create_at": date("2013-05-22"), "description": "keep rolling your eyes, maybe you'll find a brain back there || AHS Varsity Cheerleader", "followers_count": 1151, "friends_count": 420, "statues_count": 19443 }, "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-01-03T10:00:33.000Z"), "id": 683709545919590400, "text": "@ronalan908 @BenjaminLives @TheSisko @FSM1988 work??", "in_reply_to_status": 683709401719549952, "in_reply_to_user": 2906952316, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2906952316, 276493424, 24394452, 4075759937 }}, "user": { "id": 423942749, "name": "David Eliff", "screen_name": "EliffDavid", "lang": "en", "location": "null", "create_at": date("2011-11-28"), "description": "null", "followers_count": 280, "friends_count": 1699, "statues_count": 4515 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lexington, TN", "id": "4c8ec4dd375b72c2", "name": "Lexington", "place_type": "city", "bounding_box": rectangle("-88.42873,35.611389 -88.364752,35.713527") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47077, "countyName": "Henderson", "cityID": 4741980, "cityName": "Lexington" } }
+{ "create_at": datetime("2016-01-03T10:00:33.000Z"), "id": 683709546078941184, "text": "Me confie��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 3010722661, "name": "Luna7", "screen_name": "ELCR7_OFFICIAL", "lang": "en", "location": "lost in paradise", "create_at": date("2015-02-05"), "description": "null", "followers_count": 197, "friends_count": 446, "statues_count": 2355 }, "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-03T10:00:33.000Z"), "id": 683709546338992128, "text": "New Signature #Sales #Job: Sales Account Executive (Boston) (#Cambridge, MA) https://t.co/RpYxSbnt2t #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.1096528,42.3726399"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "Job", "Cambridge", "Jobs", "Hiring" }}, "user": { "id": 4499586313, "name": "New Signature Jobs", "screen_name": "NewSigCareers", "lang": "en", "location": "null", "create_at": date("2015-12-08"), "description": "New Signature is hiring qualified professionals with a background in #information #technology, #computer #science, #web #development or #graphic #design.", "followers_count": 10, "friends_count": 83, "statues_count": 22 }, "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-01-03T10:00:33.000Z"), "id": 683709546502602753, "text": "Do you speak black? https://t.co/6nSawQlJhq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1016007504, "name": "SunnyD", "screen_name": "sdelgado18", "lang": "en", "location": "Santa Clara, CA", "create_at": date("2012-12-16"), "description": "Since '96", "followers_count": 378, "friends_count": 195, "statues_count": 17632 }, "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-03T10:00:33.000Z"), "id": 683709546540367872, "text": "Oldies day fasho", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2947362800, "name": "Chiques", "screen_name": "ox_medina", "lang": "en", "location": "Oxnard, CA ", "create_at": date("2014-12-28"), "description": "Xicano ☕️", "followers_count": 259, "friends_count": 175, "statues_count": 23551 }, "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-03T10:00:33.000Z"), "id": 683709546544656384, "text": "@Jcrimz44 wg fam where's my money https://t.co/u20PH2zhW6", "in_reply_to_status": -1, "in_reply_to_user": 350684544, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 350684544 }}, "user": { "id": 574213527, "name": "Isuzu", "screen_name": "ChickMarkham", "lang": "en", "location": "null", "create_at": date("2012-05-07"), "description": "#RIPKJH #RIPRW add me on snapchat @tsteph95", "followers_count": 548, "friends_count": 648, "statues_count": 5014 }, "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-01-03T10:00:33.000Z"), "id": 683709546708086784, "text": "Want to work at CVS Health? We're #hiring in #Littleton, CO! Click for details: https://t.co/IndJCxJo6C #Clerical #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.0166498,39.613321"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Littleton", "Clerical", "Job", "Jobs", "CareerArc" }}, "user": { "id": 23015676, "name": "TMJ-DEN Cleric. Jobs", "screen_name": "tmj_den_cler", "lang": "en", "location": "Denver, CO", "create_at": date("2009-03-05"), "description": "Follow this account for geo-targeted Office/Clerical job tweets in Denver, CO. Need help? Tweet us at @CareerArc!", "followers_count": 302, "friends_count": 200, "statues_count": 151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Littleton, CO", "id": "910868722e1e4825", "name": "Littleton", "place_type": "city", "bounding_box": rectangle("-105.062023,39.563868 -104.973551,39.62983") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8005, "countyName": "Arapahoe", "cityID": 845255, "cityName": "Littleton" } }
+{ "create_at": datetime("2016-01-03T10:00:33.000Z"), "id": 683709546729177088, "text": "I'm surprised there was actually a team who signed Josh Freeman. He was terrible when he played for the Bucs...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 65460268, "name": "Gina Navarra", "screen_name": "gmnavarra", "lang": "en", "location": "Tallahassee, FL", "create_at": date("2009-08-13"), "description": "I love my family, friends, sports, adventures and coffee. UCF grad. Making a living in the entertainment & college hoops world in FL's capital city #IHeartTally", "followers_count": 978, "friends_count": 1115, "statues_count": 37481 }, "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-03T10:00:33.000Z"), "id": 683709546766925828, "text": "Man that's my twin brother ina blue ������������ https://t.co/CF4KxHRWs9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 290040124, "name": "I aint shit!", "screen_name": "ripsmoke_", "lang": "en", "location": "Smoke Ave 4L!!! ", "create_at": date("2011-04-29"), "description": "All tha free's & Rip Smoke Jigg Nd Briana #Freeyungyuano ! • Christian Was Here❤️ bff here too❣@onlycaasshhh SHANICE HERE❤️ Keisha Here Too Tf❤️", "followers_count": 832, "friends_count": 704, "statues_count": 33348 }, "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-03T10:00:33.000Z"), "id": 683709546787778560, "text": "@MsPackyetti @SparrowOnTheEye https://t.co/IXtYNBQS0x", "in_reply_to_status": 683546820627148804, "in_reply_to_user": 239509917, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 239509917, 58981717 }}, "user": { "id": 209568114, "name": "Devout Coward", "screen_name": "LesJulia9", "lang": "en", "location": "New Orleans", "create_at": date("2010-10-29"), "description": "We can't do anything..my fav subjects racism/philosophy, Your will is not yours, we are all innocent, we all go to heaven", "followers_count": 422, "friends_count": 1085, "statues_count": 36580 }, "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-03T10:00:33.000Z"), "id": 683709547102474242, "text": "Lunch with sweet Ashley before TEC meeting.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.422179,33.985788"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14171051, "name": "SusanLPreece", "screen_name": "SusanLPreece", "lang": "en", "location": "Georgia", "create_at": date("2008-03-18"), "description": "Child of God, Wife, Mother of 3 & crazy church lady!", "followers_count": 353, "friends_count": 872, "statues_count": 3561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Zoës Kitchen", "id": "07d9ce38a3086001", "name": "Zoës Kitchen", "place_type": "poi", "bounding_box": rectangle("-84.4221791,33.9857879 -84.422179,33.985788") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1368516, "cityName": "Sandy Springs" } }
+{ "create_at": datetime("2016-01-03T10:00:33.000Z"), "id": 683709547173785601, "text": "New Years party still going strong 3 days in", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 368685347, "name": "J.Cook", "screen_name": "J_C_60trey", "lang": "en", "location": "New England ➡️ The Dirty Sipp", "create_at": date("2011-09-05"), "description": "#LLOK || ΠΚΦ || Ole Miss '19", "followers_count": 341, "friends_count": 296, "statues_count": 7607 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Rutherford, NJ", "id": "df14cd4d61b8e457", "name": "East Rutherford", "place_type": "city", "bounding_box": rectangle("-74.120861,40.788991 -74.061214,40.853386") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3419510, "cityName": "East Rutherford" } }
+{ "create_at": datetime("2016-01-03T10:00:33.000Z"), "id": 683709547291136000, "text": "Best bet of the day Vikings +3.5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.8033003,39.5009534"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1476107802, "name": "jkruzel25", "screen_name": "jkruzel25", "lang": "en", "location": "Reno, NV", "create_at": date("2013-06-01"), "description": "Lucky to be in the USA", "followers_count": 1650, "friends_count": 1991, "statues_count": 20365 }, "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-03T10:00:33.000Z"), "id": 683709547328999424, "text": "Now you wanna say wassup 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": 1143834126, "name": "MariahG", "screen_name": "mariahgrooms97", "lang": "en", "location": "Newark, OH", "create_at": date("2013-02-02"), "description": "running thru the 6 with my phone", "followers_count": 1068, "friends_count": 925, "statues_count": 12726 }, "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-01-03T10:00:33.000Z"), "id": 683709547492450305, "text": "VCU Health System: Clinical Nurse (#Richmond, VA) https://t.co/DMyDKiOG80 #Nursing #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.4296882,37.5396582"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "hashtags": {{ "Richmond", "Nursing", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 26802909, "name": "TMJ-RCH Nursing Jobs", "screen_name": "tmj_rch_nursing", "lang": "en", "location": "Richmond, VA", "create_at": date("2009-03-26"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Richmond, VA. Need help? Tweet us at @CareerArc!", "followers_count": 396, "friends_count": 305, "statues_count": 258 }, "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-03T10:00:33.000Z"), "id": 683709547517706240, "text": "@Chermillexo ha ha.", "in_reply_to_status": 683709108495585280, "in_reply_to_user": 3157484284, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 3157484284 }}, "user": { "id": 4352656812, "name": "Jerin Allen", "screen_name": "jayhendrix95", "lang": "en", "location": "null", "create_at": date("2015-11-24"), "description": "Isaiah 54:17. #HindsTrack. JUCO All-American. #TrackNation. 228 bred.", "followers_count": 186, "friends_count": 258, "statues_count": 1009 }, "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-01-03T10:00:33.000Z"), "id": 683709547848974336, "text": "@palmerholmquist see ya soon��", "in_reply_to_status": 683709383600115712, "in_reply_to_user": 1594088928, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1594088928 }}, "user": { "id": 2474320807, "name": "Rae", "screen_name": "the_drakkee", "lang": "en", "location": "null", "create_at": date("2014-05-02"), "description": "• prospering •", "followers_count": 599, "friends_count": 389, "statues_count": 10003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rendon, TX", "id": "00386da25bfa7106", "name": "Rendon", "place_type": "city", "bounding_box": rectangle("-97.30462,32.551138 -97.190905,32.610434") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4861568, "cityName": "Rendon" } }
+{ "create_at": datetime("2016-01-03T10:00:33.000Z"), "id": 683709547911884801, "text": "Dose of the Dead coming ? On @TheSoundLA with @MImi_TheSound #Dead Levi Stadium https://t.co/gDhV4iaP7O", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Dead" }}, "user_mentions": {{ 15484092, 23226974 }}, "user": { "id": 427192540, "name": "Martin Kline", "screen_name": "KlineMartin", "lang": "en", "location": "Southern California", "create_at": date("2011-12-02"), "description": "25 years spent in professional sports, currently works in real estate, an avid sports follower and classic rock junkie. Horns & Hammonds", "followers_count": 347, "friends_count": 800, "statues_count": 5527 }, "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-03T10:00:33.000Z"), "id": 683709547983159296, "text": "#Hospitality #Job in #Houston, TX: Area Director at IHOP ACG Texas https://t.co/S0OnZpfCjw #IHOP #htx #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.3890659,29.776353"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "Job", "Houston", "IHOP", "htx", "Jobs", "Hiring" }}, "user": { "id": 216428496, "name": "IHOP_ACG", "screen_name": "IHOP_ACGTexas", "lang": "en", "location": "Richardson, TX", "create_at": date("2010-11-16"), "description": "ACG Texas is proud to be Texas' largest #IHOP Franchisee. #Hospitality #jobs in 66 restaurants in Texas. Join our Team!", "followers_count": 310, "friends_count": 619, "statues_count": 713 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-03T10:00:33.000Z"), "id": 683709548176121856, "text": "everyones tweeting about going back to school tomorrow and then there me https://t.co/hTJn1sY8Uc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2560144016, "name": "grace", "screen_name": "grace__mason", "lang": "en", "location": "null", "create_at": date("2014-06-10"), "description": "null", "followers_count": 87, "friends_count": 141, "statues_count": 340 }, "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-03T10:00:33.000Z"), "id": 683709548343898112, "text": "Human Resources - Aircraft Technician - Cincinnati, OH - Procter & Gamble: (#Cincinnati, OH) https://t.co/qkE8IDRcfP #Clerical #Veterans", "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": {{ "Cincinnati", "Clerical", "Veterans" }}, "user": { "id": 23023747, "name": "TMJ-CIN Cleric. Jobs", "screen_name": "tmj_cin_cler", "lang": "en", "location": "Cincinnati, OH", "create_at": date("2009-03-05"), "description": "Follow this account for geo-targeted Office/Clerical job tweets in Cincinnati, OH. Need help? Tweet us at @CareerArc!", "followers_count": 242, "friends_count": 198, "statues_count": 86 }, "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-03T10:00:33.000Z"), "id": 683709548893437953, "text": "lets go cowboys ��⚪️��⚪️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1730139822, "name": "donnnn", "screen_name": "donishaaa__", "lang": "en", "location": "Alexandria, VA", "create_at": date("2013-09-04"), "description": "#ODU | Women's Basketball Manager | @emyvne_ ❤️", "followers_count": 595, "friends_count": 505, "statues_count": 25458 }, "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-03T10:00:33.000Z"), "id": 683709548922667008, "text": "#Denver, CO #Aerospace #Job: Aircraft Maintenance Programs Technical Writer at Frontier Airlines https://t.co/INf8W6uKGS #Jobs #Hiring", "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": {{ "Denver", "Aerospace", "Job", "Jobs", "Hiring" }}, "user": { "id": 3015856778, "name": "Fly Frontier Jobs", "screen_name": "FlyFrontierJobs", "lang": "en", "location": "null", "create_at": date("2015-02-09"), "description": "We offer service to more than 75 destinations in the US, Mexico, Costa Rica, & the Dominican Republic. Check here for current openings.", "followers_count": 173, "friends_count": 495, "statues_count": 86 }, "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-03T10:00:33.000Z"), "id": 683709548947849216, "text": "Can't wait too go see future in March ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 902864618, "name": "carla✨", "screen_name": "_carlaventura", "lang": "en", "location": "Houston tx ", "create_at": date("2012-10-24"), "description": "i'm just living life.", "followers_count": 330, "friends_count": 214, "statues_count": 4059 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-03T10:00:33.000Z"), "id": 683709549165936640, "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": 1618643906, "name": "Kaitlyn Bullock", "screen_name": "KaitlynBullock4", "lang": "en", "location": "louisiana ", "create_at": date("2013-07-24"), "description": "null", "followers_count": 731, "friends_count": 396, "statues_count": 4780 }, "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-03T10:00:33.000Z"), "id": 683709549283553280, "text": "It's a beautiful day! Happy Sunday! ☀️�� I♥️NY ������ https://t.co/6STiWTtPjT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 66151500, "name": "Tommy Gee", "screen_name": "tommygeemusic", "lang": "en", "location": "New York, NY", "create_at": date("2009-08-16"), "description": "I'm in a NEW YORK state of mind. I❤️NY #NewYorkCity // LIFE IS GOOD // ARSENAL is my team. #TommyGee Snapchat: @ djtommygee", "followers_count": 584, "friends_count": 429, "statues_count": 13945 }, "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-03T10:00:33.000Z"), "id": 683709549304360960, "text": "@OPB trolling Portland with \"Let it snow\". Nicely done. Now please stop talking about some retarded hicks in Burns.", "in_reply_to_status": -1, "in_reply_to_user": 21512383, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21512383 }}, "user": { "id": 13480052, "name": "Sascha Siekmann", "screen_name": "ssiekmann", "lang": "en", "location": "Portland, OR", "create_at": date("2008-02-14"), "description": "It's a long story. I like 3.141592 and 2600", "followers_count": 382, "friends_count": 1674, "statues_count": 5047 }, "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-03T10:00:33.000Z"), "id": 683709549396803586, "text": "I'm at Hotel Nikko San Francisco - @hotelnikkosf in San Francisco, CA https://t.co/DctXs1BC3Y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.40929246,37.78604644"), "retweet_count": 0, "lang": "it", "is_retweet": false, "user_mentions": {{ 46425371 }}, "user": { "id": 134524130, "name": "Marcos F. Martínez", "screen_name": "MakuTFA", "lang": "es", "location": "Puebla, México", "create_at": date("2010-04-18"), "description": "Hunter from Pokke Village. Light artillery sharpshooter and melee supporter. Coffee addict, and PVC collector. Knitting enthusiast and cooking maniac.", "followers_count": 172, "friends_count": 160, "statues_count": 1201 }, "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-03T10:00:33.000Z"), "id": 683709549409398788, "text": "GREATER\nIS\nCOMING. (2016)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 189708692, "name": "️Wayne", "screen_name": "chefwayne___", "lang": "en", "location": "null", "create_at": date("2010-09-11"), "description": "null", "followers_count": 322, "friends_count": 364, "statues_count": 6033 }, "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-03T10:00:33.000Z"), "id": 683709549522628608, "text": "HAPPY BIRTHDAY @Madison_Lynnnnn \n❤️- the small little boy/ugly sister twin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 836466842 }}, "user": { "id": 1370382606, "name": "Taylor Queen", "screen_name": "taylorrheannak", "lang": "en", "location": "Tampa, FL", "create_at": date("2013-04-21"), "description": "null", "followers_count": 384, "friends_count": 285, "statues_count": 5822 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bushnell, FL", "id": "3b84519ede034bf2", "name": "Bushnell", "place_type": "city", "bounding_box": rectangle("-82.14301,28.653914 -82.098675,28.683212") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12119, "countyName": "Sumter", "cityID": 1209625, "cityName": "Bushnell" } }
+{ "create_at": datetime("2016-01-03T10:00:34.000Z"), "id": 683709549711351808, "text": "Drinking a Cack-a-lacky™ by @fullsteam at @dpac — https://t.co/pk8h3hEvR8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.9022,35.9935"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 10257492, 14718975 }}, "user": { "id": 1335080995, "name": "Jeff Hall", "screen_name": "MrWendell24", "lang": "en", "location": "Greensboro, NC", "create_at": date("2013-04-07"), "description": "null", "followers_count": 70, "friends_count": 156, "statues_count": 1000 }, "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-03T10:00:34.000Z"), "id": 683709549799432193, "text": "@seitzybaby gotta do you bro !", "in_reply_to_status": 683709431515885569, "in_reply_to_user": 386317053, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 386317053 }}, "user": { "id": 295458084, "name": "SMASH WILLIAMS", "screen_name": "carson_willie", "lang": "en", "location": "null", "create_at": date("2011-05-08"), "description": "hi im carson i enjoy long walks on the beach and @a_bunk", "followers_count": 646, "friends_count": 491, "statues_count": 23738 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hilltop, OH", "id": "012804cc7b82ba2c", "name": "Hilltop", "place_type": "city", "bounding_box": rectangle("-80.768834,41.147302 -80.732769,41.170461") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39155, "countyName": "Trumbull", "cityID": 3935613, "cityName": "Hilltop" } }
+{ "create_at": datetime("2016-01-03T10:00:34.000Z"), "id": 683709550411825157, "text": "I need to puff ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4336738096, "name": "Kayla Cummings", "screen_name": "Italyprincess6", "lang": "en", "location": "Binghamton, NY", "create_at": date("2015-11-30"), "description": "All about the money ,love to travel ✈ Selfie queen come follow me on my wild journey, Proud French, Italian, and Irish ❤", "followers_count": 68, "friends_count": 240, "statues_count": 131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Binghamton, NY", "id": "859b7f1502db8fe9", "name": "Binghamton", "place_type": "city", "bounding_box": rectangle("-75.95015,42.067082 -75.857034,42.156489") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36007, "countyName": "Broome", "cityID": 3606607, "cityName": "Binghamton" } }
+{ "create_at": datetime("2016-01-03T10:00:34.000Z"), "id": 683709550462144515, "text": "\"spill milk on mommis chest, kentucky!\" is what i \"kept\" saying to kassie. lol!", "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. King, Of Tennessee. Married. 25. BLACKBILLY. I'm Silly. SloppyTweeter. Kid Komedian: IN PROGRESS. #BOSSING.", "followers_count": 35, "friends_count": 74, "statues_count": 32507 }, "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-03T10:00:34.000Z"), "id": 683709550675902464, "text": "I like Shawn Mendes as much as you can possibly like a person without wanting to fuck 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": 190180205, "name": "/////", "screen_name": "gianthotdogs", "lang": "en", "location": "Califournia", "create_at": date("2010-09-13"), "description": "(she/her)\n\nwildly bitter; Keanu Reeves is my dad", "followers_count": 758, "friends_count": 433, "statues_count": 69312 }, "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-03T10:00:34.000Z"), "id": 683709550831079427, "text": "Head, baccwood and animie https://t.co/Pi24RuAxlZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 430221250, "name": "GOD'S STEP SON", "screen_name": "L0USER_", "lang": "en", "location": "California, USA", "create_at": date("2011-12-06"), "description": "Joe Budden blocked me cause he sensitive", "followers_count": 1309, "friends_count": 633, "statues_count": 46363 }, "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-03T10:00:34.000Z"), "id": 683709550889840640, "text": "The family Kwanzaa selfie ������ @ D.C\\Maryland\\Virginia https://t.co/w5cnuteoLR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.9234118,38.7821643"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 60456784, "name": "Nu' The Mayor", "screen_name": "NuTheMayor", "lang": "en", "location": "Born Uptown DC/lives in ATL", "create_at": date("2009-07-26"), "description": "#GrindAndShine coming soon! Rapper|Radio host of #NuTheMayorShow @TheBlackCNN Sundays 7pm - 9pm only on @RippedRadio| CEO @GOVLife | nuthemayor@gmail.com", "followers_count": 1973, "friends_count": 1490, "statues_count": 84783 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clinton, MD", "id": "63c27109cee3362f", "name": "Clinton", "place_type": "city", "bounding_box": rectangle("-76.973314,38.702045 -76.843494,38.7986") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2417900, "cityName": "Clinton" } }
+{ "create_at": datetime("2016-01-03T10:00:34.000Z"), "id": 683709551124844544, "text": "@DearIgnazio @DemosAmorArg @piero_barone grazie per dividere questa bellezza con me ��PB my munchkin-pooh ��❤❤❤", "in_reply_to_status": 683655128537280513, "in_reply_to_user": 2906091356, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user_mentions": {{ 2906091356, 590356494, 465908990 }}, "user": { "id": 22061601, "name": "wAngelgirlDorough", "screen_name": "AngelgirlD", "lang": "en", "location": "Howie Dorough's World!", "create_at": date("2009-02-26"), "description": "null", "followers_count": 155, "friends_count": 41, "statues_count": 6980 }, "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-03T10:00:34.000Z"), "id": 683709551137275906, "text": "@M_CookinItUp false ��", "in_reply_to_status": 683706533788581889, "in_reply_to_user": 293517059, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 293517059 }}, "user": { "id": 39322030, "name": "Jessyka bailey", "screen_name": "Bailelicious", "lang": "en", "location": "Creative//Skateboarder", "create_at": date("2009-05-11"), "description": "Tempe, http://shootbailey.com | IG: FridgeDigital", "followers_count": 696, "friends_count": 329, "statues_count": 13288 }, "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-04T10:00:00.000Z"), "id": 684071794924388354, "text": "LMFAOOOOOOOOO", "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": 592, "friends_count": 523, "statues_count": 38502 }, "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-04T10:00:00.000Z"), "id": 684071794953818112, "text": "@WIRED @megadadsblog thoughts?", "in_reply_to_status": 684053985058263041, "in_reply_to_user": 1344951, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1344951, 2322797365 }}, "user": { "id": 1725683666, "name": "WoWMeaghan", "screen_name": "WoWmeaghan", "lang": "en", "location": "Holland, MI", "create_at": date("2013-09-03"), "description": "I'm a mom nerd, main squeeze is @Warcraft and Halo 5! Marketing student & Queen of the Dwarves", "followers_count": 900, "friends_count": 1997, "statues_count": 2108 }, "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-01-04T10:00:00.000Z"), "id": 684071795079512064, "text": "Want to work at J.Crew Factory? We're #hiring in #Paramus, NJ! Click for details: https://t.co/FhNvJsSGTm #Retail #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0754189,40.9445428"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Paramus", "Retail", "Job", "Jobs", "CareerArc" }}, "user": { "id": 28480310, "name": "TMJ-NJN Retail Jobs", "screen_name": "tmj_njn_retail", "lang": "en", "location": "Northern NJ", "create_at": date("2009-04-02"), "description": "Follow this account for geo-targeted Retail job tweets in Northern NJ. Need help? Tweet us at @CareerArc!", "followers_count": 443, "friends_count": 314, "statues_count": 372 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paramus, NJ", "id": "4b6a1f748c5219fa", "name": "Paramus", "place_type": "city", "bounding_box": rectangle("-74.101395,40.908544 -74.038944,40.980598") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3455950, "cityName": "Paramus" } }
+{ "create_at": datetime("2016-01-04T10:00:00.000Z"), "id": 684071795217940480, "text": "@diplo tonight ������������", "in_reply_to_status": -1, "in_reply_to_user": 17174309, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17174309 }}, "user": { "id": 3494897665, "name": "Vic", "screen_name": "victro25", "lang": "en", "location": "null", "create_at": date("2015-09-08"), "description": "sinning on my mind. #TxSU19 ♏️", "followers_count": 138, "friends_count": 129, "statues_count": 1890 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-04T10:00:00.000Z"), "id": 684071795553472512, "text": "\"@awakendmt: Just because you can't see it, Doesn't mean mental illness doesn't exist. https://t.co/QeJIZ7573i\" exactly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-100.3825548,31.4885081"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1962569347 }}, "user": { "id": 313765215, "name": "Vegeta", "screen_name": "EthanMBrown", "lang": "en", "location": "Hampton, VA to 325/806ix", "create_at": date("2011-06-08"), "description": "Straight Outta Bompton. Toyota MR2 Turbo. Snapchat & Instagram: Ethanb14", "followers_count": 1181, "friends_count": 722, "statues_count": 50198 }, "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-04T10:00:00.000Z"), "id": 684071795553611776, "text": "Temp: 64.8°F | Humidity: 19% | Wind: N @ 0.0 mph | Barometer: 30.03 in | Dewpoint: 21.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": 23, "friends_count": 1, "statues_count": 164228 }, "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-04T10:00:00.000Z"), "id": 684071795566186497, "text": "@damnsonnierboy dats my girl Emily", "in_reply_to_status": 684071627873718272, "in_reply_to_user": 48927372, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 48927372 }}, "user": { "id": 308830032, "name": "sass", "screen_name": "NewYearNewAt", "lang": "en", "location": "Between the Hedges ", "create_at": date("2011-05-31"), "description": "I'm just a girl with a high GPA and an iPhone full of trap music", "followers_count": 573, "friends_count": 109, "statues_count": 28459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodstock, GA", "id": "986b24341eced063", "name": "Woodstock", "place_type": "city", "bounding_box": rectangle("-84.556535,34.074044 -84.449598,34.147508") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13057, "countyName": "Cherokee", "cityID": 1384176, "cityName": "Woodstock" } }
+{ "create_at": datetime("2016-01-04T10:00:00.000Z"), "id": 684071795691945985, "text": "@Is_ThAt_DC it's a sensorpedic bro", "in_reply_to_status": 684071666738008065, "in_reply_to_user": 226019377, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 226019377 }}, "user": { "id": 35631003, "name": "#ChosenOne", "screen_name": "Peti___", "lang": "en", "location": "Silsbee, Texas", "create_at": date("2009-04-26"), "description": "#RIPJakaylaB", "followers_count": 1066, "friends_count": 651, "statues_count": 31897 }, "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-01-04T10:00:00.000Z"), "id": 684071795918516225, "text": "#Breaking: CCSO searching for suspects after reported shots fired near Kano St. (near Bohicket Rd.) on Johns Island. #chsnews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Breaking", "chsnews" }}, "user": { "id": 211016708, "name": "Melissa Egan", "screen_name": "Melissa_Live5", "lang": "en", "location": " Charleston, SC", "create_at": date("2010-11-01"), "description": "Producer @Live5News, WCSC. UF Alumni. Go Gators! #Chomp.", "followers_count": 844, "friends_count": 425, "statues_count": 4077 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charleston, SC", "id": "7422f29331ca8257", "name": "Charleston", "place_type": "city", "bounding_box": rectangle("-80.134867,32.673231 -79.879867,32.866916") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45019, "countyName": "Charleston", "cityID": 4513330, "cityName": "Charleston" } }
+{ "create_at": datetime("2016-01-04T10:00:00.000Z"), "id": 684071796191158272, "text": "@t_fitz13 Lol I seen it! He look like Josh Barker ����", "in_reply_to_status": 684066378115067904, "in_reply_to_user": 248267973, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 248267973 }}, "user": { "id": 101404349, "name": "Money Man P", "screen_name": "KingVCURam", "lang": "en", "location": "Danraq to RVA", "create_at": date("2010-01-02"), "description": "VCU '16.", "followers_count": 53, "friends_count": 38, "statues_count": 491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Martinsville, VA", "id": "00225f886fc1c2ff", "name": "Martinsville", "place_type": "city", "bounding_box": rectangle("-79.910545,36.609078 -79.802055,36.715335") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51690, "countyName": "Martinsville", "cityID": 5149784, "cityName": "Martinsville" } }
+{ "create_at": datetime("2016-01-04T10:00:00.000Z"), "id": 684071796224561152, "text": "Edm old to me already. It all sounds the same and seems like you gotta be really messed up to listen. No thanks ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 500405356, "name": "C. Hurst", "screen_name": "Callan_Hurst", "lang": "en", "location": "Texas State! ", "create_at": date("2012-02-22"), "description": "Life is what you make it. What doesn't kill you makes you stronger. If you woke up today, you have no reason to be mad. #RIPROBINANN #CowboysNation", "followers_count": 1192, "friends_count": 867, "statues_count": 15025 }, "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-04T10:00:00.000Z"), "id": 684071796610625536, "text": "Never saw so many girls wearing pink ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 596909134, "name": "alex ♡", "screen_name": "alexrajere", "lang": "en", "location": "null", "create_at": date("2012-06-01"), "description": "Cole is King", "followers_count": 978, "friends_count": 316, "statues_count": 23306 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Southport, NY", "id": "50d23f3c2aff80d3", "name": "Southport", "place_type": "city", "bounding_box": rectangle("-76.902403,42.014436 -76.745063,42.092362") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36015, "countyName": "Chemung", "cityID": 3669606, "cityName": "Southport" } }
+{ "create_at": datetime("2016-01-04T10:00:00.000Z"), "id": 684071796627386370, "text": "Did the holiday season cause you to fall behind at work? If so, our print and business services can help you out of a jam.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2680425499, "name": "The UPS Store 4084", "screen_name": "TheUPSStore4084", "lang": "en", "location": "Falls Pointe, Raleigh, NC", "create_at": date("2014-07-25"), "description": "Serving North Raleigh since 2001.", "followers_count": 32, "friends_count": 15, "statues_count": 298 }, "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-04T10:00:00.000Z"), "id": 684071796837068800, "text": "Very high liquor feel to it. - Drinking a StonerWitch by @tiredhandsbeer @ Tasting Table — https://t.co/WeNuE8lnHd #photo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.2905,40.0076"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "photo" }}, "user_mentions": {{ 235595866 }}, "user": { "id": 239504136, "name": "✖️Knuckle Punk✖️", "screen_name": "danbrah", "lang": "en", "location": "Melrose Diner", "create_at": date("2011-01-17"), "description": "Just a hardcore kid living in a hardcore world. I'm in the Navy. I hate you all. Except the Flyers...and The Wonder Years...#LC", "followers_count": 540, "friends_count": 947, "statues_count": 2686 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ardmore, PA", "id": "1f5306f35e51eee4", "name": "Ardmore", "place_type": "city", "bounding_box": rectangle("-75.32233,39.990009 -75.273474,40.018207") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42091, "countyName": "Montgomery", "cityID": 4202896, "cityName": "Ardmore" } }
+{ "create_at": datetime("2016-01-04T10:00:00.000Z"), "id": 684071797013131264, "text": "Zoës Kitchen: Kitchen Leader (#BIRMINGHAM, AL) https://t.co/NTdw5XyDUe #Hospitality #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.80249,33.5206608"), "retweet_count": 0, "lang": "fi", "is_retweet": false, "hashtags": {{ "BIRMINGHAM", "Hospitality", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22455262, "name": "TMJ-BHM HRTA Jobs", "screen_name": "tmj_bhm_hrta", "lang": "en", "location": "Birmingham, AL", "create_at": date("2009-03-01"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Birmingham, AL. Need help? Tweet us at @CareerArc!", "followers_count": 337, "friends_count": 291, "statues_count": 285 }, "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-04T10:00:00.000Z"), "id": 684071797013241856, "text": "I'm at Jojo's Barbershop in Fayetteville, NC https://t.co/QEyjm2cmlD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.00920106,35.08238129"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2788363293, "name": "Swole Lightweight", "screen_name": "seriouscook", "lang": "en", "location": "Lost....don't follow me!!! ", "create_at": date("2014-09-27"), "description": "Doing me....not worried about the rest. Working toward my goals and living my life.", "followers_count": 42, "friends_count": 171, "statues_count": 1027 }, "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-04T10:00:00.000Z"), "id": 684071797034233856, "text": "Ripley SW Limestone Co. Temp: 36.0°F Wind:5.4mph Pressure: 1006.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": 42, "friends_count": 33, "statues_count": 47046 }, "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-04T10:00:00.000Z"), "id": 684071797109555200, "text": "@iHeartRadio @ToriKelly Please play #History by @onedirection", "in_reply_to_status": 684068048815259648, "in_reply_to_user": 46116615, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "History" }}, "user_mentions": {{ 46116615, 19017237, 209708391 }}, "user": { "id": 1135599187, "name": "HARRY'S FOOL", "screen_name": "Foolish4Harry", "lang": "en", "location": "stumbling through the dark ", "create_at": date("2013-01-30"), "description": "This is not the end", "followers_count": 4837, "friends_count": 2483, "statues_count": 299820 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:00.000Z"), "id": 684071797243817984, "text": "@AndrewHall343 Thank You �� Andrew", "in_reply_to_status": -1, "in_reply_to_user": 3309165093, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3309165093 }}, "user": { "id": 22189163, "name": "Dr Carl Metzger", "screen_name": "ClearMindedness", "lang": "en", "location": "Bend, OR", "create_at": date("2009-02-27"), "description": "ClearMindedness phone talking/books-917.675.0781 http://ClearMindedness.com", "followers_count": 38917, "friends_count": 33259, "statues_count": 230191 }, "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-01-04T10:00:00.000Z"), "id": 684071797273276416, "text": "Garlic Caesar salad was a bad choice. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34146944, "name": "mily not miley.", "screen_name": "milyonaire", "lang": "en", "location": "Over the hills and far away", "create_at": date("2009-04-21"), "description": "Slushee Connoisseur. Landon fangirl. Freddie Mercury Enthusiast. Mrs Mojo Risin. The girl Robert Plant ❤ w/ the long black wavy hair. #USMNT #Bayern #LetsGoFish", "followers_count": 444, "friends_count": 707, "statues_count": 27995 }, "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-01-04T10:00:00.000Z"), "id": 684071797415899138, "text": "Ima turn into the boy soon", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 290836346, "name": "Isaiah Matthew Stone", "screen_name": "Freakyzeeky8", "lang": "en", "location": "EAST CHARLOTTE idlewild ", "create_at": date("2011-04-30"), "description": "304➡️704 its alot of things I couldn't have did without God.... (Im a new man) #F4mily really does matter--- #WeTheBoyz", "followers_count": 1019, "friends_count": 714, "statues_count": 51924 }, "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-04T10:00:00.000Z"), "id": 684071797726294016, "text": "Blunts & coffee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2957628704, "name": "LaleBron James", "screen_name": "LaleDewis", "lang": "en", "location": "Westville, IL", "create_at": date("2015-01-03"), "description": "IG & Snapchat: lale_dewis • Just an ugly mf who be doing a lot • @FATTREL is my real dad • Follow the Baby 5 @Baby___Gang • #BroncosNation", "followers_count": 468, "friends_count": 115, "statues_count": 17197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westville, IL", "id": "1717fe79c0ff883d", "name": "Westville", "place_type": "city", "bounding_box": rectangle("-87.649669,40.020534 -87.624305,40.065486") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17183, "countyName": "Vermilion", "cityID": 1780931, "cityName": "Westville" } }
+{ "create_at": datetime("2016-01-04T10:00:00.000Z"), "id": 684071797734699009, "text": "���� @ibeesthablock_ @Dem_loading https://t.co/hV5MzySxl1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 169791174, 541217923 }}, "user": { "id": 282818271, "name": "Sebastian K.", "screen_name": "SonOfMalcolm", "lang": "en", "location": "Not Tennessee", "create_at": date("2011-04-15"), "description": "I'm lost right now, but I'll find my way . . #TxSU19", "followers_count": 1272, "friends_count": 1249, "statues_count": 34427 }, "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-01-04T10:00:00.000Z"), "id": 684071797738876928, "text": "@CleveSmith you eat sardines bih", "in_reply_to_status": 684070866393563136, "in_reply_to_user": 142173788, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 142173788 }}, "user": { "id": 297521181, "name": "Johnny Boy", "screen_name": "johnnyhannan247", "lang": "en", "location": "Miami FL", "create_at": date("2011-05-12"), "description": "stay in the fight... #TheU R&B IG Joe_nay 21", "followers_count": 680, "friends_count": 714, "statues_count": 46465 }, "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-01-04T10:00:00.000Z"), "id": 684071798242172931, "text": "Chapels are cool. @ Bellarmine University https://t.co/RvGVDj0FsN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.70414783,38.21910928"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 152046307, "name": "ColTRON", "screen_name": "iColtonSmith", "lang": "en", "location": "null", "create_at": date("2010-06-04"), "description": "“To see the world, things dangerous to come to, to see behind walls, draw closer, to find each other and to feel. That is the purpose of life.”", "followers_count": 319, "friends_count": 379, "statues_count": 5393 }, "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-01-04T10:00:00.000Z"), "id": 684071798355279872, "text": "I just realized I didn't get it back �� ruined the yearly tradition ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 616229375, "name": "Chris Gonzales", "screen_name": "_OfficialChris", "lang": "en", "location": "Ceres, CA", "create_at": date("2012-06-23"), "description": "Hi, I'm 17, writing and producing music , Follow me! :)", "followers_count": 785, "friends_count": 16, "statues_count": 3996 }, "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-04T10:00:00.000Z"), "id": 684071798413996035, "text": "Systems Administrator for Growing IT Team!! - Robert Half Technology: (#FISHERS, IN) https://t.co/jiYL9z6f0K #IT #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.0223611,39.9645049"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FISHERS", "IT", "Job", "Jobs", "Hiring" }}, "user": { "id": 99429860, "name": "TMJ-IND IT Adm. Jobs", "screen_name": "tmj_IND_adm", "lang": "en", "location": "Indianapolis, IN", "create_at": date("2009-12-25"), "description": "Follow this account for geo-targeted IT-Support/Administration job tweets in Indianapolis, IN from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 300, "friends_count": 279, "statues_count": 61 }, "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-01-04T10:00:00.000Z"), "id": 684071798430826496, "text": "Might drop this class cuz the teacher writes on a chalk board wall and that sound 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": 1732458884, "name": "Michael Vielma", "screen_name": "Michael_Vielma", "lang": "en", "location": "null", "create_at": date("2013-09-05"), "description": "Retweet me so I know it's real.", "followers_count": 161, "friends_count": 145, "statues_count": 1387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:00.000Z"), "id": 684071798468575232, "text": "@JFD0912 Is this the REAL @EhmTeeGee? Welcome to Twitter! We've been waiting for you. #wherehaveyoubeenallourlives https://t.co/WbHqWWkLzS", "in_reply_to_status": -1, "in_reply_to_user": 143257048, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wherehaveyoubeenallourlives" }}, "user_mentions": {{ 143257048, 4709355584 }}, "user": { "id": 422150704, "name": "Ron Fooks Jr.", "screen_name": "refooksjr", "lang": "en", "location": "Los Angeles", "create_at": date("2011-11-26"), "description": "elementary school teacher, trivia host, dui book member, 2 hand toucher, philly sports phan", "followers_count": 290, "friends_count": 813, "statues_count": 1906 }, "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-04T10:00:00.000Z"), "id": 684071798489526272, "text": "@MisterEd5 @DarrenDegaetano As dysfunctional as #Eagles have been #Dolphins and #Browns are in far worse shape in front office", "in_reply_to_status": 684071132710989825, "in_reply_to_user": 1028877696, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Eagles", "Dolphins", "Browns" }}, "user_mentions": {{ 1028877696, 353438885 }}, "user": { "id": 30291637, "name": "John McMullen", "screen_name": "JFMcMullen", "lang": "en", "location": "Philadelphia", "create_at": date("2009-04-10"), "description": "National NFL columnist @fanragsports @TodaysPigskin, #Eagles insider for @973espn - talker on YAHOO!, ESPN & FOX Sports Radio. jmcmullen@phanaticmag.com", "followers_count": 1184, "friends_count": 2017, "statues_count": 23121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stratford, NJ", "id": "80254a7349f2a7cf", "name": "Stratford", "place_type": "city", "bounding_box": rectangle("-75.038474,39.818965 -74.993797,39.841525") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34007, "countyName": "Camden", "cityID": 3471220, "cityName": "Stratford" } }
+{ "create_at": datetime("2016-01-04T10:00:00.000Z"), "id": 684071798581821440, "text": "Bro really died !?!?! �� https://t.co/rd31V5iBZW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3258044241, "name": "Big Boss Daddy.", "screen_name": "zayrobbins62", "lang": "en", "location": "null", "create_at": date("2015-05-15"), "description": "Dobson football #52", "followers_count": 364, "friends_count": 285, "statues_count": 3770 }, "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-04T10:00:00.000Z"), "id": 684071798720180228, "text": "this girl done seriously fucked over my whole mood", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 301816433, "name": "dudua", "screen_name": "hotboidu", "lang": "en", "location": "null", "create_at": date("2011-05-19"), "description": "II.II.XV ❤️ #newyearnewmoney", "followers_count": 2054, "friends_count": 697, "statues_count": 91642 }, "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-04T10:00:00.000Z"), "id": 684071798787457024, "text": "If someone tells you that you've hurt them, you DONT get to decide that you didn't", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 316004992, "name": "Shayy-Nuhh☯✝♊️", "screen_name": "_DamnItsShaix3", "lang": "en", "location": "••3 3 6••", "create_at": date("2011-06-12"), "description": "IG; shayy_nuhhh || Jan122015❤️", "followers_count": 961, "friends_count": 941, "statues_count": 64959 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clemmons, NC", "id": "290f62c6f654e14f", "name": "Clemmons", "place_type": "city", "bounding_box": rectangle("-80.432988,35.93042 -80.339893,36.078123") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37067, "countyName": "Forsyth", "cityID": 3712960, "cityName": "Clemmons" } }
+{ "create_at": datetime("2016-01-04T10:00:00.000Z"), "id": 684071799039115265, "text": "SE ACABARON LAS MENTIRAS!!!!! https://t.co/djFcITG0bh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.95395795,40.68269521"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 134590913, "name": "AP0L0", "screen_name": "apolo_7", "lang": "es", "location": "New York", "create_at": date("2010-04-18"), "description": "Venezuelan who moved to NYC trying to live the Big American Dream. Be open to sarcasm, do NOT take my tweets seriously.", "followers_count": 829, "friends_count": 151, "statues_count": 41995 }, "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-04T10:00:01.000Z"), "id": 684071799345156096, "text": "#Goodyear, AZ #Healthcare #Job: Facility Coder at Cancer Treatment Centers of America https://t.co/SdzlW7AdZu #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.3596206,33.4528164"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Goodyear", "Healthcare", "Job", "Jobs", "Hiring" }}, "user": { "id": 128004921, "name": "CTCA Jobs", "screen_name": "CTCAJobs", "lang": "en", "location": "#AZ #GA #IL #OK #PA #FL", "create_at": date("2010-03-30"), "description": "Here you'll find our latest job opportunities across the U.S. Follow @CTCACareers to learn more & ask us questions! #jobs #careers #healthcare", "followers_count": 844, "friends_count": 484, "statues_count": 10735 }, "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-04T10:00:01.000Z"), "id": 684071799437549568, "text": "@Kysociety ☺️ https://t.co/6FOT3O5GqL", "in_reply_to_status": -1, "in_reply_to_user": 352694814, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 352694814 }}, "user": { "id": 2411388691, "name": "Jamaya Dandridge", "screen_name": "Mayad__", "lang": "en", "location": "null", "create_at": date("2014-03-25"), "description": "IG\\\\ @mayad__ ✨", "followers_count": 768, "friends_count": 278, "statues_count": 8732 }, "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-04T10:00:01.000Z"), "id": 684071799722766337, "text": "���� https://t.co/6kkRAsLvso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 32784989, "name": "Ajami14", "screen_name": "HamoudyAjami14", "lang": "en", "location": "Detroit, MI", "create_at": date("2009-04-17"), "description": "#Lebanese Snap-Ajami14 Instagram HamoudyAjami14 Proud Fan OF The SanFranCisco 49ers #gemini Dont let anyone steal your dreams ... East Lansing..", "followers_count": 649, "friends_count": 790, "statues_count": 13360 }, "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-01-04T10:00:01.000Z"), "id": 684071799777185792, "text": "@faith_peery so I need to talk. ��", "in_reply_to_status": 684071755032301568, "in_reply_to_user": 570496866, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 570496866 }}, "user": { "id": 2236729050, "name": "B", "screen_name": "brookevb15", "lang": "en", "location": "TEXAS", "create_at": date("2013-12-08"), "description": "I ain't pageant material the only crown is in my glass.", "followers_count": 867, "friends_count": 1221, "statues_count": 24440 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Columbia, TX", "id": "784d3cc53ed49744", "name": "West Columbia", "place_type": "city", "bounding_box": rectangle("-95.668535,29.120079 -95.611306,29.172655") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4877416, "cityName": "West Columbia" } }
+{ "create_at": datetime("2016-01-04T10:00:01.000Z"), "id": 684071799856865280, "text": "https://t.co/CuXqKexFHV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 875007740, "name": "kRiderGeekNationNews", "screen_name": "Krider57253355", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-10-11"), "description": "I'm a tokasatsu sentai Kamen Rider anime & comic book fanatic who likes to share what he likes so Check out my youtube channel", "followers_count": 225, "friends_count": 703, "statues_count": 1924 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:01.000Z"), "id": 684071800339300352, "text": "Yours must be pretty sucky , because you seem pretty jealous ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1258418714, "name": "IndiraMelisa", "screen_name": "indiraaaK", "lang": "en", "location": "Queens, NY", "create_at": date("2013-03-10"), "description": "add my snap! -indiramelisax3", "followers_count": 190, "friends_count": 174, "statues_count": 3861 }, "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-04T10:00:01.000Z"), "id": 684071800397942789, "text": "@webster https://t.co/KiqypV6rnt", "in_reply_to_status": 684052862851129344, "in_reply_to_user": 7632132, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 7632132 }}, "user": { "id": 22330197, "name": "Albiniak", "screen_name": "malbiniak", "lang": "en", "location": "Portland, Ore / Kepler 452-b", "create_at": date("2009-02-28"), "description": "People tell me I remind them of Howard Hughes (the late years). Opinions are mine, bozo. #RCTID #LGRW", "followers_count": 1907, "friends_count": 356, "statues_count": 17042 }, "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-04T10:00:01.000Z"), "id": 684071800536317952, "text": "Should I rock adidas spikes or Nike ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2198723772, "name": "Tremaine Henry™", "screen_name": "20_yeezus", "lang": "en", "location": "Alexandria,La", "create_at": date("2013-11-16"), "description": "Alexandria Sr. High Product| Track God | C/o 2k16", "followers_count": 1036, "friends_count": 2039, "statues_count": 19507 }, "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-01-04T10:00:01.000Z"), "id": 684071800725098497, "text": "https://t.co/KjUJM5UHNW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 117574545, "name": "Mike and Deb Granado", "screen_name": "MnDgranado2008", "lang": "en", "location": "chicago- south side", "create_at": date("2010-02-25"), "description": "grifter and drifter", "followers_count": 24, "friends_count": 35, "statues_count": 306 }, "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-04T10:00:01.000Z"), "id": 684071800880377856, "text": "@Arsh_Ley ������", "in_reply_to_status": 684071466846040064, "in_reply_to_user": 559774659, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 559774659 }}, "user": { "id": 190885097, "name": "Tràcy", "screen_name": "tracepeacelove", "lang": "en", "location": "sorta belong in the west cost", "create_at": date("2010-09-14"), "description": "null", "followers_count": 355, "friends_count": 236, "statues_count": 43855 }, "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-04T10:00:01.000Z"), "id": 684071800947523584, "text": "Savage ASF �� https://t.co/i2g9M90nH0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3111626946, "name": "‼️❌El Despiadado❌‼️", "screen_name": "ITSNATEJACK", "lang": "en", "location": "null", "create_at": date("2015-03-27"), "description": "# Luke 23:34 ❤️MT❤️", "followers_count": 356, "friends_count": 275, "statues_count": 965 }, "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-04T10:00:01.000Z"), "id": 684071801178161153, "text": "I need to get my hair done . But I don't wanna get it done on 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": 233425321, "name": "DrewHen", "screen_name": "drewINGTON", "lang": "en", "location": "in your panties", "create_at": date("2011-01-02"), "description": "Fighting on Arrival, Fighting for Survival http://mixcrate.com/drewington LISTEN TO MY MIXES!", "followers_count": 1297, "friends_count": 780, "statues_count": 183993 }, "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-04T10:00:01.000Z"), "id": 684071801375215617, "text": "We're #hiring! Read about our latest #job opening here: PAINTER - https://t.co/VQZvreDRS0 #va #Williamsburg, VA #Construction", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.6350285,37.2668356"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "va", "Williamsburg", "Construction" }}, "user": { "id": 3015648286, "name": "King's Creek Jobs", "screen_name": "KingsCreekJobs", "lang": "en", "location": "null", "create_at": date("2015-02-03"), "description": "Check here for current job opportunities with King's Creek Plantation, in beautiful Williamsburg, VA.", "followers_count": 21, "friends_count": 46, "statues_count": 5 }, "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": 51199, "countyName": "York" } }
+{ "create_at": datetime("2016-01-04T10:00:01.000Z"), "id": 684071801710723072, "text": "#Hospitality #Job: Shift Supervisor (US) at Starbucks https://t.co/gqrwIK9TNT #Veterans #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.4644673,33.9400371"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "Job", "Veterans", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 88113750, "name": "TMJ-GA HRTA Jobs", "screen_name": "tmj_ga_hrta", "lang": "en", "location": "Georgia", "create_at": date("2009-11-06"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Georgia Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 400, "friends_count": 288, "statues_count": 1386 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Athens, GA", "id": "01f9c9fd7bb1aa6b", "name": "Athens", "place_type": "city", "bounding_box": rectangle("-83.521866,33.877554 -83.277693,34.035918") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13059, "countyName": "Clarke", "cityID": 1303440, "cityName": "Athens-Clarke County unified government (balance)" } }
+{ "create_at": datetime("2016-01-04T10:00:01.000Z"), "id": 684071801710874625, "text": "Paying rent in my pajamas just always makes me feel like an adult.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 403099105, "name": "Sean Reeves", "screen_name": "TheSeanReeves", "lang": "en", "location": "Cincinnati, OH", "create_at": date("2011-11-01"), "description": "Have fun, regret nothing, and write a book about it later", "followers_count": 181, "friends_count": 227, "statues_count": 4662 }, "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-04T10:00:01.000Z"), "id": 684071801757020160, "text": "Military homecoming videos get me crying����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 383585886, "name": "andreaaa", "screen_name": "andreapapp11", "lang": "en", "location": "null", "create_at": date("2011-10-01"), "description": "18. NYC. Binghamton University '19", "followers_count": 282, "friends_count": 280, "statues_count": 9619 }, "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-04T10:00:01.000Z"), "id": 684071801979322370, "text": "Let me go ahead and take off these dates that I need for homecoming ��", "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": 2383, "friends_count": 1900, "statues_count": 64686 }, "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-01-04T10:00:01.000Z"), "id": 684071802092425216, "text": "@zarinatweets No, just holiday stuff. I got hand sanitizer for work that was 10c each.", "in_reply_to_status": 684071175933300736, "in_reply_to_user": 16947427, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16947427 }}, "user": { "id": 65655993, "name": "ella", "screen_name": "ellaellabobella", "lang": "en", "location": "null", "create_at": date("2009-08-14"), "description": "freelance shenanigan consultant", "followers_count": 98, "friends_count": 126, "statues_count": 3266 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kuna, ID", "id": "e96d79ee36026a02", "name": "Kuna", "place_type": "city", "bounding_box": rectangle("-116.453164,43.473902 -116.398828,43.521123") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16001, "countyName": "Ada", "cityID": 1644290, "cityName": "Kuna" } }
+{ "create_at": datetime("2016-01-04T10:00:01.000Z"), "id": 684071802163871744, "text": "it's snowing omg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2735798499, "name": "madi", "screen_name": "madison_rotondo", "lang": "en", "location": "Everett, MA", "create_at": date("2014-08-06"), "description": "null", "followers_count": 106, "friends_count": 258, "statues_count": 2016 }, "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-01-04T10:00:01.000Z"), "id": 684071802708992000, "text": "#Accounting #Job in #LosAngeles, CA: Tax Manager at Robert Half Management Resources https://t.co/XT0Am12nh8 #RHMRJobs #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.2548551,34.052996"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Accounting", "Job", "LosAngeles", "RHMRJobs", "Jobs", "Hiring" }}, "user": { "id": 791930767, "name": "RHMR Jobs", "screen_name": "RHMRJobs", "lang": "en", "location": "null", "create_at": date("2012-08-30"), "description": "Robert Half Management Resources is the world's premier provider of senior-level accounting, finance & business systems professionals. See our newest jobs here!", "followers_count": 105, "friends_count": 8, "statues_count": 830 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:01.000Z"), "id": 684071802914471936, "text": "This is wild —> RT @kwameopam: Facebook made its Android app crash to test your loyalty https://t.co/XxmPz5SgS4 https://t.co/IWUfTyUIpJ", "in_reply_to_status": 684071533178961924, "in_reply_to_user": 20728459, "favorite_count": 0, "coordinate": point("-115.15487664,36.13254865"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20728459 }}, "user": { "id": 69426451, "name": "Casey Newton", "screen_name": "CaseyNewton", "lang": "en", "location": "San Francisco", "create_at": date("2009-08-27"), "description": "Silicon Valley editor @verge. Freakishly tall, anime hair. My Starbucks name is Tasty. (DMs open but pitches are ignored. Email me instead.)", "followers_count": 26205, "friends_count": 541, "statues_count": 21129 }, "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-04T10:00:02.000Z"), "id": 684071803321487361, "text": "Temp: 45.9°F Wind:2.7mph Pressure: 30.292hpa 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": 55849 }, "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-04T10:00:02.000Z"), "id": 684071803539570688, "text": "@PaulTenorio typo in the graph about heading to Miami", "in_reply_to_status": 684067771114778624, "in_reply_to_user": 59230820, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 59230820 }}, "user": { "id": 68680928, "name": "The Rick", "screen_name": "FatGuyFix", "lang": "en", "location": "Orlando, Fl", "create_at": date("2009-08-25"), "description": "Diehard UCF Knight and doer of many things. Owner of @entertainingorl", "followers_count": 746, "friends_count": 820, "statues_count": 56761 }, "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-04T10:00:02.000Z"), "id": 684071803795456000, "text": "@_bethrussell don't have ur number!!!! :(", "in_reply_to_status": 684071727442345984, "in_reply_to_user": 1353104406, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1353104406 }}, "user": { "id": 500503884, "name": "Stephanie Byrnes", "screen_name": "sjbyrnes33", "lang": "en", "location": "null", "create_at": date("2012-02-22"), "description": "And I think to myself, what a wonderful world", "followers_count": 662, "friends_count": 653, "statues_count": 16549 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Medford, NY", "id": "76520d7cbad3096e", "name": "Medford", "place_type": "city", "bounding_box": rectangle("-73.021438,40.794104 -72.953955,40.85056") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3646404, "cityName": "Medford" } }
+{ "create_at": datetime("2016-01-04T10:00:02.000Z"), "id": 684071803875110912, "text": "Wind 1.0 mph NNW. Barometer 30.517 in, Steady. Temperature 34.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": 118, "friends_count": 0, "statues_count": 110082 }, "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-04T10:00:02.000Z"), "id": 684071803954802688, "text": "Wind 4.0 mph NNE. Barometer 30.408 in, Rising slowly. Temperature 33.1 °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": 16617 }, "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-04T10:00:02.000Z"), "id": 684071804059557888, "text": "Can you recommend anyone for this #HealthWelfare #job? https://t.co/02uyniemqN #Honolulu, HI #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8241911,21.2877723"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HealthWelfare", "job", "Honolulu", "Hiring", "CareerArc" }}, "user": { "id": 23526678, "name": "TMJ- HON Health Jobs", "screen_name": "tmj_hon_health", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Healthcare job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 407, "friends_count": 304, "statues_count": 200 }, "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-04T10:00:02.000Z"), "id": 684071804143407105, "text": "Holy Ffffffffffff having some AMAZING homemade Beef Stew that @Xthefox made. #MouthMurrrs https://t.co/W1fJzwvhHU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MouthMurrrs" }}, "user_mentions": {{ 780338120 }}, "user": { "id": 332616090, "name": "I'm a Pretty Bear", "screen_name": "Its_Begger", "lang": "en", "location": "Michigan", "create_at": date("2011-07-09"), "description": "B-E-Double-Guh-ER!!! That's Me!!!", "followers_count": 1000, "friends_count": 972, "statues_count": 10707 }, "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-01-04T10:00:02.000Z"), "id": 684071804198064129, "text": "Working (@ Comcast in Alpharetta, GA) https://t.co/D423x9sUPf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.26877609,34.05602726"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 81299799, "name": "♈Mishy♈", "screen_name": "MsMiamiMami", "lang": "en", "location": "I'm in the A now!", "create_at": date("2009-10-09"), "description": "From the Bottom but I ALWAYS come out on TOP! Single, educated diva raising my mini diva! I'm my own #TT! #Writer #Poet #Naturalista #Militant #Aries #FutureCEO", "followers_count": 1163, "friends_count": 1810, "statues_count": 16918 }, "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-04T10:00:02.000Z"), "id": 684071804491579392, "text": "2016: Grind • Hustle • Cleanse • Travel • Being Me. ���� #winning #resolutions #KillingIt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "winning", "resolutions", "KillingIt" }}, "user": { "id": 21575163, "name": "Ray Rubio", "screen_name": "RayRubio", "lang": "en", "location": "Los Angeles", "create_at": date("2009-02-22"), "description": "Blogger, World Traveler, and your best friend. Family First. Always keeping it 100 . Instagram: raytakesovertheworld", "followers_count": 415, "friends_count": 223, "statues_count": 7998 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:02.000Z"), "id": 684071804592336899, "text": "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": 59942228, "name": "17thAve / M.T", "screen_name": "TerryPresume", "lang": "en", "location": "#17thAve ", "create_at": date("2009-07-24"), "description": "PunkRap | South Flawda | 17th Ave | ⭕❌ http://soundcloud.com/terry-presume #17thAve", "followers_count": 2184, "friends_count": 3089, "statues_count": 16573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Golden Gate, FL", "id": "0bc912edeeadc71a", "name": "Golden Gate", "place_type": "city", "bounding_box": rectangle("-81.719513,26.168823 -81.686846,26.19952") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12021, "countyName": "Collier", "cityID": 1226300, "cityName": "Golden Gate" } }
+{ "create_at": datetime("2016-01-04T10:00:02.000Z"), "id": 684071804659302400, "text": "need to find a job that's okay with me taking 2+ weeks off each month for all my backpacking/ summiting trips I have planned����⛺️����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 304793501, "name": "Mars", "screen_name": "marianagorceac", "lang": "en", "location": "null", "create_at": date("2011-05-24"), "description": "I enjoy climbing mountains, drinking wine, and dogs", "followers_count": 266, "friends_count": 276, "statues_count": 1942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Auburn, CA", "id": "007b9457220cf355", "name": "Auburn", "place_type": "city", "bounding_box": rectangle("-121.12142,38.859561 -121.05382,38.933349") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 603204, "cityName": "Auburn" } }
+{ "create_at": datetime("2016-01-04T10:00:02.000Z"), "id": 684071804701380608, "text": "Mr Tambourine Man by The Byrds 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": 1320, "friends_count": 0, "statues_count": 1355305 }, "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-04T10:00:02.000Z"), "id": 684071805003247616, "text": "This #Sales #job might be a great fit for you: Sales Representative - Facility Services - https://t.co/EUm9I2iZTH #Cumberland, RI #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.4327778,41.9666667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "job", "Cumberland", "Hiring" }}, "user": { "id": 78995297, "name": "Cintas Careers", "screen_name": "CintasCareers", "lang": "en", "location": "null", "create_at": date("2009-10-01"), "description": "Looking for a career, and not just another job? Consider Cintas, where the spirit is the difference. Follow us for our latest career opportunities.", "followers_count": 256, "friends_count": 2, "statues_count": 9424 }, "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": 44007, "countyName": "Providence" } }
+{ "create_at": datetime("2016-01-04T10:00:02.000Z"), "id": 684071805036826624, "text": "Mostly sunny this afternoon, high 46 (8 C). Low 27 (-3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15953346, "name": "Amarillo Weather", "screen_name": "AmarilloWeather", "lang": "en", "location": "Amarillo, Texas", "create_at": date("2008-08-22"), "description": "Forecasts and advisories for Amarillo, Texas", "followers_count": 4042, "friends_count": 395, "statues_count": 8708 }, "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-01-04T10:00:02.000Z"), "id": 684071805061967873, "text": "@AightBetty no", "in_reply_to_status": 684070209796321280, "in_reply_to_user": 241411870, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 241411870 }}, "user": { "id": 31561855, "name": "Sandra Bland", "screen_name": "COMEUPQUEEN", "lang": "en", "location": "United States", "create_at": date("2009-04-15"), "description": "NYC", "followers_count": 725, "friends_count": 181, "statues_count": 29502 }, "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-04T10:00:02.000Z"), "id": 684071805070348288, "text": "@jadykinz_pj @baddestoutfits come back to sac????", "in_reply_to_status": 681329275878809602, "in_reply_to_user": 543157750, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 543157750, 2985169300 }}, "user": { "id": 2326415504, "name": "H$", "screen_name": "hanahconda", "lang": "en", "location": "Arcata, CA", "create_at": date("2014-02-03"), "description": "humboldt!!!!!!!", "followers_count": 355, "friends_count": 339, "statues_count": 5501 }, "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-01-04T10:00:02.000Z"), "id": 684071805448011777, "text": "Sick of temporary", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3331434095, "name": "VickWillMadeIt", "screen_name": "Vicke_Will", "lang": "en", "location": "Chasing Dreams", "create_at": date("2015-06-17"), "description": "Mean☺️ snapchat: @indievee16 . No IG #SSU17 Warner Robins ✈️Savannah", "followers_count": 740, "friends_count": 707, "statues_count": 5385 }, "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-01-04T10:00:02.000Z"), "id": 684071805561126912, "text": "@kaylamjeter truuuuu��", "in_reply_to_status": 684071684765134848, "in_reply_to_user": 39059753, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 39059753 }}, "user": { "id": 65678981, "name": "Kerstt", "screen_name": "KerstynNyckole", "lang": "en", "location": "Independence, MO", "create_at": date("2009-08-14"), "description": "God is great, beer is good, and people are crazy", "followers_count": 606, "friends_count": 282, "statues_count": 21583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Blue Springs, MO", "id": "24af70ccf8cd2bfb", "name": "Blue Springs", "place_type": "city", "bounding_box": rectangle("-94.337321,38.962193 -94.220922,39.067528") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2906652, "cityName": "Blue Springs" } }
+{ "create_at": datetime("2016-01-04T10:00:02.000Z"), "id": 684071805565325312, "text": "This #Retail #job might be a great fit for you: Sales Associate - https://t.co/qpripo428y #Cullman, AL #Hiring https://t.co/E9v340vKEw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.8436124,34.1748208"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "Cullman", "Hiring" }}, "user": { "id": 2259229658, "name": "maurices careers", "screen_name": "mauricescareers", "lang": "en", "location": "U.S.", "create_at": date("2013-12-23"), "description": "Love Fashion? Join our team! @maurices is a world-class specialty retailer that caters to young-at-heart fashionistas. Check here for #job opportunities.", "followers_count": 343, "friends_count": 26, "statues_count": 2258 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cullman, AL", "id": "6ebac9d8febba273", "name": "Cullman", "place_type": "city", "bounding_box": rectangle("-86.895519,34.125239 -86.795043,34.238319") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1043, "countyName": "Cullman", "cityID": 118976, "cityName": "Cullman" } }
+{ "create_at": datetime("2016-01-04T10:00:02.000Z"), "id": 684071805603065857, "text": "#Outnumbered 11 Days until the tragedy #Benghazi hits the big screen @13hours", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Outnumbered", "Benghazi" }}, "user_mentions": {{ 3288074557 }}, "user": { "id": 1618794644, "name": "Tom P", "screen_name": "JustTom076", "lang": "en", "location": "Somewhere Nice", "create_at": date("2013-07-24"), "description": "Conservative, NRA, 2nd A, Smaller Government, Pro-Life, Retired L.E.O, SWAT, Firearms Instructor, @MichiganStateSpartans, Support our Troops\n#wakeupAmerica", "followers_count": 2410, "friends_count": 2832, "statues_count": 17066 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pontiac, MI", "id": "c13938f4ec2dd011", "name": "Pontiac", "place_type": "city", "bounding_box": rectangle("-83.335007,42.603793 -83.247842,42.699542") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2665440, "cityName": "Pontiac" } }
+{ "create_at": datetime("2016-01-04T10:00:02.000Z"), "id": 684071805636710400, "text": "Where is Oklahoma City on the map? Play the game at https://t.co/FSM7nn90Kw #Oklahoma City", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.5164,35.4676"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Oklahoma" }}, "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": 2511695 }, "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-04T10:00:02.000Z"), "id": 684071806194470912, "text": "Me either it's so slippery https://t.co/3Q9YPtYjPY", "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": "HHS || SC: love_brookiiee || Packers", "followers_count": 489, "friends_count": 460, "statues_count": 6480 }, "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-04T10:00:02.000Z"), "id": 684071806202855424, "text": "@YouGoofyFam how is that ? Are u gonna write him ?...", "in_reply_to_status": 684071626644697088, "in_reply_to_user": 1677235134, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1677235134 }}, "user": { "id": 130327087, "name": "Fredrick", "screen_name": "Fredrickmartinn", "lang": "en", "location": "Wylie, TX", "create_at": date("2010-04-06"), "description": "18, - CLASS OF 2016", "followers_count": 967, "friends_count": 791, "statues_count": 57720 }, "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-04T10:00:02.000Z"), "id": 684071806249009152, "text": "Retail Sales Consultant Pittsburgh PA (Pittsburgh North East) - AT&T: (#Pittsburgh, PA) https://t.co/1ZYKJd8cq0 #Retail #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.9958864,40.4406248"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Pittsburgh", "Retail", "Job", "Jobs", "Hiring" }}, "user": { "id": 28697104, "name": "TMJ-PIT Retail Jobs", "screen_name": "tmj_pit_retail", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Pittsburgh, PA. Need help? Tweet us at @CareerArc!", "followers_count": 355, "friends_count": 310, "statues_count": 323 }, "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-04T10:00:02.000Z"), "id": 684071806274240512, "text": "@theillustrious_ thank you!", "in_reply_to_status": 684071565810634753, "in_reply_to_user": 3772428675, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3772428675 }}, "user": { "id": 634956176, "name": "Sucka Free‼️", "screen_name": "Jassmariexoxo", "lang": "en", "location": "stay out my dm's", "create_at": date("2012-07-13"), "description": "Don't come around me with that fake love.", "followers_count": 2079, "friends_count": 1349, "statues_count": 21054 }, "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-04T10:00:02.000Z"), "id": 684071806349623296, "text": "54.9F (Feels: 54.9F) - Humidity: 49% - Wind: 5.4mph NE - Gust: 8.3mph - Pressure: 1039.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": 220998 }, "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-04T10:00:02.000Z"), "id": 684071806462853120, "text": "@Avous_Yelmurc What ethnicity is behind it. People are talking about Muslims, Blacks, and Americans. Which Race is responsible?", "in_reply_to_status": 684071152982056960, "in_reply_to_user": 256568386, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 256568386 }}, "user": { "id": 1866163098, "name": "Official Nili Reyy", "screen_name": "NiliReyy", "lang": "en", "location": "San Antonio, Texas", "create_at": date("2013-09-14"), "description": "#MusicLife Music is my escape from this broken world. - I AM THE RIDICULOUS CHILD -", "followers_count": 390, "friends_count": 234, "statues_count": 1455 }, "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-01-04T10:00:02.000Z"), "id": 684071806500794368, "text": "#AskRR When is your first day of filming?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AskRR" }}, "user": { "id": 2909438360, "name": "Patricia Coleman", "screen_name": "patecoleman", "lang": "en", "location": "null", "create_at": date("2014-11-24"), "description": "null", "followers_count": 4, "friends_count": 58, "statues_count": 15 }, "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-04T10:00:02.000Z"), "id": 684071806718709762, "text": "Call me what you want but jersey shore is entertaining l o l", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 571953721, "name": "Arturo Reyes", "screen_name": "openmailboxx", "lang": "en", "location": "null", "create_at": date("2012-05-05"), "description": "0four0three199five | KΣ | SJSU", "followers_count": 618, "friends_count": 521, "statues_count": 8453 }, "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-04T10:00:02.000Z"), "id": 684071806882324480, "text": "@itsbriedarlingg yes!!", "in_reply_to_status": 683931292522524672, "in_reply_to_user": 146333452, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 146333452 }}, "user": { "id": 337329821, "name": "Sarah Elizabeth", "screen_name": "S_Mumps", "lang": "en", "location": "Californ-I-A☀", "create_at": date("2011-07-17"), "description": "♿️. ☕️. ♉️. MAS/FD", "followers_count": 547, "friends_count": 1021, "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.685512") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-01-04T10:00:02.000Z"), "id": 684071806895022080, "text": "#Support Origin Melissa! 36.3°F Wind:9.2mph 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": {{ "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": 306182 }, "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-04T10:00:02.000Z"), "id": 684071807075368961, "text": "@_EVANGELO dick labeau is the only geezer I want on this team", "in_reply_to_status": 684071508482895872, "in_reply_to_user": 95962476, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 95962476 }}, "user": { "id": 86122547, "name": "King Darius", "screen_name": "TRYING2GAF", "lang": "en", "location": "ask Daniel about the Den", "create_at": date("2009-10-29"), "description": "Either a good man with asshole tendencies or an asshole with good intentions...You decide", "followers_count": 1107, "friends_count": 992, "statues_count": 68871 }, "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-04T10:00:02.000Z"), "id": 684071807314472960, "text": "https://t.co/WurDfxf1D3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3317808374, "name": "Sebastian Wolff", "screen_name": "wolff247", "lang": "en", "location": "Miami, FL", "create_at": date("2015-08-17"), "description": "Christian, conservative, military historian, economist, and attorney. I've lost faith in the electorate, but as Christians we ALL have a duty to speak out.", "followers_count": 779, "friends_count": 827, "statues_count": 2841 }, "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-04T10:00:02.000Z"), "id": 684071807322816512, "text": "Find for today - #browserify\nModularizing my client-side! \\(^_^)/\n#nodejs #NoobsFindings", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "browserify", "nodejs", "NoobsFindings" }}, "user": { "id": 2756998818, "name": "Divya Konda", "screen_name": "div_konda", "lang": "en", "location": "Mountain View, CA", "create_at": date("2014-08-22"), "description": "null", "followers_count": 10, "friends_count": 41, "statues_count": 9 }, "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-04T10:00:02.000Z"), "id": 684071807398199296, "text": "Last semester 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": 2275908549, "name": "Miguel.♕╰☆╮", "screen_name": "MiguelC136", "lang": "en", "location": "null", "create_at": date("2014-01-11"), "description": "Hobbs NM Class of 2016 | Snapchat-Cuellar_22 |Instagram - miguel__cuellar", "followers_count": 165, "friends_count": 128, "statues_count": 2189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hobbs, NM", "id": "fc045378889ef7d6", "name": "Hobbs", "place_type": "city", "bounding_box": rectangle("-103.201769,32.680274 -103.093307,32.803199") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35025, "countyName": "Lea", "cityID": 3532520, "cityName": "Hobbs" } }
+{ "create_at": datetime("2016-01-04T10:00:03.000Z"), "id": 684071807620648960, "text": "https://t.co/yBbOSDZczH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2385536239, "name": "chris", "screen_name": "chrisfoley55", "lang": "en", "location": "Kent, OH", "create_at": date("2014-03-12"), "description": "suck brick, kid", "followers_count": 902, "friends_count": 939, "statues_count": 16170 }, "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-01-04T10:00:03.000Z"), "id": 684071807696146432, "text": "�������� https://t.co/bsagkJ315U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 956329225, "name": "doc.khalid", "screen_name": "dr_ak74", "lang": "en", "location": "Manhattan, NY", "create_at": date("2012-11-18"), "description": "الانسانيه تأتي اولا قبل الدين والمذهب والفصل واللون خلقنا من طين و سواسيه my #snap dr.kss #طبيب_أسنان #لا_للعنصريه_القذره #اكتب_لنفسي_فهي_لا_تستحق #حمبسلوح", "followers_count": 830, "friends_count": 827, "statues_count": 15918 }, "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-04T10:00:03.000Z"), "id": 684071807767474178, "text": "Hundo RT'in all type of opposites on tl ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2263125948, "name": "Sharpe", "screen_name": "_MarathonMike", "lang": "en", "location": "Temple University", "create_at": date("2013-12-26"), "description": "No nigga like me. Rip Reem", "followers_count": 479, "friends_count": 398, "statues_count": 5069 }, "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-04T10:00:03.000Z"), "id": 684071807805034496, "text": "Partly cloudy this afternoon, high 43 (6 C). Low 28 (-2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712333, "name": "LubbockWeather", "screen_name": "LubbockWeather", "lang": "en", "location": "Lubbock, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Lubbock, Texas", "followers_count": 1187, "friends_count": 93, "statues_count": 8041 }, "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-04T10:00:03.000Z"), "id": 684071807851343872, "text": "Not many things refresh my soul like being in the drizzly redwoods, but this warm cup of delight… https://t.co/iE3xncsuhj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.0266647,36.9763222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25771994, "name": "Malia Clark", "screen_name": "maliaclark", "lang": "en", "location": "San Jose, CA", "create_at": date("2009-03-21"), "description": "null", "followers_count": 179, "friends_count": 170, "statues_count": 3537 }, "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-01-04T10:00:03.000Z"), "id": 684071808002166784, "text": "It took me longer to get here this time because I took a longer and more grueling route. #cycling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-123.087315,48.07646"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "cycling" }}, "user": { "id": 916596266, "name": "竹下E本", "screen_name": "takenji_ebooks", "lang": "en", "location": "The Past", "create_at": date("2012-10-31"), "description": "Eh, close enough.", "followers_count": 147, "friends_count": 1, "statues_count": 99402 }, "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-04T10:00:03.000Z"), "id": 684071808128028672, "text": "people don't give Khloe Kardashian enough credit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 994241390, "name": "raquel", "screen_name": "raquelmmedina", "lang": "en", "location": "Denver, CO", "create_at": date("2012-12-06"), "description": "ig; @ raquelmmedina | sths | sc; @ raquel_medina", "followers_count": 248, "friends_count": 351, "statues_count": 699 }, "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-04T10:00:03.000Z"), "id": 684071808144809984, "text": "#Trenton, MI #Sales #Job: Event Specialist Part Time Sales at Advantage Solutions https://t.co/QaHZSPgx53 #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.1782587,42.1394858"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trenton", "Sales", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22292507, "name": "TMJ-MI-US SalesJobs", "screen_name": "tmj_mi_sales", "lang": "en", "location": "Michigan", "create_at": date("2009-02-28"), "description": "Follow this account for geo-targeted Sales job tweets in Michigan Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 401, "friends_count": 302, "statues_count": 365 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Trenton, MI", "id": "3c4cb62907491ed8", "name": "Trenton", "place_type": "city", "bounding_box": rectangle("-83.218644,42.111921 -83.165338,42.170748") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2680420, "cityName": "Trenton" } }
+{ "create_at": datetime("2016-01-04T10:00:03.000Z"), "id": 684071808207831040, "text": "Wind 0.7 mph W. Barometer 30.19 in, Rising slowly. Temperature 15.6 °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 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": 156418 }, "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-04T10:00:03.000Z"), "id": 684071808379650049, "text": "My next relationship I promise I'll be trained for that shit , cause now I'm thinking like a nigga so I get played like 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": 433377317, "name": "AlmightyJae", "screen_name": "xjazaline_", "lang": "en", "location": "null", "create_at": date("2011-12-10"), "description": "CLE | 19 BlackRican.", "followers_count": 1033, "friends_count": 366, "statues_count": 23360 }, "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-04T10:00:03.000Z"), "id": 684071808471977984, "text": "How many times will I hear the phrase \"hit the ground running\" today?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14939056, "name": "Kevin Vandever", "screen_name": "kevinvandever", "lang": "en", "location": "Huntington Beach / New Orleans", "create_at": date("2008-05-28"), "description": "I'm your worst nightmare! Well, actually, I'm not. I'm the guy whose picture my ex-girl friends' Moms would display well after the relationships ended.", "followers_count": 1122, "friends_count": 646, "statues_count": 12196 }, "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-04T10:00:03.000Z"), "id": 684071808509685760, "text": "https://t.co/C276aWixsQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1959047250, "name": "Iron Patriot Media", "screen_name": "Ipatriotmedia", "lang": "en", "location": "Quinlan, TX", "create_at": date("2013-10-13"), "description": "Publishing official conservative political news and commentary from Texas and around the country. We will be publishing a news website in the near future.", "followers_count": 769, "friends_count": 2286, "statues_count": 14022 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Quinlan, TX", "id": "ca0086eee6e5a7aa", "name": "Quinlan", "place_type": "city", "bounding_box": rectangle("-96.154053,32.888325 -96.097296,32.920033") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48231, "countyName": "Hunt", "cityID": 4860140, "cityName": "Quinlan" } }
+{ "create_at": datetime("2016-01-04T10:00:03.000Z"), "id": 684071808601960448, "text": "Cramps are killing 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": 935953549, "name": "DjLizLiz", "screen_name": "Lizzieissosexy", "lang": "en", "location": "null", "create_at": date("2012-11-08"), "description": "✌good gracious ass is bodacious. love yourself, girl, or nobody will -JCole/6 word poem", "followers_count": 256, "friends_count": 209, "statues_count": 3974 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minnetonka, MN", "id": "555209e7a34626a2", "name": "Minnetonka", "place_type": "city", "bounding_box": rectangle("-93.523339,44.890915 -93.398853,44.978956") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743252, "cityName": "Minnetonka" } }
+{ "create_at": datetime("2016-01-04T10:00:03.000Z"), "id": 684071809034010624, "text": "@RAWhuman__ happy birthday fam!! ⛽️��", "in_reply_to_status": -1, "in_reply_to_user": 2460543042, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2460543042 }}, "user": { "id": 394764366, "name": "Bellezza.", "screen_name": "ambalinaa_", "lang": "en", "location": "SLF", "create_at": date("2011-10-20"), "description": "12-1-12 ^_^ with my boo @Hi_Diyaaa #R.I.P Hadiya&Jayelo I love you.❤️ #equestrian,actress,bookworm,science geek.", "followers_count": 1136, "friends_count": 675, "statues_count": 41739 }, "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-04T10:00:03.000Z"), "id": 684071809247920129, "text": "Want to work in #Tampa, FL? View our latest opening: https://t.co/NaJJeGK1kC #Retail #HSN #HSNCareers #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.4651441,27.949436"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Tampa", "Retail", "HSN", "HSNCareers", "Job", "Jobs", "Hiring" }}, "user": { "id": 48348692, "name": "HSNCareers", "screen_name": "HSNCareers", "lang": "en", "location": "St. Petersburg, Florida", "create_at": date("2009-06-18"), "description": "Check out the GREAT CAREERS at HSN", "followers_count": 575, "friends_count": 77, "statues_count": 842 }, "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-04T10:00:03.000Z"), "id": 684071809348587521, "text": "Seriously anyone want to buy me @juanpalaciosx merch??", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2382401064 }}, "user": { "id": 1025480996, "name": "SxB✨Sydney", "screen_name": "imSydnxy", "lang": "en", "location": " San Antonio✨ Burbank HS", "create_at": date("2012-12-20"), "description": "16//concerts//Taylor Caniff is my sunshine♥//", "followers_count": 238, "friends_count": 198, "statues_count": 12353 }, "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-04T10:00:03.000Z"), "id": 684071809575157760, "text": "Portland to Punta Cana on Delta Air Lines for $391 #CheapFlights #PDX https://t.co/6T8rAB1nrA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.597504,45.5886"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CheapFlights", "PDX" }}, "user": { "id": 756129644, "name": "Flight Fishing", "screen_name": "FlightFishing", "lang": "en", "location": "Orlando, FL", "create_at": date("2012-08-13"), "description": "Collecting the internet's best airfares for flexible explorers. Sign up for free at http://www.flightfishing.com", "followers_count": 1136, "friends_count": 1505, "statues_count": 7988 }, "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-04T10:00:03.000Z"), "id": 684071809646465028, "text": "I make myself look stupid as fuck every fucking day. ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1114772023, "name": "Shaelyn blood♋", "screen_name": "otfblood", "lang": "en", "location": "Brockport, NY", "create_at": date("2013-01-23"), "description": "null", "followers_count": 378, "friends_count": 134, "statues_count": 20325 }, "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-01-04T10:00:03.000Z"), "id": 684071810200023040, "text": "Dude, this is my dude �� https://t.co/eUR36JA2rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 86060364, "name": "Nina Rykaart", "screen_name": "Blndeecurlznina", "lang": "en", "location": "California, USA", "create_at": date("2009-10-29"), "description": "null", "followers_count": 495, "friends_count": 236, "statues_count": 2131 }, "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-04T10:00:03.000Z"), "id": 684071810225188864, "text": "@TheTumblrPosts @hbisek my entire life", "in_reply_to_status": 684061074602213376, "in_reply_to_user": 798234950, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 798234950, 2736272643 }}, "user": { "id": 544505085, "name": "Catherine Fensom", "screen_name": "kitkatfensom", "lang": "en", "location": "Minnesota", "create_at": date("2012-04-03"), "description": "null", "followers_count": 139, "friends_count": 90, "statues_count": 2420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moorhead, MN", "id": "ddd9aed661af0c05", "name": "Moorhead", "place_type": "city", "bounding_box": rectangle("-96.79398,46.818321 -96.705732,46.917616") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27027, "countyName": "Clay", "cityID": 2743864, "cityName": "Moorhead" } }
+{ "create_at": datetime("2016-01-04T10:00:03.000Z"), "id": 684071810263060480, "text": "#Cristiano \n#Cristiano \n#Cristiano \n#Cristiano \n#Cristiano \n#Cristiano \n#Cristiano \n#Cristiano \n#Cristiano \n#Cristiano \n#Cristiano", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Cristiano", "Cristiano", "Cristiano", "Cristiano", "Cristiano", "Cristiano", "Cristiano", "Cristiano", "Cristiano", "Cristiano", "Cristiano" }}, "user": { "id": 178894712, "name": "LETTY_PINEDA", "screen_name": "blancalettyp", "lang": "en", "location": "Miami, FL", "create_at": date("2010-08-15"), "description": "El futbol es mi pasión, Merengue y Americanista de ....❤ Twits 90% de futbol.⚽ #RazaAdicta #SiempreEsperandoElPróximoPartido ⚽️", "followers_count": 1817, "friends_count": 399, "statues_count": 37249 }, "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-04T10:00:03.000Z"), "id": 684071810372100097, "text": "#Repost @88mphpod\n・・・\n#nerds #ilovethe80s #irantradio #movies #80sbaby #80shorror #horrorfan… https://t.co/t9QdAuVSI8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.003029,33.948117"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Repost", "nerds", "ilovethe80s", "irantradio", "movies", "80sbaby", "80shorror", "horrorfan" }}, "user_mentions": {{ 2282758640 }}, "user": { "id": 744647971, "name": "Rant Radio", "screen_name": "iRantRadio", "lang": "en", "location": "Los Angeles & around the WORLD", "create_at": date("2012-08-08"), "description": "iRant Radio is about you, radio your way! You just never know what's going to happen Rant@RantRadioNetwork.com Call/Text us 626.869.RANT or 855-969-RANT", "followers_count": 37421, "friends_count": 32373, "statues_count": 15378 }, "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-04T10:00:03.000Z"), "id": 684071810522988544, "text": "I asked my teacher if I could run to the restroom and he told me to \"just walk\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1000378885, "name": "Kristen Hazel", "screen_name": "arnoldkristen_", "lang": "en", "location": "Oklahoma(:", "create_at": date("2012-12-09"), "description": "Screws Fall Out All The Time, The World Is An Imperfect Place...#rampride Est.1997", "followers_count": 147, "friends_count": 167, "statues_count": 1026 }, "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-01-04T10:00:03.000Z"), "id": 684071810665590784, "text": "13:00 34.7°F Feels:29.8°F (Hi35.2°F/Lo26.8°F) Hum:53% Wnd:N 5.5MPH Baro:30.30in. 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": 158, "friends_count": 261, "statues_count": 24287 }, "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-04T10:00:03.000Z"), "id": 684071810678181888, "text": "Morning clouds/afternoon sun this afternoon, high 51 (11 C). Low 36 (2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712562, "name": "ElPasoWX", "screen_name": "ElPasoWX", "lang": "en", "location": "El Paso, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for El Paso, Texas", "followers_count": 526, "friends_count": 93, "statues_count": 8099 }, "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-04T10:00:03.000Z"), "id": 684071810732703744, "text": "Egypt Pulls Plug on https://t.co/sVwXs4TJli Free Basics https://t.co/ltQs7pvBN2", "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": 17699, "friends_count": 17576, "statues_count": 66582 }, "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-04T10:00:03.000Z"), "id": 684071810862821376, "text": "@omfgkeema https://t.co/S9f1tvqU0G", "in_reply_to_status": -1, "in_reply_to_user": 1170188048, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1170188048 }}, "user": { "id": 2657701926, "name": "TAELEFLARE", "screen_name": "TAELEFLARE", "lang": "en", "location": "Atlanta, GA", "create_at": date("2014-07-18"), "description": "Creative director I am the vision ++", "followers_count": 497, "friends_count": 296, "statues_count": 9475 }, "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-01-04T10:00:03.000Z"), "id": 684071811219230720, "text": "oh fuck no. salads more expensive anyway. imma continue being happy and unhealthy with they chicken sandwiches", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 102540879, "name": "dreadhead papi", "screen_name": "melvinsucks", "lang": "en", "location": "Howard University", "create_at": date("2010-01-06"), "description": "vibes n stuff", "followers_count": 1495, "friends_count": 971, "statues_count": 94812 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inniswold, LA", "id": "fbcefe96d630085b", "name": "Inniswold", "place_type": "city", "bounding_box": rectangle("-91.092831,30.363623 -91.040089,30.412138") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2237270, "cityName": "Inniswold" } }
+{ "create_at": datetime("2016-01-04T10:00:03.000Z"), "id": 684071811366141957, "text": "This wolfchase Chick-Fil-A packed !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 317393506, "name": "✌ Ty ✌️", "screen_name": "tybeasty", "lang": "en", "location": "Nashville✈️Knoxville ", "create_at": date("2011-06-14"), "description": "18 | HumeFogg Alumnus | Political Science Major w/ Psychology Minor #UTK19✌️", "followers_count": 666, "friends_count": 1157, "statues_count": 12941 }, "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-04T10:00:03.000Z"), "id": 684071811412185088, "text": "Can you recommend anyone for this #job? RN II/ ICU/ 7p-7a $5,000 Sign on bonus - https://t.co/NhMsedJtSX #Baytown, TX #Nursing #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.978977,29.769031"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Baytown", "Nursing", "Hiring" }}, "user": { "id": 22873671, "name": "TMJ-TX Nursing Jobs", "screen_name": "tmj_tx_nursing", "lang": "en", "location": "Texas", "create_at": date("2009-03-04"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Texas Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 485, "friends_count": 300, "statues_count": 1600 }, "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-01-04T10:00:04.000Z"), "id": 684071811827535879, "text": "One-minute hike: Finger Lakes Trail in Letchworth https://t.co/5uNGZs7R1e #ROC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ROC" }}, "user": { "id": 17022537, "name": "Rochester D and C", "screen_name": "DandC", "lang": "en", "location": "Rochester, NY", "create_at": date("2008-10-28"), "description": "Download the D&C app for everything you need to #ROCtheWINTER. iPhone: http://apple.co/1uuMuqe Android: http://bit.ly/1tzAtw7 iPad: http://bit.ly/1tzAtw7", "followers_count": 51846, "friends_count": 4260, "statues_count": 84898 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rochester, NY", "id": "2f1fc0d72969452b", "name": "Rochester", "place_type": "city", "bounding_box": rectangle("-92.558347,43.099801 -77.533421,44.108056") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3663000, "cityName": "Rochester" } }
+{ "create_at": datetime("2016-01-04T10:00:04.000Z"), "id": 684071812276199425, "text": "I've decided Remi is never going to school and he is going to stay a baby forever https://t.co/x2X1CdKGag", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3002129334, "name": "Casey Iverson", "screen_name": "caseyiverson909", "lang": "en", "location": "null", "create_at": date("2015-01-29"), "description": "Remi & Ed", "followers_count": 8328, "friends_count": 309, "statues_count": 3808 }, "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.069786,31.639828") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4876000, "cityName": "Waco" } }
+{ "create_at": datetime("2016-01-04T10:00:04.000Z"), "id": 684071812448178176, "text": "Relationships are 50/50, but if one is slacking you should be about to carry them until they get right.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1378494782, "name": "T⚡️", "screen_name": "TMFnClay_", "lang": "en", "location": "Dallas, Flexa$", "create_at": date("2013-04-24"), "description": "23| Scorpio| #CowboysNation| 1738 | #UNTForHoweverLong| Snapchat: tdot_14", "followers_count": 1670, "friends_count": 999, "statues_count": 124572 }, "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-04T10:00:04.000Z"), "id": 684071812469264385, "text": "wait but there are snow flurries falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 263878810, "name": "emma watson", "screen_name": "_emroseeee", "lang": "en", "location": "MereCo", "create_at": date("2011-03-10"), "description": "null", "followers_count": 1148, "friends_count": 318, "statues_count": 14940 }, "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-04T10:00:04.000Z"), "id": 684071812594974720, "text": "@thegrapevineres The Grapevine Restaurant and Catering #Buffalo https://t.co/U71qb3fsCH Excellent https://t.co/91wdeh8tfE", "in_reply_to_status": -1, "in_reply_to_user": 449182761, "favorite_count": 0, "coordinate": point("-78.8214798,43.0238113"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Buffalo" }}, "user_mentions": {{ 449182761 }}, "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": 161851 }, "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-04T10:00:04.000Z"), "id": 684071812653801472, "text": "Wind 1.7 mph ENE. Barometer 30.288 in, Rising. Temperature 50.9 °F. Rain today 0.00 in. Humidity 54%", "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": 55850 }, "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-04T10:00:04.000Z"), "id": 684071812842565633, "text": "same niggas fronted now they asking you to ride ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 240095548, "name": "MsSimone♉️", "screen_name": "YouAre_ASIN", "lang": "en", "location": "UPTOWN Somerville . ", "create_at": date("2011-01-18"), "description": "DESTROY WHAT DESTROYS YU!", "followers_count": 969, "friends_count": 339, "statues_count": 45291 }, "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-04T10:00:04.000Z"), "id": 684071813110870016, "text": "they look like odd future socks.. boi https://t.co/0tXOkZaukS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 241365522, "name": "britt", "screen_name": "staaysteezy", "lang": "en", "location": "HTX", "create_at": date("2011-01-21"), "description": "u r b u r n t", "followers_count": 1177, "friends_count": 142, "statues_count": 29838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-04T10:00:04.000Z"), "id": 684071813366726656, "text": "Sunny this afternoon, high 48 (9 C). Low 31 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712512, "name": "AbileneWeather", "screen_name": "AbileneWeather", "lang": "en", "location": "Abilene, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Abilene, Texas", "followers_count": 928, "friends_count": 93, "statues_count": 8139 }, "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-04T10:00:04.000Z"), "id": 684071813446385664, "text": "Someone go to the gym 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": 366789893, "name": "Red Queen♕", "screen_name": "frecklesthekid", "lang": "en", "location": "Dirty T", "create_at": date("2011-09-02"), "description": "he liked it so he put a ring on it", "followers_count": 436, "friends_count": 294, "statues_count": 13962 }, "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-04T10:00:04.000Z"), "id": 684071813513637888, "text": "I wish Kurt was home ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287926839, "name": "Alessandra Widmann", "screen_name": "A_Widmann17", "lang": "en", "location": "null", "create_at": date("2011-04-25"), "description": "Life is short but sweet for certain ☀️ URI '16", "followers_count": 380, "friends_count": 296, "statues_count": 24156 }, "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-04T10:00:04.000Z"), "id": 684071813647732736, "text": "Tune in live to #ATTDevSummit tomorrow at 9am PT for the latest #CES2016 news from @ATT: https://t.co/gpqauO9U8d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ATTDevSummit", "CES2016" }}, "user_mentions": {{ 14620190 }}, "user": { "id": 119997243, "name": "Emily Edmonds", "screen_name": "emilyjedmonds", "lang": "en", "location": "Atlanta, Georgia", "create_at": date("2010-03-04"), "description": "Director of Global Media Relations at AT&T. Atlanta via New York. PR guru for 13 years. Previously at the WSJ & Time Inc. Auburn football fan. Avid traveler.", "followers_count": 509, "friends_count": 716, "statues_count": 500 }, "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-04T10:00:04.000Z"), "id": 684071813840809984, "text": "Wow I'm a huge @Braves but please stop screwing the team up and trading our best players like @kimbrel46 https://t.co/m1zRGJcQUU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21436663, 1033178532 }}, "user": { "id": 2157902531, "name": "Sonny & Dallas", "screen_name": "doggy8923", "lang": "en", "location": "null", "create_at": date("2013-10-28"), "description": "Sonny and Dallas morrell we have a 4 year old son and YouTube sub to us look up.sonny and Dallas morrell", "followers_count": 378, "friends_count": 808, "statues_count": 5284 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Church Hill, TN", "id": "06c31f44d1d9dbc8", "name": "Church Hill", "place_type": "city", "bounding_box": rectangle("-82.82035,36.492621 -82.673575,36.554308") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47073, "countyName": "Hawkins", "cityID": 4714980, "cityName": "Church Hill" } }
+{ "create_at": datetime("2016-01-04T10:00:04.000Z"), "id": 684071814432198656, "text": "Travelling to Oklahoma City or just twittering about Oklahoma City? https://t.co/FSM7nn90Kw #Oklahoma City", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.5164,35.4676"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Oklahoma" }}, "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": 2511700 }, "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-04T10:00:04.000Z"), "id": 684071814448861184, "text": "Really just ready for Friday to get here ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 449286095, "name": "Little Hammer", "screen_name": "HammerKaden", "lang": "en", "location": "null", "create_at": date("2011-12-28"), "description": "null", "followers_count": 355, "friends_count": 416, "statues_count": 7226 }, "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-04T10:00:04.000Z"), "id": 684071814448984064, "text": "all these hittas, but my eyes on u����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 808011854, "name": "HER..", "screen_name": "ivoryzhanae", "lang": "en", "location": "null", "create_at": date("2012-09-06"), "description": "remain humble, but let these bitches know!", "followers_count": 1666, "friends_count": 922, "statues_count": 40363 }, "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-04T10:00:04.000Z"), "id": 684071814469791744, "text": "1.74 magnitude #earthquake. 9 km from Volcano, HI, #UnitedStates https://t.co/2UwhubcxL9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-155.321167,19.4335003"), "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": 60177, "friends_count": 10, "statues_count": 92392 }, "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", "cityID": 1572350, "cityName": "Volcano" } }
+{ "create_at": datetime("2016-01-04T10:00:04.000Z"), "id": 684071814688030720, "text": "I'm just ganna enjoy whatever's left before it vanishes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 558092626, "name": "ThaniaaGarciaa", "screen_name": "_WhaYouCallThat", "lang": "en", "location": "null", "create_at": date("2012-04-19"), "description": "| NJ | Exit9", "followers_count": 378, "friends_count": 418, "statues_count": 12621 }, "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-04T10:00:04.000Z"), "id": 684071814713085953, "text": "@clark_emily13 spaghetti squash with a vinegarette oil with grill chicken over rice", "in_reply_to_status": 684071365591347201, "in_reply_to_user": 2312830561, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2312830561 }}, "user": { "id": 2976706605, "name": "RaeAnn Feltis | EZ", "screen_name": "raeann_feltis", "lang": "en", "location": "null", "create_at": date("2015-01-13"), "description": "I am a Christ loving teen from Fort Worth Texas on a mission to stop bullying!!! Jeremiah 29:11 auditioning to be on #thevoice2016", "followers_count": 235, "friends_count": 986, "statues_count": 1763 }, "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-04T10:00:04.000Z"), "id": 684071814796976129, "text": "I pray this is real https://t.co/Z5ld0Fk2O4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 555715512, "name": "Ellie Seiter", "screen_name": "EllieSeiter", "lang": "en", "location": "null", "create_at": date("2012-04-16"), "description": "I want my everyday to make God belly laugh, glad that he gave life to someone who loves the gift.", "followers_count": 444, "friends_count": 383, "statues_count": 3646 }, "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-04T10:00:04.000Z"), "id": 684071814805504001, "text": "My poppa bear is my #mcm because I'll always be a daddy's girl. @ Glenwood Springs, Colorado https://t.co/gf4lxxl2MC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-107.329,39.5396"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "mcm" }}, "user": { "id": 535850614, "name": "Alexa Eleanor", "screen_name": "ClearlyCrazy", "lang": "en", "location": "CO born, TX raised, KY living", "create_at": date("2012-03-24"), "description": "Just a girl wanting to make it to the big leagues. University of Kentucky. College of Nursing. Matthew 5:14-16", "followers_count": 470, "friends_count": 459, "statues_count": 11413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glenwood Springs, CO", "id": "7cbee617cc64f301", "name": "Glenwood Springs", "place_type": "city", "bounding_box": rectangle("-107.374027,39.494044 -107.301755,39.569159") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8045, "countyName": "Garfield", "cityID": 830780, "cityName": "Glenwood Springs" } }
+{ "create_at": datetime("2016-01-04T10:00:04.000Z"), "id": 684071815044534273, "text": "Going to @gomocsWBK game tonight? Bring in your ticket before or after the game and get $1 off any drink. #eatFIVE https://t.co/igv8P4uBWd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "eatFIVE" }}, "user_mentions": {{ 59554532 }}, "user": { "id": 3241699435, "name": "FIVE Chattanooga", "screen_name": "FIVEchattanooga", "lang": "en", "location": "null", "create_at": date("2015-06-10"), "description": "null", "followers_count": 34, "friends_count": 24, "statues_count": 138 }, "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-04T10:00:04.000Z"), "id": 684071815195529217, "text": "@DaCarolinaKidd @On1yMeee @TiricoHairline LMFAOOO", "in_reply_to_status": 684071575369338880, "in_reply_to_user": 174540621, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 174540621, 3346131755, 64322070 }}, "user": { "id": 211418311, "name": "‎‎ ­ ­ ­ ­ ­ ­ ­ ­ ­", "screen_name": "Rebel2Amerikkka", "lang": "en", "location": "null", "create_at": date("2010-11-02"), "description": "I only love her if she was born on the 24th of June", "followers_count": 1696, "friends_count": 985, "statues_count": 81192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sugaw Creek-Ritch Ave, Charlotte", "id": "36d3c7b4b6123905", "name": "Sugaw Creek-Ritch Ave", "place_type": "neighborhood", "bounding_box": rectangle("-80.819837,35.247246 -80.791086,35.278936") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-01-04T10:00:04.000Z"), "id": 684071815216406528, "text": "#CareerArc #Labor #Job alert: Truck Driver – CDL Class A or B | CVS Health | #KAPOLEI, HI https://t.co/1DIrG17wfX #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-158.056892,21.3354034"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CareerArc", "Labor", "Job", "KAPOLEI", "Jobs", "Hiring" }}, "user": { "id": 118676555, "name": "TMJ-HI Labor Jobs", "screen_name": "tmj_HI_LABOR", "lang": "en", "location": "Hawaii", "create_at": date("2010-03-01"), "description": "Follow this account for geo-targeted General Labor job tweets in Hawaii Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 283, "friends_count": 279, "statues_count": 4 }, "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-04T10:00:04.000Z"), "id": 684071815292030976, "text": "\"Hey lets get pancakes!\" *can't even finish them* https://t.co/SrCflQukGN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 384482876, "name": "Riley Campbell", "screen_name": "rileycampbell3", "lang": "en", "location": "Athens, OH", "create_at": date("2011-10-03"), "description": "you can pay to go to school, but you can't buy class", "followers_count": 490, "friends_count": 276, "statues_count": 23412 }, "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-01-04T10:00:04.000Z"), "id": 684071815405158400, "text": "The feels are real", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2241394922, "name": "Jacesus Christ", "screen_name": "jacemoonmoon", "lang": "en", "location": "null", "create_at": date("2013-12-11"), "description": "I like to think of Jesus as a mischievous badger", "followers_count": 381, "friends_count": 459, "statues_count": 5292 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Draper, UT", "id": "4c3c7782398bd438", "name": "Draper", "place_type": "city", "bounding_box": rectangle("-111.921988,40.45236 -111.823749,40.544422") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4920120, "cityName": "Draper" } }
+{ "create_at": datetime("2016-01-04T10:00:04.000Z"), "id": 684071815577092096, "text": "Lowe's: Receiver/Stocker (#Honolulu, HI) https://t.co/HG029xsxt3 #Labor #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Honolulu", "Labor", "Job", "Jobs", "Hiring" }}, "user": { "id": 22777294, "name": "TMJ-HON Labor Jobs", "screen_name": "tmj_hon_labor", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-04"), "description": "Follow this account for geo-targeted General Labor job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 298, "friends_count": 291, "statues_count": 2 }, "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-04T10:00:04.000Z"), "id": 684071815732314112, "text": "That's what you get for buying a Kia https://t.co/chZYvtNfMw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 114358448, "name": "Tanner Cremer", "screen_name": "Tcreem", "lang": "en", "location": "Keizer, OR", "create_at": date("2010-02-14"), "description": "Late to my own funeral.", "followers_count": 574, "friends_count": 668, "statues_count": 14352 }, "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-01-04T10:00:05.000Z"), "id": 684071816139304962, "text": "@AmandaaMariee22 who me?", "in_reply_to_status": 684060999687749632, "in_reply_to_user": 4441325255, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4441325255 }}, "user": { "id": 290184306, "name": "Smiley", "screen_name": "LoopstarSmiley", "lang": "en", "location": "585 / 716 / 704/ 619", "create_at": date("2011-04-29"), "description": ".... #NoMoreSmilez Mixtape coming soon 2016!!!", "followers_count": 3126, "friends_count": 1489, "statues_count": 104916 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lockport, NY", "id": "45009687ba062971", "name": "Lockport", "place_type": "city", "bounding_box": rectangle("-78.744599,43.138056 -78.618744,43.205149") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36063, "countyName": "Niagara", "cityID": 3643082, "cityName": "Lockport" } }
+{ "create_at": datetime("2016-01-04T10:00:05.000Z"), "id": 684071816168632321, "text": "\"If you obey all the rules, you miss all the fun.\" -Katharine Hepburn.\"#quotes#motivation #inspiration #motivationapp via @MotivationApp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "inspiration", "motivationapp" }}, "user_mentions": {{ 1671400987 }}, "user": { "id": 1014336336, "name": "JOHN BOWEN", "screen_name": "johnbowen531", "lang": "en", "location": "NORTH EAST. PA", "create_at": date("2012-12-15"), "description": "null", "followers_count": 9, "friends_count": 51, "statues_count": 9775 }, "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-01-04T10:00:05.000Z"), "id": 684071816193814529, "text": "@BananaLove82 I became addicted to the weights ��", "in_reply_to_status": 684071511175528448, "in_reply_to_user": 525573430, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 525573430 }}, "user": { "id": 105360282, "name": "☀️ Bre ☀️", "screen_name": "Bre_Ruble", "lang": "en", "location": "Visalia, California\n", "create_at": date("2010-01-15"), "description": "God. Fitness. Volleyball Player. Nanny. Volleyball Coach. Runner. Hunter. Gun Lover. College Student. Sports Lover. Cali Grown. Cali Livin. #TeamSwoleU", "followers_count": 923, "friends_count": 373, "statues_count": 11118 }, "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-04T10:00:05.000Z"), "id": 684071816214802433, "text": "Former M&T VP sentenced to prison https://t.co/XsL79fK89x #ROC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ROC" }}, "user": { "id": 17022537, "name": "Rochester D and C", "screen_name": "DandC", "lang": "en", "location": "Rochester, NY", "create_at": date("2008-10-28"), "description": "Download the D&C app for everything you need to #ROCtheWINTER. iPhone: http://apple.co/1uuMuqe Android: http://bit.ly/1tzAtw7 iPad: http://bit.ly/1tzAtw7", "followers_count": 51846, "friends_count": 4260, "statues_count": 84899 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rochester, NY", "id": "2f1fc0d72969452b", "name": "Rochester", "place_type": "city", "bounding_box": rectangle("-92.558347,43.099801 -77.533421,44.108056") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3663000, "cityName": "Rochester" } }
+{ "create_at": datetime("2016-01-04T10:00:05.000Z"), "id": 684071816588079104, "text": "Me waking Brian up: \"good morning baby, I found that song you asked me to find\"\nHim: *picks up his phone calls someone ignores 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": 464656712, "name": "Jules", "screen_name": "bakerbuddy1", "lang": "en", "location": "null", "create_at": date("2012-01-15"), "description": "You can't sit with us, you can't hit the blunt.", "followers_count": 463, "friends_count": 203, "statues_count": 15529 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rochester, IN", "id": "cf7036581a451292", "name": "Rochester", "place_type": "city", "bounding_box": rectangle("-86.24189,41.038581 -86.159562,41.080211") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18049, "countyName": "Fulton", "cityID": 1865214, "cityName": "Rochester" } }
+{ "create_at": datetime("2016-01-04T10:00:05.000Z"), "id": 684071816793436161, "text": "Cloud Peak Energy: Electrician (#Gillette, WY) https://t.co/uJFDvFH9JJ #SkilledTrade #electrician #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.5022205,44.2910915"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Gillette", "SkilledTrade", "electrician", "Job", "Jobs", "Hiring" }}, "user": { "id": 3011572896, "name": "Cloud Peak Jobs", "screen_name": "CloudPeakJobs", "lang": "en", "location": "null", "create_at": date("2015-02-06"), "description": "Cloud Peak Energy aims to be a leading energy producer operating in a safe, responsible and caring manner. Check here for current job openings.", "followers_count": 56, "friends_count": 270, "statues_count": 15 }, "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-01-04T10:00:05.000Z"), "id": 684071816873119744, "text": "We're #hiring! Click to apply: Coordinator, Education & Communication - Sex Abuse Treatment - https://t.co/QSvx3qHnnU #Job #Honolulu, HI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8480364,21.3136151"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Job", "Honolulu" }}, "user": { "id": 23493115, "name": "TMJ-HON Advert Jobs", "screen_name": "tmj_hon_adv", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Marketing/Ad/PR job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 290, "friends_count": 193, "statues_count": 9 }, "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-04T10:00:05.000Z"), "id": 684071817049276417, "text": "Mostly sunny this afternoon, high 47 (8 C). Low 34 (1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712530, "name": "MidlandWeather", "screen_name": "MidlandWeather", "lang": "en", "location": "Midland/Odessa, TX", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Midland/Odessa, TX", "followers_count": 880, "friends_count": 93, "statues_count": 8123 }, "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-01-04T10:00:05.000Z"), "id": 684071817070288897, "text": "@GreekGeekshere # Danny Sambora # what do you guys think about bonjovi with out they're founding guitarist richie sambora.", "in_reply_to_status": -1, "in_reply_to_user": 52102296, "favorite_count": 0, "coordinate": point("-118.1323866,34.1754586"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 52102296 }}, "user": { "id": 2905883317, "name": "Danny Sambora", "screen_name": "dansambora75", "lang": "en", "location": "null", "create_at": date("2014-11-21"), "description": "null", "followers_count": 11, "friends_count": 141, "statues_count": 55 }, "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-04T10:00:05.000Z"), "id": 684071817217216512, "text": "@Mochus12 shhhhh (location)", "in_reply_to_status": 684071268665053185, "in_reply_to_user": 232630835, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 232630835 }}, "user": { "id": 1355419184, "name": "sdizzle", "screen_name": "sdendler", "lang": "en", "location": "null", "create_at": date("2013-04-15"), "description": "sometimes i cant do the thing | NJ", "followers_count": 520, "friends_count": 360, "statues_count": 7456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hamilton Square, NJ", "id": "002044f173d275fd", "name": "Hamilton Square", "place_type": "city", "bounding_box": rectangle("-74.67339,40.198798 -74.628097,40.250928") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34021, "countyName": "Mercer", "cityID": 3429370, "cityName": "Hamilton Square" } }
+{ "create_at": datetime("2016-01-04T10:00:05.000Z"), "id": 684071817393262592, "text": "Dedesi devşirme ama kendini Cengiz'in torunu sanıyor\nDedesi reaya idi ama kendini Fatih'in torunu sanıyor\nFiravunu öven Yahudiler bunlar :)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "user": { "id": 238776270, "name": "Edip Yüksel", "screen_name": "edipyuksel", "lang": "en", "location": "US 619 894 6346", "create_at": date("2011-01-15"), "description": "La ilahe illa Allah. Nokta.", "followers_count": 41300, "friends_count": 160, "statues_count": 26574 }, "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-04T10:00:05.000Z"), "id": 684071817410117632, "text": "If there's something I've learned from a million mistakes you're the one that I want at the end 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": 525742153, "name": "janay.", "screen_name": "janay_traylor", "lang": "en", "location": "grand rapids, mi", "create_at": date("2012-03-15"), "description": "aquinas college", "followers_count": 958, "friends_count": 281, "statues_count": 37304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wyoming, MI", "id": "77b8f2664985738a", "name": "Wyoming", "place_type": "city", "bounding_box": rectangle("-85.782295,42.85341 -85.635295,42.942348") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26081, "countyName": "Kent", "cityID": 2688940, "cityName": "Wyoming" } }
+{ "create_at": datetime("2016-01-04T10:00:05.000Z"), "id": 684071817426907136, "text": "Felt like this weekend was never ending bc NY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 86527354, "name": "G$", "screen_name": "BeGlorifieddd", "lang": "en", "location": "Chi", "create_at": date("2009-10-31"), "description": "Listen to the kids, bro.", "followers_count": 308, "friends_count": 193, "statues_count": 8347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bensenville, IL", "id": "eaaf978cf85d1347", "name": "Bensenville", "place_type": "city", "bounding_box": rectangle("-87.968865,41.930763 -87.914471,41.993423") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1705248, "cityName": "Bensenville" } }
+{ "create_at": datetime("2016-01-04T10:00:05.000Z"), "id": 684071817451974657, "text": "Wind 2.0 mph S. Barometer 29.792 in, Rising. Temperature 48.1 °F. Rain today 0.00 in. Humidity 46%", "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": 191, "friends_count": 58, "statues_count": 239987 }, "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-04T10:00:05.000Z"), "id": 684071817850556416, "text": "I ain't like him. ��", "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": ". February 12th❤️", "screen_name": "___Shannonnn", "lang": "en", "location": "662 ✈️ 601 Hattiesburg, MS ", "create_at": date("2011-03-10"), "description": "#USM18 .", "followers_count": 1377, "friends_count": 841, "statues_count": 47200 }, "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-01-04T10:00:05.000Z"), "id": 684071818093703168, "text": "#Manufacturing #Job alert: Aluminum Welder | Aerotek | #Honolulu, HI https://t.co/l6rMf1Gw2Z #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "nl", "is_retweet": false, "hashtags": {{ "Manufacturing", "Job", "Honolulu", "Jobs", "Hiring" }}, "user": { "id": 23527770, "name": "TMJ - HON Manuf Jobs", "screen_name": "tmj_hon_manuf", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Manufacturing job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 224, "friends_count": 202, "statues_count": 1 }, "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-04T10:00:05.000Z"), "id": 684071818106408960, "text": "@emprxll you're bae", "in_reply_to_status": 684065126287314944, "in_reply_to_user": 3953082794, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3953082794 }}, "user": { "id": 2581072189, "name": "Emily Hoskins", "screen_name": "EmilyHoskins14", "lang": "en", "location": "Rusvegas or Sareptasaurus ", "create_at": date("2014-06-21"), "description": "LaTech18 (maybe) Either working, at a local coffee shop, church, gym, sleeping, or sleeping in class.", "followers_count": 476, "friends_count": 303, "statues_count": 5959 }, "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-04T10:00:05.000Z"), "id": 684071818370650112, "text": "@datboywolf As though we raised ourselves.", "in_reply_to_status": 684071503554449408, "in_reply_to_user": 20876069, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20876069 }}, "user": { "id": 14344402, "name": "Megan Romer ⚜", "screen_name": "meganromer", "lang": "en", "location": "Lafayette, LA", "create_at": date("2008-04-09"), "description": "Slinging words for @aboutdotcom (and elsewhere), tunes for @iheartradio (and elsewhere), & otherwise dishing out opinions, solicited & otherwise. Malcolm's Mom.", "followers_count": 4064, "friends_count": 1841, "statues_count": 62253 }, "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-04T10:00:05.000Z"), "id": 684071818710274048, "text": "I want to die. https://t.co/4Qc398aD8f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 419118033, "name": "Madi Murray ♡", "screen_name": "MadiLeeMurray", "lang": "en", "location": "null", "create_at": date("2011-11-22"), "description": "Licensed CNA. CC//WFW.", "followers_count": 692, "friends_count": 593, "statues_count": 14613 }, "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.923837,46.764122") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53041, "countyName": "Lewis", "cityID": 5311160, "cityName": "Centralia" } }
+{ "create_at": datetime("2016-01-04T10:00:05.000Z"), "id": 684071819163222016, "text": "IM SO TIRED OF GETTING CURVEBALLS THROWN AT ME RIGHT AFTER I GET GOOD NEWS. JFC.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 247007508, "name": "CFK.", "screen_name": "colton_kiser19", "lang": "en", "location": "Lubbock/Seattle/Melbourne ", "create_at": date("2011-02-03"), "description": "One fine day, gonna leave you all behind. SSFC, MUFC, StL Blues, MVFC, Hawthorn Footy Club, Seahawks. Founder of @BluesRantDotCom", "followers_count": 535, "friends_count": 1311, "statues_count": 16632 }, "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-04T10:00:05.000Z"), "id": 684071819431653377, "text": "Can't keep my hands 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": 250723155, "name": "£mily", "screen_name": "EmmilyWamboldt", "lang": "en", "location": "Wheat Ridge, CO", "create_at": date("2011-02-11"), "description": "id rather wear flowers in my hair than dimonds around my neck", "followers_count": 486, "friends_count": 188, "statues_count": 20377 }, "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-01-04T10:00:05.000Z"), "id": 684071819817558016, "text": "Join the Starbucks team! See our latest #Hospitality #job opening here: https://t.co/7nmE89iGXW #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-158.1860241,21.4393347"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 23897767, "name": "TMJ-HON Hosp. Jobs", "screen_name": "tmj_hon_hrta", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-11"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 412, "friends_count": 292, "statues_count": 264 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waianae, HI", "id": "06f2757ba285a3ef", "name": "Waianae", "place_type": "city", "bounding_box": rectangle("-158.200392,21.428803 -158.145479,21.478996") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1574450, "cityName": "Waianae" } }
+{ "create_at": datetime("2016-01-04T10:00:05.000Z"), "id": 684071819905773572, "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": 976, "friends_count": 312, "statues_count": 2511702 }, "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-04T10:00:05.000Z"), "id": 684071820010631169, "text": "Adoresmiracle https://t.co/QQ3UtxhMp9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 3249902922, "name": "Li' Shawt ❣", "screen_name": "miraclelashayy", "lang": "en", "location": "@financer_ ", "create_at": date("2015-06-19"), "description": "SC: adoresmiracle || fineeassk ❤️ @prettyasanna long_liveexx", "followers_count": 667, "friends_count": 473, "statues_count": 26706 }, "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-04T10:00:06.000Z"), "id": 684071820341940225, "text": "@savanaburroughs aw whatta babe", "in_reply_to_status": 684069553672957952, "in_reply_to_user": 2760790275, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2760790275 }}, "user": { "id": 2927342955, "name": "Caro", "screen_name": "curalern", "lang": "en", "location": "South Patrick Shores, FL", "create_at": date("2014-12-17"), "description": "groovin//satellite varsity volleyball", "followers_count": 634, "friends_count": 463, "statues_count": 3368 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Satellite Beach, FL", "id": "530c7d91d01f97da", "name": "Satellite Beach", "place_type": "city", "bounding_box": rectangle("-80.610732,28.157527 -80.584795,28.194225") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12009, "countyName": "Brevard", "cityID": 1264400, "cityName": "Satellite Beach" } }
+{ "create_at": datetime("2016-01-04T10:00:06.000Z"), "id": 684071820362944512, "text": "Don't fool yourself. There is only 1 candidate that is ONLY doing this because he cares. @RealBenCarson Every1 else has additional motives.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1180379185 }}, "user": { "id": 2439134820, "name": "Daniel Leitner", "screen_name": "DanielLeitner2", "lang": "en", "location": "Louisville KY", "create_at": date("2014-04-11"), "description": "Avid Football Analyst. Eagles/Ducks Fan. #Christ Follower w/a 166 I.Q... Proof one can be intelligent, & still believe in a Supreme Ruler of the Universe. :)", "followers_count": 24, "friends_count": 39, "statues_count": 1778 }, "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-01-04T10:00:06.000Z"), "id": 684071820698320896, "text": "Want to work at Central Peninsula Hospital? We're #hiring in #Soldotna, AK! Click for details: https://t.co/x709rPzHF0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-151.0583332,60.4877778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Soldotna" }}, "user": { "id": 3278999924, "name": "CPH Jobs", "screen_name": "CPHJobs", "lang": "en", "location": "Soldotna, AK", "create_at": date("2015-07-13"), "description": "At CPH every employee is considered a caregiver, an integral part of our health care team. Follow us to learn more!", "followers_count": 25, "friends_count": 77, "statues_count": 49 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Soldotna, AK", "id": "350518f7c1ac93ff", "name": "Soldotna", "place_type": "city", "bounding_box": rectangle("-151.193024,60.458405 -151.048563,60.511008") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2122, "countyName": "Kenai Peninsula", "cityID": 271640, "cityName": "Soldotna" } }
+{ "create_at": datetime("2016-01-04T10:00:06.000Z"), "id": 684071820715266049, "text": "This cutie is being adorable #kassidykitty #thequeen #dababy @… https://t.co/ZRDUwq8Fh0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.6095,42.8828"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "kassidykitty", "thequeen", "dababy" }}, "user": { "id": 67414258, "name": "brenna neitzke", "screen_name": "bneitzke14", "lang": "en", "location": "grand rapids, Michigan", "create_at": date("2009-08-20"), "description": "Wannabe writer, artist, and poet. supervisor at The Pearl Street Grill.", "followers_count": 45, "friends_count": 136, "statues_count": 1977 }, "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-01-04T10:00:06.000Z"), "id": 684071820962574336, "text": "oh my what a wonderful time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3116206001, "name": "Michael Sylvester", "screen_name": "MichaelSS18", "lang": "en", "location": "Mesquite, TX", "create_at": date("2015-03-25"), "description": "null", "followers_count": 123, "friends_count": 80, "statues_count": 1842 }, "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-04T10:00:06.000Z"), "id": 684071821042307072, "text": "Wind 0.0 mph N. Barometer 29.83 in, Rising slowly. Temperature 55.9 °F. Rain today 0.01 in. Humidity 59%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.13472222,33.81666667"), "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": 94, "statues_count": 13420 }, "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-01-04T10:00:06.000Z"), "id": 684071821260374016, "text": "#Honolulu, Hawaii #Retail #Job: New Store Opening - Retail Sales - Women's Active and Swim at Nordstrom https://t.co/j1c85Mevym #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Honolulu", "Retail", "Job", "Jobs" }}, "user": { "id": 28625039, "name": "TMJ-HON Retail Jobs", "screen_name": "tmj_hon_retail", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 376, "friends_count": 311, "statues_count": 448 }, "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-04T10:00:06.000Z"), "id": 684071821306654721, "text": "@tygwop - tyeeshat", "in_reply_to_status": 684071746920660992, "in_reply_to_user": 628137606, "favorite_count": 0, "retweet_count": 0, "lang": "hi", "is_retweet": false, "user_mentions": {{ 628137606 }}, "user": { "id": 628137606, "name": "Tyeesha", "screen_name": "tygwop", "lang": "en", "location": "null", "create_at": date("2012-07-05"), "description": "Grind now, $tunt later ⚡️ [ Snapchat: tyeeshat ] #18yearsyoung", "followers_count": 1215, "friends_count": 1147, "statues_count": 33203 }, "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-01-04T10:00:06.000Z"), "id": 684071821348491265, "text": "Elite, Elite, Elite! Happy birthday @Elite ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17022240 }}, "user": { "id": 153955013, "name": "Veronica.", "screen_name": "veronicaariana_", "lang": "en", "location": "Tucson, AirZona", "create_at": date("2010-06-09"), "description": "UA '19. #Dreamville", "followers_count": 932, "friends_count": 760, "statues_count": 53351 }, "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-01-04T10:00:06.000Z"), "id": 684071821466025984, "text": "Wow https://t.co/oxHy6zbq8J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 801873144, "name": "adrianna richards", "screen_name": "adri_rich", "lang": "en", "location": "null", "create_at": date("2012-09-03"), "description": "university of Cincinnati ; princess/killer Chloe", "followers_count": 618, "friends_count": 251, "statues_count": 40466 }, "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-04T10:00:06.000Z"), "id": 684071821755322368, "text": "Class number 1: done", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 86645874, "name": "ur mom", "screen_name": "vcrislands", "lang": "en", "location": "Hogwarts", "create_at": date("2009-10-31"), "description": "make believe worlds make us all feel alive.", "followers_count": 18925, "friends_count": 9369, "statues_count": 108880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:06.000Z"), "id": 684071821830918144, "text": "13:00:01 |Temp: 51.5ºF | Wind Chill 51.5ºF |Dew Point 37.2ºF | Rain today: 0.01 inches | Wind: 1.0 mph from the NNW, Gusting to 6.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": 87997 }, "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-04T10:00:06.000Z"), "id": 684071821986164736, "text": "Need I say more?\n\nLet's have some fun, y'all. https://t.co/eb2cbaJmKC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 173693752, "name": "BIGELOW", "screen_name": "ItsAllOK1", "lang": "en", "location": "BOFA, USA", "create_at": date("2010-08-01"), "description": "'til the casket drops *thud*.", "followers_count": 1234, "friends_count": 836, "statues_count": 35686 }, "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-04T10:00:06.000Z"), "id": 684071822170701833, "text": "@alkinikin ....why", "in_reply_to_status": 684071780613525504, "in_reply_to_user": 486091414, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 486091414 }}, "user": { "id": 438516157, "name": "steph", "screen_name": "StephanieRautio", "lang": "en", "location": "null", "create_at": date("2011-12-16"), "description": "snapchat & instagram: stephanierautio", "followers_count": 1071, "friends_count": 722, "statues_count": 17472 }, "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-01-04T10:00:06.000Z"), "id": 684071822258737152, "text": "Wind 4.0 mph NNE. Barometer 30.051 in, Steady. Temperature 28.6 °F. Rain today 0.00 in. Humidity 47%", "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": 3369 }, "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-04T10:00:06.000Z"), "id": 684071822522896384, "text": "Reasons why I'm BROKE!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29340140, "name": "Jacqueline Frimpoma", "screen_name": "AFRICANSrBETTER", "lang": "en", "location": "Gold Coast, Chicago", "create_at": date("2009-04-06"), "description": "I'd rather die ENORMOUS than live dormant... #GodBlessMyHomelandGHANA", "followers_count": 958, "friends_count": 720, "statues_count": 57097 }, "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-04T10:00:06.000Z"), "id": 684071822854234112, "text": "Me rn https://t.co/IDh9d7CnOJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1549027560, "name": "aprill", "screen_name": "__aprill__", "lang": "en", "location": "Upland, CA", "create_at": date("2013-06-26"), "description": "appreciate all the little blessings", "followers_count": 845, "friends_count": 578, "statues_count": 23090 }, "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-04T10:00:06.000Z"), "id": 684071822883700736, "text": "Wind 0 mph --. Barometer 30.28 in, Steady. Temperature 46.0 °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": 19958 }, "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-04T10:00:06.000Z"), "id": 684071822908899328, "text": "I'm at Miller's Twist in Philadelphia, PA https://t.co/JuloEQxO4B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.15928361,39.95364633"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 5723252, "name": "Joia", "screen_name": "boredgirl260", "lang": "en", "location": "Philadelphia, PA, USA", "create_at": date("2007-05-02"), "description": "32 year old girl from Philly, tired of all the bullshit.", "followers_count": 2070, "friends_count": 2201, "statues_count": 80997 }, "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-04T10:00:06.000Z"), "id": 684071823034597376, "text": "Yesterday I blew chunks all over myself in Dereks car and he cleaned it... Definitely took our relationship to a whole new level LOL", "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 | Houston, TX", "followers_count": 940, "friends_count": 518, "statues_count": 24741 }, "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-04T10:00:06.000Z"), "id": 684071823126872065, "text": "Sunny this afternoon, high 60 (16 C). Low 46 (8 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712593, "name": "CorpusChristiWX", "screen_name": "CorpusChristiWX", "lang": "en", "location": "Corpus Christi, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Corpus Christi, Texas", "followers_count": 1013, "friends_count": 93, "statues_count": 8114 }, "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-04T10:00:06.000Z"), "id": 684071823147937793, "text": "nobody listens!!! https://t.co/BpzDupjrS5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 190076085, "name": "honeyberrybri!", "screen_name": "selfiequeenbri", "lang": "en", "location": "RECKLESSTX 2 HOWARDU", "create_at": date("2010-09-12"), "description": "NEVERFOLDSAYLESSDOMORE", "followers_count": 3729, "friends_count": 985, "statues_count": 207095 }, "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-04T10:00:06.000Z"), "id": 684071823240081408, "text": "Use this Reddit post to help you understand where @BernieSanders tax plan is coming from: https://t.co/falT7NgQ3U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 216776631 }}, "user": { "id": 965328150, "name": "Jennifer Schmelder", "screen_name": "jennschmelder", "lang": "en", "location": "null", "create_at": date("2012-11-22"), "description": "Independent Thinker", "followers_count": 72, "friends_count": 98, "statues_count": 1240 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Geneva, IL", "id": "a2df08f56f75a9b4", "name": "Geneva", "place_type": "city", "bounding_box": rectangle("-88.398748,41.861729 -88.262399,41.899198") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17089, "countyName": "Kane", "cityID": 1728872, "cityName": "Geneva" } }
+{ "create_at": datetime("2016-01-04T10:00:06.000Z"), "id": 684071823449796608, "text": "@BasedGAWDUchiha @xoxoashleyadams am I too big or could I make it in porn ;)", "in_reply_to_status": 683870586255216640, "in_reply_to_user": 1921490210, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1921490210, 2910404614 }}, "user": { "id": 1921490210, "name": "TrunkJesus™", "screen_name": "BasedGAWDUchiha", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2013-09-30"), "description": "The Messiah, TrunkJesus. Mindfucking the world with a sex drive that makes Charlie Sheen look like a virgin. Hung BBC Bull #TrunkNation\n#Srirachalord\n#Based", "followers_count": 868, "friends_count": 3959, "statues_count": 5277 }, "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-04T10:00:06.000Z"), "id": 684071823508520960, "text": "It ain't happen to me but that's one of the most disrespectful things you can do somebody", "in_reply_to_status": 684071097537400832, "in_reply_to_user": 378734845, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 378734845, "name": "Fredo.", "screen_name": "saucefredo_", "lang": "en", "location": "Houston/Fort Bend|San Marcos", "create_at": date("2011-09-23"), "description": "8️⃣3️⃣2️⃣ to my city. Texas State University. #eXclusivePromo", "followers_count": 1512, "friends_count": 1194, "statues_count": 35282 }, "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-01-04T10:00:06.000Z"), "id": 684071823676313600, "text": "@_hallijacobson it says he's coming out", "in_reply_to_status": 684071727907749888, "in_reply_to_user": 1663367210, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1663367210 }}, "user": { "id": 818531814, "name": "hannah", "screen_name": "_hannahdoss", "lang": "en", "location": "ptown", "create_at": date("2012-09-11"), "description": "lost in the thrill of it all // ॐ", "followers_count": 972, "friends_count": 725, "statues_count": 22845 }, "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-01-04T10:00:06.000Z"), "id": 684071823764533248, "text": "Come craft a Little Tug with me on Jan. 30 at the Brooklyn Public Library. https://t.co/tacI0NPHvX @nporterbooks @bookbrag @parkslope5thave", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20846564, 544304020, 38336232 }}, "user": { "id": 141726392, "name": "Stephen Savage", "screen_name": "Savageartist", "lang": "en", "location": "Brooklyn", "create_at": date("2010-05-08"), "description": "Bestselling illustrator/author. Creator of WHERE'S WALRUS? AND PENGUIN?, SUPERTRUCK and LITTLE TUG. 2015 Sendak Fellow. Dad. Teacher. Brooklynite.", "followers_count": 1232, "friends_count": 1104, "statues_count": 3609 }, "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-04T10:00:06.000Z"), "id": 684071823948951552, "text": "My mom always said that if a man wanted to see you, he'll put forth the time & effort. You don't always have to go out of your way..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27983804, "name": "Jasmine Hendrix", "screen_name": "juicyJAYCouture", "lang": "en", "location": "Tampa, FL", "create_at": date("2009-03-31"), "description": "AkilahJanaeBlount | 21 | YourFavoriteFlight Attendant | IG - JaszCouture_ | Black&Italian", "followers_count": 3303, "friends_count": 858, "statues_count": 85331 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McGovern, PA", "id": "010f5a2b48665dc5", "name": "McGovern", "place_type": "city", "bounding_box": rectangle("-80.243973,40.212004 -80.17736,40.25042") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42125, "countyName": "Washington", "cityID": 4246160, "cityName": "McGovern" } }
+{ "create_at": datetime("2016-01-04T10:00:06.000Z"), "id": 684071824251031552, "text": "#MCM Poe Dameron more like HOT DAMNeron am I riiiiiii...illjustseemyselfout https://t.co/pFK1zITEsQ", "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": 23785401, "name": "Andrea Duffy", "screen_name": "_AndreaDuffy", "lang": "en", "location": "Philadelphia", "create_at": date("2009-03-11"), "description": "Mid-days 9a-2p on @MoreFmPhilly. One-?th of @NoGhostsImprov. Voiceover Gal. @TempleUniv alum. Stone Cold Weirdo. IG: http://instagram.com/_andreaduffy/", "followers_count": 634, "friends_count": 1732, "statues_count": 4587 }, "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-04T10:00:07.000Z"), "id": 684071824271904768, "text": "@switmode someone like that can't be associated with again", "in_reply_to_status": 684071613831221248, "in_reply_to_user": 749110051, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 749110051 }}, "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": 1192, "friends_count": 993, "statues_count": 97108 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pecan Grove, TX", "id": "010edcb7655caac2", "name": "Pecan Grove", "place_type": "city", "bounding_box": rectangle("-95.765227,29.585216 -95.678943,29.684395") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4856482, "cityName": "Pecan Grove" } }
+{ "create_at": datetime("2016-01-04T10:00:07.000Z"), "id": 684071824284631040, "text": "@The_BASEDJESUS hell yeah. You gonna be like https://t.co/tDCYx5FHsd", "in_reply_to_status": 684071578460499968, "in_reply_to_user": 2569584455, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2569584455 }}, "user": { "id": 376790553, "name": "Jean Kiddo", "screen_name": "Kidd_Funkadelic", "lang": "en", "location": "Somewhere in the Midwest", "create_at": date("2011-09-20"), "description": "My name is John, it's nice to meet you. I love you and hope you do wonderful things in life. No, I don't rap. #GorgeousGang", "followers_count": 775, "friends_count": 532, "statues_count": 43541 }, "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-04T10:00:07.000Z"), "id": 684071824355946496, "text": "Good Luck today! @Moneyball_AW1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2384118126 }}, "user": { "id": 358517100, "name": "Akwia", "screen_name": "Til_10", "lang": "en", "location": "Michigan, USA", "create_at": date("2011-08-19"), "description": "Roeper '16 College of Wooster '20", "followers_count": 499, "friends_count": 336, "statues_count": 10766 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Birmingham, MI", "id": "b456150eed80704e", "name": "Birmingham", "place_type": "city", "bounding_box": rectangle("-83.248591,42.530837 -83.185059,42.566543") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2608640, "cityName": "Birmingham" } }
+{ "create_at": datetime("2016-01-04T10:00:07.000Z"), "id": 684071824523673604, "text": "the feeling is mutual", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1195037119, "name": "lauren layne", "screen_name": "laurenelaynee", "lang": "en", "location": "asheville, nc", "create_at": date("2013-02-18"), "description": "♎️♎️", "followers_count": 624, "friends_count": 701, "statues_count": 4135 }, "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-04T10:00:07.000Z"), "id": 684071824557117440, "text": "@MyBlackMatters https://t.co/0ATnv5uoJb", "in_reply_to_status": 684030435115053056, "in_reply_to_user": 436721105, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 436721105 }}, "user": { "id": 3190189801, "name": "chidi . ❤️", "screen_name": "_chidimmaaa", "lang": "en", "location": "null", "create_at": date("2015-05-09"), "description": "Nigerian | Htx ,", "followers_count": 661, "friends_count": 570, "statues_count": 5468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-04T10:00:07.000Z"), "id": 684071824632721409, "text": "@Nicole__Wallace YES", "in_reply_to_status": 682755805305769984, "in_reply_to_user": 2428218956, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2428218956 }}, "user": { "id": 1027769246, "name": "olivia denny", "screen_name": "liv_denny", "lang": "en", "location": "Avon", "create_at": date("2012-12-21"), "description": "JM3", "followers_count": 477, "friends_count": 408, "statues_count": 5377 }, "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-01-04T10:00:07.000Z"), "id": 684071824645201920, "text": "I STAYED HOME AND BARELY WOKE UP HAHAHAHAHAH my life is a mess", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 543107122, "name": "Diljit Singh", "screen_name": "_Diljit_Singh_", "lang": "en", "location": "San Antonio, Texas", "create_at": date("2012-04-01"), "description": "~Howdy, JMHS SENIOR C|O16 ~IG-diljit_singh/FB-Diljit Singh", "followers_count": 542, "friends_count": 388, "statues_count": 7623 }, "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-04T10:00:07.000Z"), "id": 684071824729092096, "text": "1/4/2016 - 12:00\nTemp: 35.4F \nHum: 78%\nWind: 4.0 mph\nBaro: 30.475in. & 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": 105, "friends_count": 52, "statues_count": 48689 }, "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-04T10:00:07.000Z"), "id": 684071824989130752, "text": "See our latest #Lahaina, HI #job and click to apply: Key Holder - https://t.co/nD9SonVL1X #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-156.6816028,20.8852271"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Lahaina", "job", "Hiring", "CareerArc" }}, "user": { "id": 38082277, "name": "TMJ-MAU Jobs", "screen_name": "tmj_mau_jobs", "lang": "en", "location": "Maui, HI", "create_at": date("2009-05-05"), "description": "Follow this account for geo-targeted Other job tweets in Maui, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 548, "friends_count": 535, "statues_count": 16 }, "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-01-04T10:00:07.000Z"), "id": 684071825006051328, "text": "happy birthday Alex!!��✌��️ @AlexOskuie miss ya!!! Hope your day is as wonderful as you are!!���� ily #nooffense��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "nooffense" }}, "user_mentions": {{ 2493136721 }}, "user": { "id": 2615008373, "name": "ellie", "screen_name": "EllieHeider", "lang": "en", "location": "madison' 18", "create_at": date("2014-06-16"), "description": "null", "followers_count": 535, "friends_count": 492, "statues_count": 654 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakton, VA", "id": "015a1efa44b8123a", "name": "Oakton", "place_type": "city", "bounding_box": rectangle("-77.395515,38.863569 -77.253414,38.930322") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5158472, "cityName": "Oakton" } }
+{ "create_at": datetime("2016-01-04T10:00:07.000Z"), "id": 684071825060556802, "text": "Got a parking ticket in beavercreek for 4 dollars wtf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 380407494, "name": "Clark Griswold", "screen_name": "ASAP_OB", "lang": "en", "location": "Dayton, OH", "create_at": date("2011-09-26"), "description": "#RI5EABOVE the booty to get to the cooter", "followers_count": 419, "friends_count": 508, "statues_count": 4617 }, "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-01-04T10:00:07.000Z"), "id": 684071825316405248, "text": "Fortira, Inc. is hiring! Computers #jobs in PRINCETON Apply today https://t.co/G7JJHKdvgH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.662206,40.378246"), "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": 169, "friends_count": 42, "statues_count": 72083 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Princeton North, NJ", "id": "012c2b170b251774", "name": "Princeton North", "place_type": "city", "bounding_box": rectangle("-74.697861,40.352614 -74.620837,40.393136") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34021, "countyName": "Mercer", "cityID": 3460900, "cityName": "Princeton" } }
+{ "create_at": datetime("2016-01-04T10:00:07.000Z"), "id": 684071825475674112, "text": "Ain't no one gon discuss this or we just gonna let it slide? https://t.co/0h82dpcZDH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 253339856, "name": "Pusha Man", "screen_name": "ThouMustChill", "lang": "en", "location": "UNT", "create_at": date("2011-02-16"), "description": "It's a free-for-all out here. Save Yourself.", "followers_count": 287, "friends_count": 204, "statues_count": 22904 }, "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-04T10:00:07.000Z"), "id": 684071825576349696, "text": "#IT #Job in #RedwoodShores, CA: Senior Cloud System Test Engineering at Oracle https://t.co/lLPB8TlGUI #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.2455364,37.5364134"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "IT", "Job", "RedwoodShores", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 112596930, "name": "TMJ-CA IT Jobs", "screen_name": "tmj_CA_it", "lang": "en", "location": "California", "create_at": date("2010-02-08"), "description": "Follow this account for geo-targeted Software Dev. - General/IT job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 480, "friends_count": 332, "statues_count": 616 }, "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-01-04T10:00:07.000Z"), "id": 684071825660338176, "text": "@Reload715 xp*", "in_reply_to_status": 684071770618523648, "in_reply_to_user": 3314611386, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2161601372 }}, "user": { "id": 3314611386, "name": "Recuff", "screen_name": "Recuff1", "lang": "en", "location": "Piqua, OH", "create_at": date("2015-08-13"), "description": "|BO3 God| @Detricakes is my E-GIRL", "followers_count": 227, "friends_count": 152, "statues_count": 2301 }, "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-04T10:00:07.000Z"), "id": 684071825765105665, "text": "Can you recommend anyone for this #job? Sales Associate - https://t.co/hwLvyiVEpq #Poughkeepsie, NY #Sales #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9209701,41.7003713"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Poughkeepsie", "Sales", "Hiring" }}, "user": { "id": 2565844555, "name": "ExpressCareers", "screen_name": "expresscareers", "lang": "en", "location": "Columbus, OH", "create_at": date("2014-06-13"), "description": "Love everything fashion and the latest trends and styles? Why not mix business with pleasure - work at Express, we're hiring!", "followers_count": 260, "friends_count": 60, "statues_count": 856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Poughkeepsie, NY", "id": "abf8452ec690299f", "name": "Poughkeepsie", "place_type": "city", "bounding_box": rectangle("-73.94318,41.669084 -73.890803,41.717381") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36027, "countyName": "Dutchess", "cityID": 3659641, "cityName": "Poughkeepsie" } }
+{ "create_at": datetime("2016-01-04T10:00:07.000Z"), "id": 684071825870028801, "text": "Beef is when your moms ain't safe up in these streets.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2378917263, "name": "MAMÍ", "screen_name": "SandysTheGEM", "lang": "en", "location": "Worcester, MA", "create_at": date("2014-03-03"), "description": "a l o n e in this world so c o l d", "followers_count": 1643, "friends_count": 1239, "statues_count": 33471 }, "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-04T10:00:07.000Z"), "id": 684071826008375296, "text": "Partly cloudy this afternoon, high 42 (6 C). Low 27 (-3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712706, "name": "TulsaWX", "screen_name": "TulsaWX", "lang": "en", "location": "Tulsa, Oklahoma", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Tulsa, Oklahoma", "followers_count": 517, "friends_count": 93, "statues_count": 8176 }, "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-04T10:00:07.000Z"), "id": 684071826624872448, "text": "We're #hiring! Click to apply: Customer Service - Restaurant - https://t.co/fDBLeXvf2Z #Hospitality #StPaul, IN #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.6283088,39.4281048"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Hospitality", "StPaul", "Job", "Jobs" }}, "user": { "id": 2651294322, "name": "Love's Jobs", "screen_name": "LovesJobs", "lang": "en", "location": "Coast to Coast", "create_at": date("2014-07-16"), "description": "Love’s believes in the unlimited potential of the employees who make up our company. Follow us to learn more about our open positions!", "followers_count": 31, "friends_count": 7, "statues_count": 6637 }, "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": 18031, "countyName": "Decatur", "cityID": 1867212, "cityName": "St. Paul" } }
+{ "create_at": datetime("2016-01-04T10:00:07.000Z"), "id": 684071826859933697, "text": "@JeffreyHayzlett thank you for following #MP", "in_reply_to_status": -1, "in_reply_to_user": 14855279, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MP" }}, "user_mentions": {{ 14855279 }}, "user": { "id": 1494430362, "name": "Mel Jones the BEAST", "screen_name": "motivationphil", "lang": "en", "location": "Williamsburg, Virginia", "create_at": date("2013-06-08"), "description": "Motivational Philosopher, Public Speaker, Personal Trainer, and Life Coach. #MotivationalPhilosopher #MP #MotivationalSpeaker #Entrepreneur", "followers_count": 1242, "friends_count": 501, "statues_count": 5882 }, "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-01-04T10:00:07.000Z"), "id": 684071827149307904, "text": "@babymom_ this my home i love u guys", "in_reply_to_status": 684071781842468864, "in_reply_to_user": 2215841165, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2215841165 }}, "user": { "id": 2215841165, "name": "x", "screen_name": "babymom_", "lang": "en", "location": "dirty jersey ", "create_at": date("2013-12-09"), "description": "i love you", "followers_count": 516, "friends_count": 219, "statues_count": 26129 }, "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-04T10:00:07.000Z"), "id": 684071827337920512, "text": "Sometimes flowers can give me great lipstick and eyeshadow color ideas...������ #colorful… https://t.co/wIhYwdtPyq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.665676,42.009311"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "colorful" }}, "user": { "id": 17967909, "name": "Sonia Roselli", "screen_name": "soniaroselli", "lang": "en", "location": "Chicago", "create_at": date("2008-12-08"), "description": "Chicago makeup artist & beauty expert.", "followers_count": 3450, "friends_count": 2115, "statues_count": 1895 }, "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-04T10:00:07.000Z"), "id": 684071827384217600, "text": "That's why I'm a fan & not a reporter anymore. Basically, I have feelings for my club that outweigh this donor die reporting. #myclub #Jets", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "myclub", "Jets" }}, "user": { "id": 516081403, "name": "Lon from @JetsTwit", "screen_name": "lonjets", "lang": "en", "location": "Tampa Bay, FL USA", "create_at": date("2012-03-05"), "description": "Lexus Certified. Delicate Genius. Talent @JetsTwit Podcast. NY Jets & Springsteen are my passions. Chrome wheeled fuel injected and steppin' out over the line.", "followers_count": 576, "friends_count": 1047, "statues_count": 11222 }, "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-01-04T10:00:07.000Z"), "id": 684071827602210816, "text": "Steroid shots hurt 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": 2243205818, "name": "PrincessPaige", "screen_name": "PaigeeGood", "lang": "en", "location": "null", "create_at": date("2013-12-12"), "description": "Blessed with the best☆\nBradley❤", "followers_count": 873, "friends_count": 827, "statues_count": 8622 }, "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-04T10:00:07.000Z"), "id": 684071827841273856, "text": "@LiberalLegion - plural of armed rednecks, according to LA Times, is activists", "in_reply_to_status": 684070836840574976, "in_reply_to_user": 24716381, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24716381 }}, "user": { "id": 18928662, "name": "Franky Lamouche", "screen_name": "FrankyLamouche", "lang": "en", "location": "Hollywood Troller (Not Pimp)", "create_at": date("2009-01-12"), "description": "Servant to God", "followers_count": 270, "friends_count": 101, "statues_count": 61155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:07.000Z"), "id": 684071827849650176, "text": "@liviwatson still very clever liv lol", "in_reply_to_status": 684071597855084545, "in_reply_to_user": 61273231, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 61273231 }}, "user": { "id": 379343781, "name": "Travis Clark", "screen_name": "Clarknado_15", "lang": "en", "location": "Bluffton, OH", "create_at": date("2011-09-24"), "description": "Student Athlete at Bluffton University #Baseball #BUClassOf2016", "followers_count": 733, "friends_count": 581, "statues_count": 35534 }, "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-04T10:00:07.000Z"), "id": 684071827883323392, "text": "merrill lynch is hiring! Computers #jobs in PENNINGTON Apply today https://t.co/GLfTcGW49e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.79563,40.328228"), "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": 169, "friends_count": 42, "statues_count": 72085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pennington, NJ", "id": "2bc2f904113c1b8a", "name": "Pennington", "place_type": "city", "bounding_box": rectangle("-74.837073,40.298925 -74.766126,40.339482") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34021, "countyName": "Mercer", "cityID": 3457600, "cityName": "Pennington" } }
+{ "create_at": datetime("2016-01-04T10:00:07.000Z"), "id": 684071827904176128, "text": "I'm a push kind of person and you're a pull kind of person and I need a push kind of person so I don't walk right thru a pull kind of person", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2337316268, "name": "j rob", "screen_name": "jadadiane16", "lang": "en", "location": "wilding somewhere", "create_at": date("2014-02-10"), "description": "• call me J • DON'T be a wanna-be • learning is a permanent change in behavior • TRAPSOUL • angel with an attitude • 18 •", "followers_count": 1374, "friends_count": 761, "statues_count": 49022 }, "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-04T10:00:07.000Z"), "id": 684071828042588160, "text": "All I wanna be, is all that I can be.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 380184172, "name": "James Dean", "screen_name": "iRun_253", "lang": "en", "location": "Tacoma ", "create_at": date("2011-09-25"), "description": "R.I.P Dad . #SDL Dream as if you'll live for ever. live as if youll die tomorrow.", "followers_count": 243, "friends_count": 184, "statues_count": 6785 }, "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-04T10:00:07.000Z"), "id": 684071828088815618, "text": "@GoMemphis @jenniferbiggs1 Great article !", "in_reply_to_status": 684070118285021184, "in_reply_to_user": 4782971, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4782971, 46463835 }}, "user": { "id": 75421732, "name": "WITIN RADIO", "screen_name": "witinradio", "lang": "en", "location": "Memphis, TN", "create_at": date("2009-09-18"), "description": "A site devoted to news, pop culture, music and life lessons. The Place For Intelligent Audio", "followers_count": 2877, "friends_count": 571, "statues_count": 15565 }, "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-04T10:00:07.000Z"), "id": 684071828390850561, "text": "@vkataxdog Running a serious presidential campaign is hard and time consuming. Any elected official, Dem or Rep is going to miss votes.", "in_reply_to_status": 684066022593302529, "in_reply_to_user": 259978022, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 259978022 }}, "user": { "id": 2391275342, "name": "Trey Hammett", "screen_name": "trey_hammett", "lang": "en", "location": "Atlanta, GA", "create_at": date("2014-03-15"), "description": "Husband, father of twins, golf enthusiast and political junkie.", "followers_count": 19, "friends_count": 56, "statues_count": 52 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Avondale Estates, GA", "id": "599785cb8cd0515d", "name": "Avondale Estates", "place_type": "city", "bounding_box": rectangle("-84.285727,33.754915 -84.250519,33.780942") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1304644, "cityName": "Avondale Estates" } }
+{ "create_at": datetime("2016-01-04T10:00:07.000Z"), "id": 684071828420202496, "text": "Some jazzy dark pop in yo’ face — Kopecky - Thrill (love the brass!) https://t.co/eZ7L8YxX9g", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 6532552, "name": "Nate Kohari", "screen_name": "nkohari", "lang": "en", "location": "Raleigh, NC, USA", "create_at": date("2007-06-02"), "description": "Co-founder of TaskTorch (http://tasktorch.com). Formerly CTO of Adzerk and co-founder of AgileZen (acquired 2010). Just a kid from Akron, Ohio.", "followers_count": 5030, "friends_count": 240, "statues_count": 14531 }, "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-04T10:00:08.000Z"), "id": 684071828558458881, "text": "Tired af", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2320159740, "name": "Eskandar Haidari", "screen_name": "Haidari_12", "lang": "en", "location": "null", "create_at": date("2014-01-30"), "description": "Sometimes in life you got to make a decision to move on from the past or relive the same mistakes you made.", "followers_count": 98, "friends_count": 145, "statues_count": 1084 }, "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-01-04T10:00:08.000Z"), "id": 684071828587851778, "text": "Partly cloudy this afternoon, high 42 (6 C). Low 28 (-2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712679, "name": "OKCWX", "screen_name": "OKCWX", "lang": "en", "location": "Oklahoma City, Oklahoma", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Oklahoma City, Oklahoma", "followers_count": 186, "friends_count": 93, "statues_count": 8203 }, "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-04T10:00:08.000Z"), "id": 684071828659150848, "text": "Have you met my bestfriend? https://t.co/7ef7SPOUND", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 69795726, "name": "megs new booty", "screen_name": "megsparkles3", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-08-28"), "description": "null", "followers_count": 228, "friends_count": 376, "statues_count": 3454 }, "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-04T10:00:08.000Z"), "id": 684071828659159040, "text": "Throat is also super sore today...along with the lower half of my body because I kicked ass at the gym yesterday. #beastmode", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "beastmode" }}, "user": { "id": 2646570752, "name": "Trash Queen 2K16", "screen_name": "StraylightWTF", "lang": "en", "location": "Colorado", "create_at": date("2014-07-14"), "description": "I cosplay. I make stuff. I art. I fursuit. I get bored easy. Often sport the maturity level of a five year old. Owner of Straylight Studio. Happily taken.", "followers_count": 162, "friends_count": 233, "statues_count": 7547 }, "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-01-04T10:00:08.000Z"), "id": 684071829087096834, "text": "https://t.co/iTjaVStbv6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 227564007, "name": "TRÉ", "screen_name": "mulattoodiablo", "lang": "en", "location": "New York, NY", "create_at": date("2010-12-16"), "description": "P O L Y M A T H", "followers_count": 4147, "friends_count": 389, "statues_count": 57871 }, "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-04T10:00:08.000Z"), "id": 684071829296693248, "text": "you wasted 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": 2888761449, "name": "princess alpal", "screen_name": "aalexxandrriaa", "lang": "en", "location": "basehor, ks ", "create_at": date("2014-11-22"), "description": "young, wild, & free((:", "followers_count": 308, "friends_count": 386, "statues_count": 2906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bonner Springs, KS", "id": "6ca2f5432d659026", "name": "Bonner Springs", "place_type": "city", "bounding_box": rectangle("-94.908753,39.047837 -94.843059,39.096687") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20209, "countyName": "Wyandotte", "cityID": 2007975, "cityName": "Bonner Springs" } }
+{ "create_at": datetime("2016-01-04T10:00:08.000Z"), "id": 684071829531701248, "text": "Verizon Wireles is hiring! Sr. Member Tech #jobs in PISCATAWAY Apply today https://t.co/ixhnOKxpSd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.464675,40.551764"), "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": 169, "friends_count": 42, "statues_count": 72086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Piscataway, NJ", "id": "01a0aa02181f66f1", "name": "Piscataway", "place_type": "city", "bounding_box": rectangle("-74.524236,40.506013 -74.40873,40.597526") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34035, "countyName": "Somerset" } }
+{ "create_at": datetime("2016-01-04T10:00:08.000Z"), "id": 684071829644836864, "text": "See our latest #Davenport, IA #job and click to apply: Caregiver - https://t.co/3s1xbY9Yre #Healthcare #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.5776367,41.5236437"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Davenport", "job", "Healthcare", "Hiring", "CareerArc" }}, "user": { "id": 57694228, "name": "TMJ-IA Health Jobs", "screen_name": "tmj_ia_health", "lang": "en", "location": "Iowa Non-Metro", "create_at": date("2009-07-17"), "description": "Follow this account for geo-targeted Healthcare job tweets in Iowa Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 345, "friends_count": 295, "statues_count": 432 }, "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-01-04T10:00:08.000Z"), "id": 684071829695270912, "text": "or #7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2806472778, "name": "Im Belle Cabrera", "screen_name": "WorldGlover", "lang": "en", "location": "Whitestone, Queens", "create_at": date("2014-09-12"), "description": "#WHO #UN English 2nd language", "followers_count": 33, "friends_count": 118, "statues_count": 45494 }, "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-04T10:00:08.000Z"), "id": 684071829821079552, "text": "@Khamaelthefirst 21��?", "in_reply_to_status": 684071729518481408, "in_reply_to_user": 983747960, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 983747960 }}, "user": { "id": 468961328, "name": "King Nathan", "screen_name": "rodriguezdagawd", "lang": "en", "location": "Norfolk, VA", "create_at": date("2012-01-19"), "description": "@Ashley_Bxtchess | Miami ✈️Norfolk | #ImUglyButMyGirlAintHive | #Sneedssons", "followers_count": 5907, "friends_count": 696, "statues_count": 114759 }, "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-04T10:00:08.000Z"), "id": 684071829971992578, "text": "Bruh I just had a crazy moment �� like throwing a shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3022916833, "name": "Flawlesstrapp", "screen_name": "FlawlessTrappp", "lang": "en", "location": "null", "create_at": date("2015-02-16"), "description": "null", "followers_count": 272, "friends_count": 301, "statues_count": 4654 }, "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-01-04T10:00:08.000Z"), "id": 684071830022275072, "text": "Af https://t.co/jPMvjE36uG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 465239669, "name": "Carvontez ♋️", "screen_name": "PlayGod_V", "lang": "en", "location": "Louisiana ✈️ Texas ", "create_at": date("2012-01-15"), "description": "19 | #TJC16 | Football | PlayGod$ | Business Major", "followers_count": 2851, "friends_count": 1035, "statues_count": 112237 }, "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-04T10:00:08.000Z"), "id": 684071830102081536, "text": "I remember the day back from break I always rocked new outfits and now I'm gonna rock all the new hoodies leggings and sweatpants I got", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1487967985, "name": "Dez", "screen_name": "DominguezDez", "lang": "en", "location": "tree hill, NC", "create_at": date("2013-06-06"), "description": "GBN varsity cheer ✨ chicago. #futurecyclone at Iowa state university", "followers_count": 1147, "friends_count": 1461, "statues_count": 12448 }, "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.887803 -88.081435,41.942768") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1711332, "cityName": "Carol Stream" } }
+{ "create_at": datetime("2016-01-04T10:00:08.000Z"), "id": 684071830273953792, "text": "@ImDatNigga_Jack You drunk bro?\nBeen poopin tf outta these tweets! ����������������", "in_reply_to_status": -1, "in_reply_to_user": 46305981, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 46305981 }}, "user": { "id": 441157727, "name": "the8ThLetter", "screen_name": "_JossySays", "lang": "en", "location": "null", "create_at": date("2011-12-19"), "description": "#TAMUC #LongLiveClyde", "followers_count": 1650, "friends_count": 1263, "statues_count": 90096 }, "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-04T10:00:08.000Z"), "id": 684071830416527360, "text": "2016 @buffalobills opponents: @steelers fans & friends who still identify w/ @Browns, you gotta join me in #buffalo https://t.co/ExSvrJBm2k", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "buffalo" }}, "user_mentions": {{ 25084916, 19426729, 40358743 }}, "user": { "id": 200493245, "name": "Kris Kennedy ", "screen_name": "kriswitha_K", "lang": "en", "location": "null", "create_at": date("2010-10-09"), "description": "twitter rookie.", "followers_count": 241, "friends_count": 294, "statues_count": 2274 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bexley, OH", "id": "d85013da290a04e7", "name": "Bexley", "place_type": "city", "bounding_box": rectangle("-82.950022,39.947426 -82.922776,39.982287") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3906278, "cityName": "Bexley" } }
+{ "create_at": datetime("2016-01-04T10:00:08.000Z"), "id": 684071830676729857, "text": "Bank of America is hiring! Computers #jobs in JERSEY CITY Apply today https://t.co/sZBTwET6d2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0775,40.7286"), "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": 169, "friends_count": 42, "statues_count": 72087 }, "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-04T10:00:08.000Z"), "id": 684071830718517248, "text": "@LucasNavarreteM is there anywhere to stream Perez press conference?", "in_reply_to_status": -1, "in_reply_to_user": 25437117, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25437117 }}, "user": { "id": 2327671638, "name": "Sy Symonds", "screen_name": "SySymonds", "lang": "en", "location": "null", "create_at": date("2014-02-04"), "description": "Madridista.", "followers_count": 118, "friends_count": 1080, "statues_count": 303 }, "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-01-04T10:00:08.000Z"), "id": 684071830773043200, "text": "So it's 2016 what's gonna change this year for the world.? Cause 2015 was all complaining and no actions.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3061607354, "name": "Carson sowards", "screen_name": "Carsonsowards2", "lang": "en", "location": "Spanish Springs, NV", "create_at": date("2015-03-04"), "description": "❤️Gay.❤️ Single . Spanish Springs High School Aka:SSHS; Troupe 6737; Theater.!", "followers_count": 76, "friends_count": 185, "statues_count": 603 }, "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-01-04T10:00:08.000Z"), "id": 684071830794145793, "text": "@abbydarkstar don't worry I did the same thing last year.", "in_reply_to_status": 684071191556964352, "in_reply_to_user": 66574266, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 66574266 }}, "user": { "id": 450095682, "name": "Ronald Frank Prado", "screen_name": "RonaldFrankPrad", "lang": "en", "location": "Miami, FL", "create_at": date("2011-12-29"), "description": "Proud Geek. Fan of Marvel & DC, Doctor Who, Dragonball, Dragonball Z, Breaking Bad, The Walking Dead, Videogames, Batman, Manga/Anime, Cosplay/Cosplayers.", "followers_count": 445, "friends_count": 1981, "statues_count": 29876 }, "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-04T10:00:08.000Z"), "id": 684071830836084740, "text": "isn’t it weird that we have one hand that knows how to do everything & then one hand that just sits there like “idk how to hold a pencil”", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 232836489, "name": "Brice ⚡", "screen_name": "Diveley116", "lang": "en", "location": "Below Paradise ", "create_at": date("2011-01-01"), "description": "I've got an ill mind", "followers_count": 156, "friends_count": 243, "statues_count": 2799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sherman, IL", "id": "0238676f95a07888", "name": "Sherman", "place_type": "city", "bounding_box": rectangle("-89.631179,39.86604 -89.576509,39.925015") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17167, "countyName": "Sangamon", "cityID": 1769342, "cityName": "Sherman" } }
+{ "create_at": datetime("2016-01-04T10:00:08.000Z"), "id": 684071830865481728, "text": "I just need my hair done and this pimple to disappear from my forehead ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 329304951, "name": "Q . ⚡️", "screen_name": "ogqueshia", "lang": "en", "location": "nowheretobefound ", "create_at": date("2011-07-04"), "description": "unbothered .", "followers_count": 821, "friends_count": 787, "statues_count": 21519 }, "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-01-04T10:00:08.000Z"), "id": 684071830869561344, "text": "I'm so sad because I don't have any more concerts coming up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2181196477, "name": "Kaya Hayes", "screen_name": "kaya_hayes", "lang": "en", "location": "null", "create_at": date("2013-11-07"), "description": "sup", "followers_count": 207, "friends_count": 83, "statues_count": 1969 }, "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-01-04T10:00:08.000Z"), "id": 684071830970224640, "text": "23 units 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": 4275740840, "name": "Lonnie", "screen_name": "yeaalon_", "lang": "en", "location": "null", "create_at": date("2015-11-17"), "description": "6'4 Shooting Guard OC Hoops #EatTeam", "followers_count": 138, "friends_count": 154, "statues_count": 452 }, "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-04T10:00:08.000Z"), "id": 684071831083610112, "text": "My dog is so over dramatic it's ridiculous", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3214551777, "name": "Abbey", "screen_name": "fuckyouabbey", "lang": "en", "location": "SC/IG; abbey_windle113", "create_at": date("2015-04-27"), "description": "Are you at all haunted by memories past? @amityaffliction", "followers_count": 179, "friends_count": 167, "statues_count": 7092 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Davison, MI", "id": "00ae43d53fe3ca08", "name": "Davison", "place_type": "city", "bounding_box": rectangle("-83.576507,42.975833 -83.488029,43.076749") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2619880, "cityName": "Davison" } }
+{ "create_at": datetime("2016-01-04T10:00:08.000Z"), "id": 684071831221866497, "text": "Oracle #Sales #Job: Senior Sales Consultant (#Princeton, NJ) https://t.co/PdhLmzilCB #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.6590472,40.3487181"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Sales", "Job", "Princeton", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 21631231, "name": "TMJ - NJC Sales Jobs", "screen_name": "tmj_njc_sales", "lang": "en", "location": "Central NJ", "create_at": date("2009-02-22"), "description": "Follow this account for geo-targeted Sales job tweets in Central NJ. Need help? Tweet us at @CareerArc!", "followers_count": 404, "friends_count": 307, "statues_count": 177 }, "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.325071 -74.619759,40.37435") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34021, "countyName": "Mercer", "cityID": 3460900, "cityName": "Princeton" } }
+{ "create_at": datetime("2016-01-04T10:00:08.000Z"), "id": 684071831247142913, "text": "my heart is so full of love for the two of you ❤️ @ Cox Convention… https://t.co/w3MrEG40hB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.5147324,35.4653893"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1032055926, "name": "k a m i l", "screen_name": "kamiliadeshae_", "lang": "en", "location": "Oklahoma, USA", "create_at": date("2012-12-23"), "description": "ROLL CHOS", "followers_count": 980, "friends_count": 715, "statues_count": 11065 }, "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-04T10:00:08.000Z"), "id": 684071831322509312, "text": "Sunny this afternoon, high 58 (14 C). Low 36 (2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712488, "name": "SanAntonioWX", "screen_name": "SanAntonioWX", "lang": "en", "location": "San Antonio, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for San Antonio, Texas", "followers_count": 251, "friends_count": 93, "statues_count": 8193 }, "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-04T10:00:08.000Z"), "id": 684071831431704576, "text": "@CLAUWBAT Mmmm hehe love teasing you soo good MyGorgeousLove Hehe We'll stop for now Mwahh, can't stop loving you :3", "in_reply_to_status": 684071242392055810, "in_reply_to_user": 137154600, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 137154600 }}, "user": { "id": 216177858, "name": "ReginaldAnderson", "screen_name": "Rfourone", "lang": "en", "location": "Brooklyn, New York.", "create_at": date("2010-11-15"), "description": "#TeamDarkSide #Gotham #Batman #DCComics I'm In love with My #Batwoman @CLAUWBAT #BFF @1michellecat 21+ {D,L,S,AT,AC,V}", "followers_count": 2865, "friends_count": 2546, "statues_count": 92333 }, "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-04T10:00:08.000Z"), "id": 684071831465115648, "text": "#NewYork, NY #Retail #Job: Stylist at bebe https://t.co/4g7vhJv6Ks #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.984945,40.7489282"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NewYork", "Retail", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 28447705, "name": "NYC Retail Jobs", "screen_name": "tmj_nyc_retail", "lang": "en", "location": "New York, NY", "create_at": date("2009-04-02"), "description": "Follow this account for geo-targeted Retail job tweets in New York, NY. Need help? Tweet us at @CareerArc!", "followers_count": 611, "friends_count": 315, "statues_count": 1073 }, "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-04T10:00:08.000Z"), "id": 684071831746158592, "text": "I'm hangry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 734321666, "name": "Isabelle", "screen_name": "isabelleeeed8", "lang": "en", "location": "Meridian, ID", "create_at": date("2012-08-02"), "description": "sc: isabelled8", "followers_count": 836, "friends_count": 699, "statues_count": 13940 }, "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-04T10:00:08.000Z"), "id": 684071831842734084, "text": "@mylestanzer my semester starts o. The 11th, otherwise i would be there like WHOA", "in_reply_to_status": 684071671645515776, "in_reply_to_user": 116588705, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 106943085 }}, "user": { "id": 116588705, "name": "LEAH CLANCY", "screen_name": "Leah_Clancy", "lang": "en", "location": "Los Angeles", "create_at": date("2010-02-22"), "description": "Poetrying MFA @CalArts || Co-Founder/editor @potluckmagazine || Formerly @NYUlocal ||Insta: leah_clancy", "followers_count": 598, "friends_count": 500, "statues_count": 10241 }, "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-04T10:00:08.000Z"), "id": 684071831880388608, "text": "same bullshit different day", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1126459508, "name": "ㅤㅤㅤ", "screen_name": "johnnyt_13", "lang": "en", "location": "Paradise Valley, AZ", "create_at": date("2013-01-27"), "description": "null", "followers_count": 622, "friends_count": 456, "statues_count": 21814 }, "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-04T10:00:08.000Z"), "id": 684071831939203072, "text": "@bigmamastaley STOP", "in_reply_to_status": 684059086086926336, "in_reply_to_user": 3529404923, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3529404923 }}, "user": { "id": 337445102, "name": "Katy", "screen_name": "Blink1katy2", "lang": "en", "location": "null", "create_at": date("2011-07-17"), "description": "*shrugs at camera*", "followers_count": 370, "friends_count": 560, "statues_count": 9031 }, "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-01-04T10:00:08.000Z"), "id": 684071832127852545, "text": "Connor & Troye https://t.co/s0ZdIxS2cu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2744042788, "name": "maura ✨", "screen_name": "keendabean95", "lang": "en", "location": "Austin, TX", "create_at": date("2014-08-15"), "description": "k,j,r,t / 6 + glozell + jenn + king bach + justin bieber ❤️", "followers_count": 597, "friends_count": 105, "statues_count": 8594 }, "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-04T10:00:08.000Z"), "id": 684071832140550144, "text": "You gon be a daddy soon with this mentality �������� https://t.co/RvQLwhY4f6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3326004867, "name": "AJ", "screen_name": "OBJumperr", "lang": "en", "location": "Champaign, IL", "create_at": date("2015-06-14"), "description": "RIP Uncle Josh, you were the realist. #IDontAnswerToFeminists.", "followers_count": 1050, "friends_count": 185, "statues_count": 414 }, "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-04T10:00:08.000Z"), "id": 684071832220139520, "text": "I love the city", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 47772477, "name": "Gina Tassi", "screen_name": "ginax0x123", "lang": "en", "location": "Sicklerville, NJ ", "create_at": date("2009-06-16"), "description": "right back on my worst behavior", "followers_count": 341, "friends_count": 504, "statues_count": 7579 }, "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-04T10:00:08.000Z"), "id": 684071832446701568, "text": "@Yahweigh The most complete player in the NBA. Outside of LeBron he's my favorite player", "in_reply_to_status": 684071628674826240, "in_reply_to_user": 3015194805, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3015194805 }}, "user": { "id": 323255173, "name": "A Bucs Life", "screen_name": "HipHopIsDeaddd", "lang": "en", "location": "Tampa Bay. Fire the cannons", "create_at": date("2011-06-24"), "description": "PRO BLACK. A BUCS FANATIC. BLACK SKATEBOARDER. HIP HOP LOVER. RAY CHARLES TO THE WHITE WOMAN", "followers_count": 6998, "friends_count": 4577, "statues_count": 240210 }, "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-04T10:00:08.000Z"), "id": 684071832580788224, "text": "::/", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2381555988, "name": "bella", "screen_name": "BellaReilley", "lang": "en", "location": "null", "create_at": date("2014-03-09"), "description": "mchs'19 | volleyball", "followers_count": 281, "friends_count": 254, "statues_count": 494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glen Ellyn, IL", "id": "d77ed7c9521c898d", "name": "Glen Ellyn", "place_type": "city", "bounding_box": rectangle("-88.089723,41.8152 -88.027005,41.903992") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1729756, "cityName": "Glen Ellyn" } }
+{ "create_at": datetime("2016-01-04T10:00:09.000Z"), "id": 684071832962658306, "text": "So they really gon give me keys to open this store tomorrow and I'm still training", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1914116737, "name": "__callme_ash", "screen_name": "ThatsASH2you", "lang": "en", "location": "NJ ✈️D[M]V ", "create_at": date("2013-09-28"), "description": "Woman of God | MSU alumni | Romans 12:12 | She who finds true love finds life ❤️", "followers_count": 275, "friends_count": 433, "statues_count": 6180 }, "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-04T10:00:09.000Z"), "id": 684071833071570945, "text": "Can you recommend anyone for this #job? Shift Manager - https://t.co/oVpw14PGFl #MadeDifferent #Mankato, MN #Hiring https://t.co/xIcHwOLo5L", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.9993996,44.1635775"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "MadeDifferent", "Mankato", "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": 376, "friends_count": 460, "statues_count": 2967 }, "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-01-04T10:00:09.000Z"), "id": 684071833105207297, "text": "There's too many people trying to live on earth", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1000591519, "name": "cait!!!!!!!!!!!", "screen_name": "___blackhole", "lang": "en", "location": "WVU", "create_at": date("2012-12-09"), "description": "my own larger than life sized best friend", "followers_count": 463, "friends_count": 241, "statues_count": 106817 }, "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-04T10:00:09.000Z"), "id": 684071833222529025, "text": "I have no insurance and I can't afford my medicine. I'm so fucking jealous of every single person that got to keep their fucking soonercare", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3193600904, "name": "whatever", "screen_name": "hxd00", "lang": "en", "location": "null", "create_at": date("2015-05-12"), "description": "null", "followers_count": 223, "friends_count": 171, "statues_count": 4434 }, "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-04T10:00:09.000Z"), "id": 684071833314930688, "text": "@MBridegam escalator myth?", "in_reply_to_status": 684071427394281472, "in_reply_to_user": 558143796, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 558143796 }}, "user": { "id": 3269706379, "name": "Blue Tarp", "screen_name": "BluTarp", "lang": "en", "location": "bikesf walksf", "create_at": date("2015-07-05"), "description": "bicyclists are gifts of god, gifts of love", "followers_count": 253, "friends_count": 354, "statues_count": 7477 }, "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-04T10:00:09.000Z"), "id": 684071833449148416, "text": "Love this beauty ❤️ @ Club Haus https://t.co/PSa2K3ebC2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.3359081,40.6699656"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 475152683, "name": "Danielle DeSantis", "screen_name": "daniellesayss", "lang": "en", "location": " ", "create_at": date("2012-01-26"), "description": "Instagram : @danielledesantiss", "followers_count": 2233, "friends_count": 1176, "statues_count": 24359 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westfield, NJ", "id": "777dd5b3b30a625e", "name": "Westfield", "place_type": "city", "bounding_box": rectangle("-74.379383,40.624437 -74.313149,40.682332") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3479040, "cityName": "Westfield" } }
+{ "create_at": datetime("2016-01-04T10:00:09.000Z"), "id": 684071833453359106, "text": "@h0tmessjess yes of course! We can get some seafood", "in_reply_to_status": 684071110971920384, "in_reply_to_user": 68518206, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 68518206 }}, "user": { "id": 251215262, "name": "Elizabeth kalmes", "screen_name": "elizabethkalmes", "lang": "en", "location": "texas", "create_at": date("2011-02-12"), "description": "my cats were right about you.", "followers_count": 64, "friends_count": 94, "statues_count": 370 }, "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-04T10:00:09.000Z"), "id": 684071833721671680, "text": "@stiffass_prince what song that is", "in_reply_to_status": 683400686189477892, "in_reply_to_user": 2240895204, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2240895204 }}, "user": { "id": 2437891598, "name": "SPEEDYHENRY‼️", "screen_name": "DizzyWebHer", "lang": "en", "location": "null", "create_at": date("2014-04-10"), "description": "Official Twitter Page Of Tazhawn Henry ❗️ Student-Athlete & Varsity Running Back at Lamar High School #IWILL. C/O18", "followers_count": 2090, "friends_count": 477, "statues_count": 23026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-04T10:00:09.000Z"), "id": 684071833977503744, "text": "Sunny this afternoon, high 51 (11 C). Low 31 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712452, "name": "FortWorthWX", "screen_name": "FortWorthWX", "lang": "en", "location": "Fort Worth, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Fort Worth, Texas", "followers_count": 218, "friends_count": 93, "statues_count": 8180 }, "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-04T10:00:09.000Z"), "id": 684071834111873026, "text": "@OmarKelly thoughts on Matt Patricia becoming our new HC?", "in_reply_to_status": -1, "in_reply_to_user": 26035722, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 26035722 }}, "user": { "id": 528731857, "name": "Justin Noel Blair", "screen_name": "JustinBlairr", "lang": "en", "location": " Miami|FIU|Sten", "create_at": date("2012-03-18"), "description": "The one who knocks. Just trying to get that cream and cheese", "followers_count": 1200, "friends_count": 1272, "statues_count": 6699 }, "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-04T10:00:09.000Z"), "id": 684071834212417536, "text": "Want to work at UMC? We're #hiring in #Lubbock, TX! Click for details: https://t.co/psqTodhVra #Nursing #cardiaccare #nurse #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-101.8919288,33.5881366"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Lubbock", "Nursing", "cardiaccare", "nurse", "Job", "Jobs" }}, "user": { "id": 2940007148, "name": "UMC Nursing Jobs", "screen_name": "UMCNursingJobs", "lang": "en", "location": "null", "create_at": date("2014-12-22"), "description": "UMC Health System is the leader in comprehensive healthcare delivery in West Texas and Eastern New Mexico. Follow us to learn about our Nursing Jobs!", "followers_count": 89, "friends_count": 278, "statues_count": 150 }, "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-04T10:00:09.000Z"), "id": 684071834392788992, "text": "@kul0tzrege89 palaaway", "in_reply_to_status": 684070973138575360, "in_reply_to_user": 541447972, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 541447972 }}, "user": { "id": 752244247, "name": "Kathleen Ong", "screen_name": "kathleennonnggg", "lang": "en", "location": "null", "create_at": date("2012-08-11"), "description": "Confidently beautiful through grace by faith", "followers_count": 1195, "friends_count": 675, "statues_count": 51880 }, "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-01-04T10:00:09.000Z"), "id": 684071834581622790, "text": "@AlenGawd https://t.co/UmfCc24NdM", "in_reply_to_status": -1, "in_reply_to_user": 1054365260, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1054365260 }}, "user": { "id": 435530024, "name": "Snoop Čuko", "screen_name": "S_Ruza", "lang": "en", "location": "Utica, NY", "create_at": date("2011-12-12"), "description": "DJ Ruža; For all and any questions call or text me at (315)534-8020; IG: S_RUZA", "followers_count": 435, "friends_count": 475, "statues_count": 18610 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Utica, NY", "id": "41f575b7eebcd4b7", "name": "Utica", "place_type": "city", "bounding_box": rectangle("-75.295291,43.050251 -75.158681,43.189108") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36065, "countyName": "Oneida", "cityID": 3676540, "cityName": "Utica" } }
+{ "create_at": datetime("2016-01-04T10:00:09.000Z"), "id": 684071834891890688, "text": "@jmarkwrite Call me real quick.", "in_reply_to_status": 684071590682730496, "in_reply_to_user": 260277053, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 260277053 }}, "user": { "id": 14319762, "name": "Tommy Cummings", "screen_name": "tommycummings", "lang": "en", "location": "Mansfield, TX", "create_at": date("2008-04-06"), "description": "Editor at @dallasnews\r\nhttp://facebook.com/tommycummings", "followers_count": 1882, "friends_count": 1726, "statues_count": 3220 }, "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-04T10:00:09.000Z"), "id": 684071834904621056, "text": "Chillinggggg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2961252923, "name": "A", "screen_name": "CaaramelFrappe_", "lang": "en", "location": "null", "create_at": date("2015-01-04"), "description": "Mack's Mom", "followers_count": 324, "friends_count": 237, "statues_count": 10438 }, "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-04T10:00:09.000Z"), "id": 684071834954903552, "text": "Kelis is evil bro.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25857272, "name": "Musa The Media Mogul", "screen_name": "MusaMajic", "lang": "en", "location": "Kemet", "create_at": date("2009-03-22"), "description": "Morgan State Alumnus #THEFUTUREOFMEDIA Click The Link in The Bio and SUBSCRIBE TO #MusaMajicTv #InTheDenWitMusaMajic MusaMajicTV@gmail.com", "followers_count": 1621, "friends_count": 1299, "statues_count": 45438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Olney, MD", "id": "7b117cdec2df9ae0", "name": "Olney", "place_type": "city", "bounding_box": rectangle("-77.119953,39.109016 -77.027505,39.185167") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2458900, "cityName": "Olney" } }
+{ "create_at": datetime("2016-01-04T10:00:09.000Z"), "id": 684071834975899648, "text": "@CoolinWith5H and if he is gay who cares as long as he does his job on sundays", "in_reply_to_status": 684071268195368961, "in_reply_to_user": 2742207604, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2742207604 }}, "user": { "id": 754480416, "name": "Andrew Scozzari", "screen_name": "andrewscozzari", "lang": "en", "location": "Shaolin", "create_at": date("2012-08-12"), "description": "I dont care", "followers_count": 1186, "friends_count": 1144, "statues_count": 88024 }, "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-04T10:00:09.000Z"), "id": 684071835051282433, "text": "❤️ https://t.co/xxgGshaS3g", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 744572700, "name": "Kaitlyn Fontaine", "screen_name": "kaitfontaine", "lang": "en", "location": "Hollister, CA", "create_at": date("2012-08-07"), "description": "But really, I have a caffine problem • ALPHAO", "followers_count": 466, "friends_count": 324, "statues_count": 9568 }, "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-04T10:00:09.000Z"), "id": 684071835336458240, "text": "This #Healthcare #job might be a great fit for you: Physical Therapist - Home Health -Primary Care Clinician... - https://t.co/fXV2v8ZakF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.6555013,36.6777372"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job" }}, "user": { "id": 718909165, "name": "Sutter Health Jobs", "screen_name": "SutterJobs", "lang": "en", "location": "Northern California", "create_at": date("2012-07-26"), "description": "Looking for a job in #healthcare? Join our team! Follow tweets for the latest job openings across Sutter Health. Get Sutter Health news at @SutterHealth.", "followers_count": 269, "friends_count": 21, "statues_count": 2870 }, "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-04T10:00:09.000Z"), "id": 684071835453919232, "text": "@catlover1943 @Syndeess These groups want a govt giveaway of public land 4them w/o compensation.It wud b welfare if not4 the guns.", "in_reply_to_status": 684065753595809792, "in_reply_to_user": 810145062, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 810145062, 1924395420 }}, "user": { "id": 2533783604, "name": "william a herrick", "screen_name": "herrick_a", "lang": "en", "location": "null", "create_at": date("2014-05-29"), "description": "I tweet, therefore I spam", "followers_count": 18946, "friends_count": 20782, "statues_count": 57879 }, "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-04T10:00:09.000Z"), "id": 684071835768520704, "text": "I have so much to do today & I'm still in Lafayette ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 289492733, "name": "Queen Victoria", "screen_name": "QueeennV_", "lang": "en", "location": "Louisiana", "create_at": date("2011-04-28"), "description": "#LSU ✨", "followers_count": 791, "friends_count": 527, "statues_count": 62501 }, "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-04T10:00:09.000Z"), "id": 684071836267622400, "text": "Wind 1.2 mph NNE. Barometer 30.419 in, Falling slowly. Temperature 54.3 °F. Rain today 0.00in. Humidity 50%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.11555556,29.6725"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25210383, "name": "New Braunfels WX", "screen_name": "jeffreycentexwx", "lang": "en", "location": "New Braunfels, TX", "create_at": date("2009-03-18"), "description": "Weather Station located on County Line Road in New Braunfels, TX.\r\n\r\nRealtime Weather @ http://www.nbtx-wx.com\r\nFollow me @ http://twitter.com/jeffreycentex", "followers_count": 379, "friends_count": 24, "statues_count": 159999 }, "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-01-04T10:00:09.000Z"), "id": 684071836443881473, "text": "https://t.co/wTyuUft04q https://t.co/B41Pal7cnu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 425683350, "name": "Scott Keyes", "screen_name": "Foot_Broadcast", "lang": "en", "location": "Utica, NY", "create_at": date("2011-12-01"), "description": "#malefeet #foot #escort #gay #cock #gayporn #socks #sneakers #shoes #toesucking #fetish http://t.co/hHnSu0FZB8 http://t.co/PwiWy9XHf7 http://t.co/nUUgUtb1z0", "followers_count": 2611, "friends_count": 615, "statues_count": 24400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Utica, NY", "id": "41f575b7eebcd4b7", "name": "Utica", "place_type": "city", "bounding_box": rectangle("-75.295291,43.050251 -75.158681,43.189108") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36065, "countyName": "Oneida", "cityID": 3676540, "cityName": "Utica" } }
+{ "create_at": datetime("2016-01-04T10:00:10.000Z"), "id": 684071836871557120, "text": "Sunny this afternoon, high 53 (12 C). Low 30 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712424, "name": "DallasWX", "screen_name": "DallasWX", "lang": "en", "location": "Dallas, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Dallas, Texas", "followers_count": 194, "friends_count": 93, "statues_count": 8157 }, "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-04T10:00:10.000Z"), "id": 684071836959649792, "text": "Add NXP to your #CES2016 must-see list: Next-gen tech for auto, personal entertainment, transport, and retail. #TheNewNXP...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CES2016", "TheNewNXP" }}, "user": { "id": 80629166, "name": "David Hoffman", "screen_name": "atxHoffman", "lang": "en", "location": "Austin, TX", "create_at": date("2009-10-07"), "description": "null", "followers_count": 174, "friends_count": 483, "statues_count": 30 }, "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-04T10:00:10.000Z"), "id": 684071837127421952, "text": "happy late birthday @nickrau24 !!!! you're cool and I hope you had an amazing day!!!!������ https://t.co/9FcpaGpuqY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2687988910 }}, "user": { "id": 2641330161, "name": "jackie", "screen_name": "jackieganem1", "lang": "en", "location": "null", "create_at": date("2014-06-25"), "description": "null", "followers_count": 220, "friends_count": 199, "statues_count": 190 }, "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-04T10:00:10.000Z"), "id": 684071837190369280, "text": "Happy birthday to my brother, homie, and 1 of the most musically gifted dudes I know @TaylorThrash welcome to the Barry Bonds years", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 41981574 }}, "user": { "id": 109475249, "name": "Costa", "screen_name": "Costa_DelMar", "lang": "en", "location": "Athens, OH/Dallas, TX", "create_at": date("2010-01-28"), "description": "live extraordinary | φφκα | πKα | #topszn COUNTRY CLVB ⛳️ All I ever asked is keep it eight more than 92 with me, 100", "followers_count": 383, "friends_count": 664, "statues_count": 7602 }, "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-01-04T10:00:10.000Z"), "id": 684071837484081156, "text": "the high is 27°? I'll pass", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 220541704, "name": "Louise Belcher", "screen_name": "xKRAYneC", "lang": "en", "location": "whereisky? \\\\ rosecrans ave. ", "create_at": date("2010-11-27"), "description": "one of them i like art type girls... • '][' • Uzuri • IG/SC: Kray.Dot", "followers_count": 818, "friends_count": 434, "statues_count": 122559 }, "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-04T10:00:10.000Z"), "id": 684071837509238784, "text": "When you have to go all the way to Chicago to find someone who knows how to take a full body picture ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 409561701, "name": "Brys Ward", "screen_name": "180Ward", "lang": "en", "location": "Champaign, IL", "create_at": date("2011-11-10"), "description": "17 • Follower of Christ • Musician • CHS 2016", "followers_count": 185, "friends_count": 187, "statues_count": 1957 }, "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-04T10:00:10.000Z"), "id": 684071837588836352, "text": "quit trying. it's never gonna happen. move on.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2648758264, "name": "Jas", "screen_name": "jasminerhene", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2014-06-27"), "description": "he belongs with me ❤\nbhs softball #8 \n I dont have time for you .", "followers_count": 203, "friends_count": 394, "statues_count": 4944 }, "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-04T10:00:10.000Z"), "id": 684071837655908352, "text": "I wanna go down to the waterfront and see the snow because pretty BUT IT'S DIFFICULT TO GET DOWN THERE.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 133527909, "name": "cockles connoisseur", "screen_name": "MusicLoverAna", "lang": "en", "location": "Bellingham, WA", "create_at": date("2010-04-15"), "description": "21. WWU. Writer. I like that show about ghosts, bros, angels, and profound bonds. Livin' in the city of subdued excitement.", "followers_count": 963, "friends_count": 747, "statues_count": 60295 }, "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-04T10:00:10.000Z"), "id": 684071837815468032, "text": "@rolandsmartin @NegusWolfgangus homesteading on federal owned land", "in_reply_to_status": 684068798463369216, "in_reply_to_user": 16116288, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16116288, 2881448815 }}, "user": { "id": 15636638, "name": "credo", "screen_name": "dowdellresearch", "lang": "en", "location": "Upper Marlboro, Maryland", "create_at": date("2008-07-28"), "description": "career influencer, investigator, legal researcher and advisor to business and non profit start ups.", "followers_count": 114, "friends_count": 184, "statues_count": 2113 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marlton, MD", "id": "d53cc5fddbb123d5", "name": "Marlton", "place_type": "city", "bounding_box": rectangle("-76.810544,38.743365 -76.757251,38.785764") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2450775, "cityName": "Marlton" } }
+{ "create_at": datetime("2016-01-04T10:00:10.000Z"), "id": 684071837848985600, "text": "omg ���������� https://t.co/e5ugxtQNjF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 181571921, "name": "rïssÿ", "screen_name": "marisasaraceni", "lang": "en", "location": "ESU 17 // ΛΙ", "create_at": date("2010-08-22"), "description": "i don't wanna be just friends", "followers_count": 1522, "friends_count": 996, "statues_count": 31542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Washington, PA", "id": "d504222869876a38", "name": "Fort Washington", "place_type": "city", "bounding_box": rectangle("-75.219948,40.126154 -75.164127,40.156451") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42091, "countyName": "Montgomery", "cityID": 4226872, "cityName": "Fort Washington" } }
+{ "create_at": datetime("2016-01-04T10:00:10.000Z"), "id": 684071838142496768, "text": "Want to work in #Montclair, CA? View our latest opening: https://t.co/00eCKj8fhS #BusinessMgmt #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.6897776,34.0775104"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Montclair", "BusinessMgmt", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22909934, "name": "TMJ-CA-US Mgmt.", "screen_name": "tmj_ca_mgmt", "lang": "en", "location": "California", "create_at": date("2009-03-05"), "description": "Follow this account for geo-targeted Business/Mgmt. job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 403, "friends_count": 258, "statues_count": 987 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montclair, CA", "id": "3153d677b795e293", "name": "Montclair", "place_type": "city", "bounding_box": rectangle("-117.721042,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-01-04T10:00:10.000Z"), "id": 684071838163595266, "text": "Cause Id take you back in heartbeat, disragarding what you put me through", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1661317249, "name": "❁gennay❁", "screen_name": "gennylo_", "lang": "en", "location": "Blades, DE", "create_at": date("2013-08-10"), "description": "RIP; JMJ❤", "followers_count": 3757, "friends_count": 3177, "statues_count": 30702 }, "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-01-04T10:00:10.000Z"), "id": 684071838549446656, "text": "@KinielCat Jolteon (artist’s rendering) https://t.co/9Dt2Kot3Bx", "in_reply_to_status": 684071292245426176, "in_reply_to_user": 40182386, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 40182386 }}, "user": { "id": 15158098, "name": "ndoto", "screen_name": "ndoto", "lang": "en", "location": "Mid-Atlantica, USA", "create_at": date("2008-06-18"), "description": "Nerd. Weird cat-person. I love technology and video games. I am a work in progress. Icon by @charmwitch. 3DS: 4038-6001-4289; NNID: neondoto", "followers_count": 549, "friends_count": 873, "statues_count": 59266 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Bethesda, MD", "id": "d3c4d514e13810f3", "name": "North Bethesda", "place_type": "city", "bounding_box": rectangle("-77.155886,39.012634 -77.084619,39.074304") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2456337, "cityName": "North Bethesda" } }
+{ "create_at": datetime("2016-01-04T10:00:10.000Z"), "id": 684071838587072513, "text": "Yo, my little cousins bought me donuts this morning, and if that isn't true love then I don't know what is.��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 88140234, "name": "Lily Zavala", "screen_name": "LilyBelenZavala", "lang": "en", "location": "Arkansas, USA", "create_at": date("2009-11-06"), "description": "Jeremiah 29:11// University of Arkansas// California Native • Arkansas Grown", "followers_count": 197, "friends_count": 173, "statues_count": 6673 }, "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-04T10:00:10.000Z"), "id": 684071838935322624, "text": "@HotWheels_24 @lackluster30 @AmberButterVibe I hear you!", "in_reply_to_status": 684071527352958976, "in_reply_to_user": 867166148, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 867166148, 850331275, 2351918005 }}, "user": { "id": 34285639, "name": "afentra", "screen_name": "afentra", "lang": "en", "location": "null", "create_at": date("2009-04-22"), "description": "Lady Legasus, defeeter of evil, scissor kick expert and downward facing dong connoisseur! #ListenLonger", "followers_count": 28425, "friends_count": 2893, "statues_count": 20837 }, "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-04T10:00:10.000Z"), "id": 684071838951972864, "text": "@noellobo yet Chantal disregarded what Lizzie said and stayed with a man who was unfaithful for 3 years", "in_reply_to_status": -1, "in_reply_to_user": 104391579, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 104391579 }}, "user": { "id": 2694345463, "name": "kel nicole", "screen_name": "kellbellaa", "lang": "en", "location": "null", "create_at": date("2014-07-30"), "description": "null", "followers_count": 38, "friends_count": 91, "statues_count": 1567 }, "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-04T10:00:10.000Z"), "id": 684071839082147840, "text": "McDonald's dirty ah, I unwrap my shit n it's missing the bottom bun wdf https://t.co/Plb72iAa7V", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2228363757, "name": "B.O.S.", "screen_name": "BrMadmoney", "lang": "en", "location": "Lovejoy, GA", "create_at": date("2013-12-16"), "description": "just a young wild nigga ian never Gave af", "followers_count": 233, "friends_count": 210, "statues_count": 2644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lovejoy, GA", "id": "00efa1e414a5e833", "name": "Lovejoy", "place_type": "city", "bounding_box": rectangle("-84.345517,33.427401 -84.300108,33.454566") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13151, "countyName": "Henry", "cityID": 1347616, "cityName": "Lovejoy" } }
+{ "create_at": datetime("2016-01-04T10:00:10.000Z"), "id": 684071839405060096, "text": "St Paul, MN: The weather is boring. 18ºF and fair.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.1,44.95"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3195747619, "name": "Weather By Brian", "screen_name": "WeatherByBrian", "lang": "en", "location": "null", "create_at": date("2015-05-14"), "description": "Tweets the weather about wherever Brian last tweets.\n\nRun by @bman4789", "followers_count": 11, "friends_count": 1, "statues_count": 1602 }, "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.89076 -93.003514,44.992279") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27123, "countyName": "Ramsey", "cityID": 2758000, "cityName": "St. Paul" } }
+{ "create_at": datetime("2016-01-04T10:00:10.000Z"), "id": 684071839434432512, "text": "����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "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": 703, "friends_count": 396, "statues_count": 93091 }, "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-04T10:00:10.000Z"), "id": 684071839547588609, "text": "Don't think I ever broke a female heart besides Brittany but that was back in 3rd grade and I cuddles again in 4th grade so it don't count", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 249960654, "name": "D Smith of Dem Boy$", "screen_name": "__DSmith__", "lang": "en", "location": "Greedy G ", "create_at": date("2011-02-09"), "description": "Boppin' aint dead, niggas just scared. Commerce Tx", "followers_count": 4310, "friends_count": 837, "statues_count": 253611 }, "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-01-04T10:00:10.000Z"), "id": 684071839593705472, "text": "Sunny this afternoon, high 56 (13 C). Low 35 (2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712472, "name": "AustinWX", "screen_name": "AustinWX", "lang": "en", "location": "Austin, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Austin, Texas", "followers_count": 1010, "friends_count": 1366, "statues_count": 8018 }, "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-04T10:00:10.000Z"), "id": 684071840252207104, "text": "I hope we're friends until we die. \n\nand then I hope we become ghost… https://t.co/i1O382e8Nl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.74072073,35.31664537"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 425783999, "name": "Brandi Kiger", "screen_name": "iBrandiKiger", "lang": "en", "location": "UNCC | ΚΔ", "create_at": date("2011-12-01"), "description": "waaayyyyy up I feel #blest", "followers_count": 706, "friends_count": 427, "statues_count": 18272 }, "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-04T10:00:10.000Z"), "id": 684071840461897728, "text": "which type of medley is most efficient?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1051852860, "name": "eternal makoto stan", "screen_name": "softmakoto", "lang": "en", "location": "Makoharu hell", "create_at": date("2012-12-31"), "description": "Always ready to talk about makoto tachibana", "followers_count": 119, "friends_count": 350, "statues_count": 4806 }, "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-01-04T10:00:10.000Z"), "id": 684071840768225281, "text": "@JosueUrrutiaDC Rev5:5 Urgent War Cry!! 4 My People Isreal Spalms 122:6 plz This Is My Hour To Live Or Die! if IAM https://t.co/a2KdkVaDkT", "in_reply_to_status": -1, "in_reply_to_user": 820918393, "favorite_count": 0, "coordinate": point("-118.2115057,34.0494007"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 820918393 }}, "user": { "id": 3163001474, "name": "ABBAJezsusJackson777", "screen_name": "JezsusJ", "lang": "en", "location": "Califas ", "create_at": date("2015-04-18"), "description": "null", "followers_count": 672, "friends_count": 1998, "statues_count": 6777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:10.000Z"), "id": 684071840805982208, "text": "Haven't seen sam 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": 572152659, "name": "Lex ♑️", "screen_name": "alexamark15", "lang": "en", "location": "null", "create_at": date("2012-05-05"), "description": "R. ❤️ 8/23", "followers_count": 485, "friends_count": 284, "statues_count": 6613 }, "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-04T10:00:10.000Z"), "id": 684071840822603776, "text": "/Josh Gordon gets suspended https://t.co/m3SbBsEPMA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "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": 1200, "friends_count": 1852, "statues_count": 186059 }, "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-04T10:00:10.000Z"), "id": 684071840910802944, "text": "Happiest day o'birth @samcasto_88", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 153507531 }}, "user": { "id": 41602558, "name": "Crystal Poe", "screen_name": "PoeInDiscord", "lang": "en", "location": "BelingtonWV", "create_at": date("2009-05-21"), "description": "Converse, bright sunglasses, nerds, and the obscure.", "followers_count": 348, "friends_count": 279, "statues_count": 11814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairmont, WV", "id": "0006c18b448ad428", "name": "Fairmont", "place_type": "city", "bounding_box": rectangle("-80.188998,39.449914 -80.104137,39.506395") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54049, "countyName": "Marion", "cityID": 5426452, "cityName": "Fairmont" } }
+{ "create_at": datetime("2016-01-04T10:00:11.000Z"), "id": 684071841279963137, "text": "�� they are jamming https://t.co/smS5oPQ49m", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 114759585, "name": "Morris Williams", "screen_name": "Morris__will", "lang": "en", "location": "Somewhere next to nowhere", "create_at": date("2010-02-16"), "description": "I am Morris Williams Lover of music, art,Fashion and poetry and overall smart ass! enjoy!", "followers_count": 320, "friends_count": 620, "statues_count": 7125 }, "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-01-04T10:00:11.000Z"), "id": 684071841619689472, "text": "���� https://t.co/SZ4Z9n3xaX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 227853866, "name": "G Ro ‼️", "screen_name": "therealromyo", "lang": "en", "location": "Indianapolis, IN", "create_at": date("2010-12-17"), "description": "Cappin", "followers_count": 1081, "friends_count": 506, "statues_count": 32086 }, "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-01-04T10:00:11.000Z"), "id": 684071841942474752, "text": "@kquinn04 copy cat", "in_reply_to_status": 684070706196398080, "in_reply_to_user": 470751394, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 470751394 }}, "user": { "id": 385161196, "name": "TOXIC", "screen_name": "just_clint_c", "lang": "en", "location": "glassboro, NJ", "create_at": date("2011-10-04"), "description": "ΣΠ ΖΧ \nClinton Conyer=DJ TOX!C. \nDeptford Township Fireman\nconyerc0@students.rowan.edu https://soundcloud.com/clintonconyer Rowan's Favorite DJ", "followers_count": 481, "friends_count": 880, "statues_count": 12770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodbury, NJ", "id": "8a6609809fa46edc", "name": "Woodbury", "place_type": "city", "bounding_box": rectangle("-75.171276,39.82399 -75.135111,39.855023") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34015, "countyName": "Gloucester", "cityID": 3482120, "cityName": "Woodbury" } }
+{ "create_at": datetime("2016-01-04T10:00:11.000Z"), "id": 684071842102030336, "text": "I'm excited though bc I love 20's hair and makeup. I just need to practice bc I've never tried doing the hairstyles", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18546249, "name": "Jesse James", "screen_name": "witchhplease", "lang": "en", "location": "Tampa,Florida", "create_at": date("2009-01-01"), "description": "Crawling on the planet's face, some insects, called the human race. Lost in time, and lost in space...and meaning.", "followers_count": 439, "friends_count": 268, "statues_count": 12968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Egypt Lake-Leto, FL", "id": "6d4de57fa00a913f", "name": "Egypt Lake-Leto", "place_type": "city", "bounding_box": rectangle("-82.527472,27.996725 -82.485096,28.037298") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1220108, "cityName": "Egypt Lake-Leto" } }
+{ "create_at": datetime("2016-01-04T10:00:11.000Z"), "id": 684071842328522754, "text": "@IsraelNewsNow YaHoo!", "in_reply_to_status": 684070682930495488, "in_reply_to_user": 30823468, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 30823468 }}, "user": { "id": 4454815234, "name": "virginia woodard", "screen_name": "VirginiaWoodard", "lang": "en", "location": "North Carolina, USA", "create_at": date("2015-12-11"), "description": "mother- grandmother-retired RN", "followers_count": 292, "friends_count": 476, "statues_count": 3193 }, "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-04T10:00:11.000Z"), "id": 684071842399784960, "text": "I kno she ain't delete that msg ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1051946504, "name": "J⭕️✨", "screen_name": "juustjo", "lang": "en", "location": "sinning", "create_at": date("2012-12-31"), "description": "Nayah..❤️", "followers_count": 1936, "friends_count": 823, "statues_count": 29634 }, "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-04T10:00:11.000Z"), "id": 684071842475151360, "text": "Sunny this afternoon, high 60 (16 C). Low 36 (2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712388, "name": "HoustonWX", "screen_name": "HoustonWX", "lang": "en", "location": "Houston, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Houston, Texas", "followers_count": 1817, "friends_count": 77, "statues_count": 8027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-04T10:00:11.000Z"), "id": 684071842747789313, "text": "@Viralscape LOW RENT FOOD ESTABLISHMENTS HAVE THE MOST VIOLENT CRIMES! THAT'S WHAT THE NEWS SAYS!", "in_reply_to_status": 667565543826714625, "in_reply_to_user": 116525488, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 116525488 }}, "user": { "id": 891150589, "name": "Don Dvil", "screen_name": "DonDvil1", "lang": "en", "location": "null", "create_at": date("2012-10-19"), "description": "Multi Media Artist,\r\nDJ /Producer ,\r\nSneaker Collector,\r\nEducator,\r\nMusic Collector,\r\nPhilosopher. \r\nI am these things, for the benefit of the planet.", "followers_count": 509, "friends_count": 476, "statues_count": 25078 }, "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-04T10:00:11.000Z"), "id": 684071842827636741, "text": "I'm 23 & single .... I refuse to be told what to do? Nah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 198647699, "name": "Aunt Koko", "screen_name": "LittleKrittt", "lang": "en", "location": "null", "create_at": date("2010-10-04"), "description": "Drink Henny, Dance Naked & Laugh EhhyDay.❤️", "followers_count": 378, "friends_count": 637, "statues_count": 17889 }, "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-04T10:00:11.000Z"), "id": 684071842890420224, "text": "@aaliyah_t_pride lemme borrow him for a couple years.....", "in_reply_to_status": 684071636627144704, "in_reply_to_user": 440565234, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 440565234 }}, "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", "followers_count": 795, "friends_count": 357, "statues_count": 39775 }, "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-04T10:00:11.000Z"), "id": 684071842986995712, "text": "Taken https://t.co/wJmYtGiEiW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2411388691, "name": "Jamaya Dandridge", "screen_name": "Mayad__", "lang": "en", "location": "null", "create_at": date("2014-03-25"), "description": "IG\\\\ @mayad__ ✨", "followers_count": 768, "friends_count": 278, "statues_count": 8733 }, "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-04T10:00:11.000Z"), "id": 684071843578314752, "text": "We're #hiring! Read about our latest #job opening here: Coding Technician - Coding and Abstracting - https://t.co/p5nM4rIwKp #Seattle, WA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.33,47.61"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Seattle" }}, "user": { "id": 117387498, "name": "Swedish Careers", "screen_name": "SwedishCareers", "lang": "en", "location": "Seattle, Washington, USA", "create_at": date("2010-02-25"), "description": "Where you work matters. \r\nFor 100 years, employees at Swedish have provided the area's best and most comprehensive health care.", "followers_count": 421, "friends_count": 260, "statues_count": 1280 }, "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-04T10:00:11.000Z"), "id": 684071843590885377, "text": "Can you recommend anyone for this #job? Tumor Immunology Scientist - https://t.co/Nm25o3aQwW #Rockville, MD #Physician #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.1527578,39.0839973"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Rockville", "Physician", "Hiring", "CareerArc" }}, "user": { "id": 71179284, "name": "DC Physician Jobs", "screen_name": "tmj_dc_physici", "lang": "en", "location": "DC Metro", "create_at": date("2009-09-02"), "description": "Follow this account for geo-targeted Physician job tweets in DC Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 110, "friends_count": 97, "statues_count": 44 }, "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-01-04T10:00:11.000Z"), "id": 684071843683123200, "text": "Sounds like Helix Lmao https://t.co/m07AMugMjg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 548141446, "name": "Titties ⚫️⛽️", "screen_name": "ThirstMcGurst", "lang": "en", "location": "null", "create_at": date("2012-04-07"), "description": "Our generation consists of PartyBuses & Jordans. Otherwise known as PB&J's. but chill w/ the sandwich jokes. This is twitter so no need for subs.", "followers_count": 2057, "friends_count": 203, "statues_count": 152763 }, "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-04T10:00:11.000Z"), "id": 684071843813146624, "text": "ResourceMFG #IT #Job: IT Network Administrator (#Richmond, KY) https://t.co/qvnt3YucpJ #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.2946539,37.7478572"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "IT", "Job", "Richmond", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 99868456, "name": "TMJ-KYL IT Adm. Jobs", "screen_name": "tmj_KYL_adm", "lang": "en", "location": "Lexington, KY", "create_at": date("2009-12-27"), "description": "Follow this account for geo-targeted IT-Support/Administration job tweets in Lexington, KY from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 273, "friends_count": 265, "statues_count": 12 }, "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-04T10:00:11.000Z"), "id": 684071844576530432, "text": "Stop waiting to be heard! #music #playmusic #newyear #newyearresolution #performance… https://t.co/oMnm7bjysd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.4942169,29.4291306"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "music", "playmusic", "newyear", "newyearresolution", "performance" }}, "user": { "id": 179674759, "name": "Alamo Music Center", "screen_name": "alamomusic", "lang": "en", "location": "San Antonio", "create_at": date("2010-08-17"), "description": "Official Twitter account for San Antonio's own Alamo Music Center, family owned since 1929. We sell, rent, and repair music instruments as well as give lessons.", "followers_count": 1995, "friends_count": 978, "statues_count": 4927 }, "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-04T10:00:11.000Z"), "id": 684071844903690241, "text": "OU No. 1 in Coaches Poll, with Kansas right behind at No. 2 #kubball https://t.co/BcffxIdpIm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "kubball" }}, "user": { "id": 35817951, "name": "JayhawkSlant", "screen_name": "JayhawkSlant", "lang": "en", "location": "Jayhawk, Kansas", "create_at": date("2009-04-27"), "description": "Jon Kirby, Shay Wildeboor, and Bryan Cisler roll out the Jayhawks team & recruiting scoop for http://Rivals.com/Yahoo Sports. #Kubball, #kufball, #Jayhawks, #KU", "followers_count": 22285, "friends_count": 1380, "statues_count": 25302 }, "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-01-04T10:00:12.000Z"), "id": 684071845264527361, "text": "-____- why do these bitches that lurk me like to say i cant stop talking about em on twitter omfg this is like the second time different hoe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1532075162, "name": "onerrrr", "screen_name": "_janinee__", "lang": "en", "location": "null", "create_at": date("2013-06-19"), "description": "null", "followers_count": 1555, "friends_count": 981, "statues_count": 68893 }, "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-04T10:00:12.000Z"), "id": 684071845276995584, "text": "Sunny this afternoon, high 51 (11 C). Low 30 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575081, "name": "San Angelo Weather", "screen_name": "SanAngeloWX", "lang": "en", "location": "San Angelo, TX", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for San Angelo, TX", "followers_count": 1943, "friends_count": 92, "statues_count": 7917 }, "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-04T10:00:12.000Z"), "id": 684071845637664769, "text": "@sirryanspoopy @Ieansquad ��������", "in_reply_to_status": 684071503986569216, "in_reply_to_user": 2229229112, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2229229112, 2496762631 }}, "user": { "id": 302725085, "name": "Doctor Cucumber.", "screen_name": "PSnotBS", "lang": "en", "location": "Niagara Falls New York", "create_at": date("2011-05-21"), "description": "Life Is A Funeral Party ⚰. Rest Easy @JiggSavage_ . Still A #SadBoy . @PlayDoughSoft Is My One And Only.", "followers_count": 932, "friends_count": 503, "statues_count": 72100 }, "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-04T10:00:12.000Z"), "id": 684071845688025088, "text": "今日のてか5日の0時10分の便で帰ります", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user": { "id": 2916399188, "name": "s e i g o", "screen_name": "seigo_orange", "lang": "ja", "location": "Foster City, CA", "create_at": date("2014-12-02"), "description": "TCU H1 15 | USA × Japan | TGS*新井ひとみ |Foster City,CA 12/27〜1/7 リア→@seigo04005399 Мари́на Андре́евна Алекса́ндрова", "followers_count": 384, "friends_count": 301, "statues_count": 14321 }, "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-01-04T10:00:12.000Z"), "id": 684071845923012608, "text": "Trust issues level 1,000 = joint Facebook account.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 183300272, "name": "Chris Jackson", "screen_name": "CJackson2013", "lang": "en", "location": "Richmond, KY", "create_at": date("2010-08-26"), "description": "Junior. #EKU Fire, Arson&Explosion Investigation major. FireFighter #EKU17", "followers_count": 692, "friends_count": 528, "statues_count": 5947 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Barbourville, KY", "id": "00b21cbb04a03744", "name": "Barbourville", "place_type": "city", "bounding_box": rectangle("-83.905409,36.842365 -83.818361,36.886082") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21121, "countyName": "Knox", "cityID": 2103574, "cityName": "Barbourville" } }
+{ "create_at": datetime("2016-01-04T10:00:12.000Z"), "id": 684071846031953921, "text": "Best Spots For Live Poetry https://t.co/nhZQbXX3t0 by me, for @VAM_STUDIO !! feat @SandboxOpenMic @YoungChiAuthors @thegalachicago & more!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3002433346, 3604993768, 113464594, 393550192 }}, "user": { "id": 24387971, "name": "+", "screen_name": "plussable", "lang": "en", "location": "chicago", "create_at": date("2009-03-14"), "description": "+ aka Plus Sign. God MC, Pop Star! utopia, a collection of essays now available! coms@tenderdiscovery.com", "followers_count": 871, "friends_count": 868, "statues_count": 50557 }, "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-04T10:00:12.000Z"), "id": 684071846229225473, "text": "@kyithediamond nothing to do with you so don't worry about it bra", "in_reply_to_status": 684071593476243456, "in_reply_to_user": 66003042, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 66003042 }}, "user": { "id": 312251833, "name": "RJfrmDaBULL", "screen_name": "whyNOt_saluteME", "lang": "en", "location": "null", "create_at": date("2011-06-06"), "description": "#statechampion #ECSU19", "followers_count": 890, "friends_count": 869, "statues_count": 13077 }, "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-04T10:00:12.000Z"), "id": 684071846401163264, "text": "I'm at Longwood Historic District in Longwood, FL https://t.co/7ny16EDQHd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.35024071,28.70117221"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35135168, "name": "✨", "screen_name": "DizzyMissRuby", "lang": "en", "location": "Hogwarts", "create_at": date("2009-04-24"), "description": "Wit beyond measure is woman's greatest treasure", "followers_count": 582, "friends_count": 187, "statues_count": 59980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Longwood, FL", "id": "f3c295d74e525431", "name": "Longwood", "place_type": "city", "bounding_box": rectangle("-81.383289,28.676878 -81.31933,28.721956") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12117, "countyName": "Seminole", "cityID": 1241250, "cityName": "Longwood" } }
+{ "create_at": datetime("2016-01-04T10:00:12.000Z"), "id": 684071846489149440, "text": "Goodwill Industries Of Mid- East... #Job: Floor Technician (#Allentown, PA) https://t.co/SZMNyEWWbR #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.4901833,40.6084305"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Job", "Allentown", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 41686294, "name": "PA Non-Metro Jobs", "screen_name": "tmj_pa_usa_jobs", "lang": "en", "location": "Pennsylvania", "create_at": date("2009-05-21"), "description": "Follow this account for geo-targeted Other job tweets in Pennsylvania Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 828, "friends_count": 540, "statues_count": 603 }, "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-04T10:00:12.000Z"), "id": 684071846501810176, "text": "But how I win the league I haven't even touched since like week 9 ... y'all suck @DKMR_ @Ace_2K https://t.co/wI4uLHCf0N", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 296963187, 2897858598 }}, "user": { "id": 349170153, "name": "The Wolf", "screen_name": "CGreen_Comik", "lang": "en", "location": "Greenville, PA", "create_at": date("2011-08-05"), "description": "One day all of this wont matter #TeamTannehill", "followers_count": 222, "friends_count": 315, "statues_count": 14607 }, "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-04T10:00:12.000Z"), "id": 684071846514311168, "text": "#SkilledTrade #Job alert: Field Repair- Holiday Centre | Jared | #Monroeville, PA https://t.co/k4NBedtgyx #Jeweler #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.7881024,40.4211798"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SkilledTrade", "Job", "Monroeville", "Jeweler", "Jobs", "Hiring" }}, "user": { "id": 3248848340, "name": "Jobs at Jared", "screen_name": "JobsAtJared", "lang": "en", "location": "null", "create_at": date("2015-06-18"), "description": "Your Future Starts Here.", "followers_count": 13, "friends_count": 1, "statues_count": 504 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Municipality of Monroeville, PA", "id": "6f8088a9a420904f", "name": "Municipality of Monroeville", "place_type": "city", "bounding_box": rectangle("-79.819677,40.385071 -79.703829,40.46208") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4250528, "cityName": "Monroeville" } }
+{ "create_at": datetime("2016-01-04T10:00:12.000Z"), "id": 684071846623449088, "text": "If a female was actually pregnant, but decided to keep it hidden from you, what would ya do, fellas?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 153998008, "name": "Lioness", "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": 1557, "friends_count": 1342, "statues_count": 176469 }, "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-04T10:00:12.000Z"), "id": 684071846635962368, "text": "You look thirsty������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 918141199, "name": "ash", "screen_name": "ashleysmitth_", "lang": "en", "location": "TX", "create_at": date("2012-10-31"), "description": "{He shall give you strength for every battle & wisdom for every decision} #SicEm @asneezy_pt", "followers_count": 1114, "friends_count": 513, "statues_count": 33079 }, "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-01-04T10:00:12.000Z"), "id": 684071846677843970, "text": "#SouthZanesville, OH #Nursing #Job: Hiring Post-anesthesia care unit RN's at Supplemental Health Care https://t.co/uZ9MGKodaz #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.02,39.9025"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SouthZanesville", "Nursing", "Job", "Jobs", "Hiring" }}, "user": { "id": 66473306, "name": "TMJ-OH Nursing Jobs", "screen_name": "tmj_oh_nursing", "lang": "en", "location": "Ohio", "create_at": date("2009-08-17"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Ohio Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 345, "friends_count": 295, "statues_count": 438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Zanesville, OH", "id": "01b907221e856414", "name": "South Zanesville", "place_type": "city", "bounding_box": rectangle("-82.069852,39.864371 -81.979134,39.91616") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39119, "countyName": "Muskingum", "cityID": 3973894, "cityName": "South Zanesville" } }
+{ "create_at": datetime("2016-01-04T10:00:12.000Z"), "id": 684071846858260480, "text": "I think nothing short of @JedYork committing seppuku live on air will satisfy #49ers fans https://t.co/gJSJImyXY1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "49ers" }}, "user_mentions": {{ 388472830 }}, "user": { "id": 29567331, "name": "Brenaldo", "screen_name": "SaintBrendan", "lang": "en", "location": "Bay Area California", "create_at": date("2009-04-07"), "description": "Celebrated indoor soccer star in California. Co-host of @WCAmateurHour podcast. Boston Sports & #SJSharks Authority. Written on @KFCradio @BarstoolNewYork. #LFC", "followers_count": 1505, "friends_count": 1074, "statues_count": 62667 }, "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-04T10:00:12.000Z"), "id": 684071847051292672, "text": "missin my friends already :/ https://t.co/zZYPourmpu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 255808407, "name": "ken ☼", "screen_name": "k_fornash", "lang": "en", "location": "null", "create_at": date("2011-02-21"), "description": "@brucybby❣", "followers_count": 1010, "friends_count": 166, "statues_count": 98345 }, "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-01-04T10:00:12.000Z"), "id": 684071847256809472, "text": "@SeaWorldLatino gracias!", "in_reply_to_status": 684071385753370624, "in_reply_to_user": 372287547, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 372287547 }}, "user": { "id": 2232812991, "name": "rickie", "screen_name": "__rickie_", "lang": "en", "location": "Puerto Rico, FL ", "create_at": date("2013-12-18"), "description": "DE ISLA PEQUEÑA PERO CON UN CORAZON GRANDE || Ormond Beach FL || Seabreeze || swimming ||", "followers_count": 2872, "friends_count": 469, "statues_count": 36973 }, "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-01-04T10:00:12.000Z"), "id": 684071847357333505, "text": "I'm disappointed that I've smoked blunts for the past two days. My father wouldn't like that. Ima get some raw cones today.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1953309102, "name": "CAM", "screen_name": "hunter_fierro", "lang": "en", "location": "•Pittsburg Tx•", "create_at": date("2013-10-10"), "description": "Sc: hunterfierro7 | Kansas City ⚪️ | #TaylorGang |", "followers_count": 487, "friends_count": 375, "statues_count": 8722 }, "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-04T10:00:12.000Z"), "id": 684071847424462848, "text": "My cutie pie https://t.co/W4QHTxByN1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3012653342, "name": "Nathan Ly", "screen_name": "WuTangMatitii", "lang": "en", "location": "null", "create_at": date("2015-02-07"), "description": "Go with the flow type of guy", "followers_count": 188, "friends_count": 206, "statues_count": 1961 }, "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-04T10:00:12.000Z"), "id": 684071847504130048, "text": "They say bloods thicker then water? I know water that will give there last and I know blood that don't even speak.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 578495674, "name": "riani.", "screen_name": "princessani__", "lang": "en", "location": "null", "create_at": date("2012-05-12"), "description": "Anthony Thomas ❤", "followers_count": 1783, "friends_count": 1209, "statues_count": 83286 }, "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-04T10:00:12.000Z"), "id": 684071848183599104, "text": "These vows (turn on your sound) �� We can't get enough of this gorgeous and touching Wilmington… https://t.co/Kd6FiQHa4R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.8354874,34.2105408"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 295359718, "name": "Blueberry Creative", "screen_name": "BluebryCreative", "lang": "en", "location": "North Carolina", "create_at": date("2011-05-08"), "description": "Freshly Picked Photography & Films", "followers_count": 645, "friends_count": 562, "statues_count": 845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seagate, NC", "id": "122cafc5c5677421", "name": "Seagate", "place_type": "city", "bounding_box": rectangle("-77.868466,34.18259 -77.8232,34.220536") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37129, "countyName": "New Hanover", "cityID": 3774440, "cityName": "Wilmington" } }
+{ "create_at": datetime("2016-01-04T10:00:12.000Z"), "id": 684071848330588160, "text": "Need to get a job 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": 346253106, "name": "Johntay Coleman", "screen_name": "DeosKeeper", "lang": "en", "location": "Chillin in my thoughts ", "create_at": date("2011-07-31"), "description": "#YoungKing |SlobBoyz| ♐️ #CowboysNation", "followers_count": 501, "friends_count": 577, "statues_count": 35554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Trotwood, OH", "id": "43f1e6b6e43df0ea", "name": "Trotwood", "place_type": "city", "bounding_box": rectangle("-84.330503,39.74629 -84.25346,39.833568") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39113, "countyName": "Montgomery", "cityID": 3977504, "cityName": "Trotwood" } }
+{ "create_at": datetime("2016-01-04T10:00:12.000Z"), "id": 684071848363966464, "text": "Sunny this afternoon, high 56 (13 C). Low 31 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575147, "name": "Baton Rouge Weather", "screen_name": "BatonRougeWX", "lang": "en", "location": "Baton Rouge, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Baton Rouge, LA", "followers_count": 2872, "friends_count": 92, "statues_count": 7870 }, "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-04T10:00:12.000Z"), "id": 684071848548511744, "text": "https://t.co/5xZziwVvnd @HelenWisner", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 241426786 }}, "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": 489, "friends_count": 2109, "statues_count": 71036 }, "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-04T10:00:12.000Z"), "id": 684071848804388865, "text": "#Sales #Job alert: Clearance Center Sales Associate | Bob's Discount Furniture | #Worcester, MA https://t.co/oJ0hVJIChf #Veterans #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.7769289,42.2956971"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "Job", "Worcester", "Veterans", "Jobs" }}, "user": { "id": 22038280, "name": "TMJ-MA-US Sales Jobs", "screen_name": "tmj_MA_sales", "lang": "en", "location": "Massachusetts", "create_at": date("2009-02-26"), "description": "Follow this account for geo-targeted Sales job tweets in Massachusetts Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 411, "friends_count": 308, "statues_count": 420 }, "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-04T10:00:12.000Z"), "id": 684071849190232064, "text": "@IMWHYJAE is probably to coolest rapper on this planet right about now. To be honest with you. He replies to you and shows you love!!", "in_reply_to_status": -1, "in_reply_to_user": 208101829, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 208101829 }}, "user": { "id": 2931487104, "name": "G", "screen_name": "JiggieThrees", "lang": "en", "location": "Scrills TX", "create_at": date("2014-12-15"), "description": "null", "followers_count": 261, "friends_count": 175, "statues_count": 9091 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, TX", "id": "723d666e3a15fd22", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-97.362644,27.850198 -97.291318,27.908126") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48409, "countyName": "San Patricio", "cityID": 4858904, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-04T10:00:12.000Z"), "id": 684071849198784513, "text": "Izzo, asked if he's somewhat relieved to not be No. 1 anymore: \"To hell with being the underdog. No, I'm really not.\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 390982515, "name": "Joe Rexrode", "screen_name": "joerexrode", "lang": "en", "location": "Okemos, Mich.", "create_at": date("2011-10-14"), "description": "Michigan State football and men's basketball beat writer for the Detroit Free Press, Monday co-host of The Drive with Jack Ebling, 3-6 p.m. on 92.1-FM Lansing.", "followers_count": 24437, "friends_count": 1262, "statues_count": 56777 }, "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-04T10:00:12.000Z"), "id": 684071849261543424, "text": "Struggling with the fish face? https://t.co/4xEb6q31Eg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 412740778, "name": "Bizzy Hendrix", "screen_name": "BizzyNextDoor", "lang": "en", "location": "CLE", "create_at": date("2011-11-14"), "description": "Alhamdillah", "followers_count": 586, "friends_count": 307, "statues_count": 61298 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westlake, OH", "id": "abcede5a546a90a3", "name": "Westlake", "place_type": "city", "bounding_box": rectangle("-81.970255,41.41104 -81.872904,41.478131") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3983622, "cityName": "Westlake" } }
+{ "create_at": datetime("2016-01-04T10:00:13.000Z"), "id": 684071849723052032, "text": "I don't know what to do with myself now that 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": 431934436, "name": "Collin Kelly", "screen_name": "PB_and_Kelly212", "lang": "en", "location": "Philly ✈️ Aston", "create_at": date("2011-12-08"), "description": "| Happiness is when what you think, what you say, and what you do are in harmony. | Aunt Bonnie♥️", "followers_count": 603, "friends_count": 574, "statues_count": 15354 }, "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-04T10:00:13.000Z"), "id": 684071849920196608, "text": "Last season was probably my favorite season on teen wolf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 323680656, "name": "αℓℓєχχα", "screen_name": "so_alleexxaa", "lang": "en", "location": "Bowling Green, KY", "create_at": date("2011-06-25"), "description": "@justinbieber WKU '19 ❤️", "followers_count": 1109, "friends_count": 935, "statues_count": 33984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Vergne, TN", "id": "5b30c2de42727ec7", "name": "La Vergne", "place_type": "city", "bounding_box": rectangle("-86.615589,35.965278 -86.506446,36.053789") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47149, "countyName": "Rutherford", "cityID": 4741200, "cityName": "La Vergne" } }
+{ "create_at": datetime("2016-01-04T10:00:13.000Z"), "id": 684071850075262976, "text": "♠️��♠️��♠️Who knew dinner could be so much fun... #oysters #bachelor #LIV #roundthree… https://t.co/7j6CPD3l5Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.1332397,25.7699108"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "oysters", "bachelor", "LIV", "roundthree" }}, "user": { "id": 43806174, "name": "Samantha Brittany", "screen_name": "Saammmyyy", "lang": "en", "location": "Miami, FL", "create_at": date("2009-05-31"), "description": "null", "followers_count": 120, "friends_count": 116, "statues_count": 487 }, "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-04T10:00:13.000Z"), "id": 684071850226233344, "text": "JonNai has this thing now that she has to come in the bathroom with me no matter what ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 636453056, "name": "Elisia Romero", "screen_name": "elisia_10", "lang": "en", "location": "San Tan Valley, AZ", "create_at": date("2012-07-15"), "description": "Nineteen * College student", "followers_count": 548, "friends_count": 415, "statues_count": 38447 }, "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-01-04T10:00:13.000Z"), "id": 684071850331119617, "text": "�������� https://t.co/JRapX7F9Zz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1595079529, "name": "Louℹe", "screen_name": "M3xicAnGeek", "lang": "en", "location": "Rialto, CA", "create_at": date("2013-07-14"), "description": "burdens.\npr♻p. 215. your future self is watching you right now through your memories\n IG:@looneylouie", "followers_count": 445, "friends_count": 387, "statues_count": 5672 }, "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-04T10:00:13.000Z"), "id": 684071850595332096, "text": ".@TheView @candacecbure @WhoopiGoldberg @ravensymone @JoyVBehar @paulafaris @michcoll Welcome back - I missed you and #TheView!", "in_reply_to_status": 683992061402550273, "in_reply_to_user": 21258337, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TheView" }}, "user_mentions": {{ 21258337, 33694840, 284602545, 45713517, 32004130, 96848782, 43909715 }}, "user": { "id": 711610075, "name": "Ms AJ Rogers", "screen_name": "MsAJRogers", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-07-22"), "description": "I work in #SoMe for #Design. I hang with #ATLMkt, #HPMkt, #LVMkt, #TheView and #SharkTank. I like to think outside the box. I am finding inspiration everywhere.", "followers_count": 2389, "friends_count": 1812, "statues_count": 12414 }, "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-04T10:00:13.000Z"), "id": 684071850687631360, "text": "Join the NASCAR team! See our latest #Marketing #job opening here: https://t.co/LhzmDdzFja #QA #Agile #Charlotte, NC #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.8431267,35.2270869"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Marketing", "job", "QA", "Agile", "Charlotte", "Hiring" }}, "user": { "id": 2935068602, "name": "NASCAR Jobs", "screen_name": "NASCARJobs", "lang": "en", "location": "null", "create_at": date("2014-12-18"), "description": "The National Association for Stock Car Auto Racing, Inc. (NASCAR) is the sanctioning body for one of North America's premier sports.", "followers_count": 1105, "friends_count": 14, "statues_count": 160 }, "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-04T10:00:13.000Z"), "id": 684071850758946816, "text": "Probably not going to graduate with a black robe ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 261364421, "name": "Catholic Buddha", "screen_name": "roni_untivero", "lang": "en", "location": "316 in a 405", "create_at": date("2011-03-05"), "description": "Not many people like me, so i must learn to love myself. missing you. Intersectional feminist", "followers_count": 531, "friends_count": 495, "statues_count": 27296 }, "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-04T10:00:13.000Z"), "id": 684071850821824512, "text": "I'm still amazed that I got pulled over at least 9 times within 18 months and still have a license", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16963242, "name": "☹♡☹♡☹", "screen_name": "iNeedAhairbrush", "lang": "en", "location": "Austin, Tx", "create_at": date("2008-10-24"), "description": "I don't need you to tell me how fucking good my coffee is, okay? I'm the one who buys it. I know how good it is.", "followers_count": 256, "friends_count": 72, "statues_count": 17565 }, "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-04T10:00:13.000Z"), "id": 684071850851303427, "text": "@teerasheaa #fb", "in_reply_to_status": -1, "in_reply_to_user": 3316454259, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "fb" }}, "user_mentions": {{ 3316454259 }}, "user": { "id": 379517365, "name": "Glopa ☀️", "screen_name": "NdShe_Sqwallow", "lang": "en", "location": "Haughville, USA", "create_at": date("2011-09-24"), "description": "They call me Uncle Grandpa...Sex.Money.Murder .InkOverHoes......Yo bitch wanna fuck the squad..FREEPEANUT&NINO TW: @CHY_REC #RecklessEmpire", "followers_count": 2672, "friends_count": 2116, "statues_count": 54476 }, "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-01-04T10:00:13.000Z"), "id": 684071850893180929, "text": "We're #hiring! Read about our latest #job opening here: ETL Developer - https://t.co/qUNqlUOgus #EdenPrairie, MN #IT #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.470786,44.8546856"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "EdenPrairie", "IT", "CareerArc" }}, "user": { "id": 21679372, "name": "TMJ - MSP IT Jobs", "screen_name": "tmj_msp_it", "lang": "en", "location": "St. Paul, MN", "create_at": date("2009-02-23"), "description": "Follow this account for geo-targeted Software Dev. - General/IT job tweets in St. Paul, MN. Need help? Tweet us at @CareerArc!", "followers_count": 441, "friends_count": 328, "statues_count": 218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eden Prairie, MN", "id": "a1a25de9f5efa69b", "name": "Eden Prairie", "place_type": "city", "bounding_box": rectangle("-93.520913,44.799239 -93.397928,44.893162") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2718116, "cityName": "Eden Prairie" } }
+{ "create_at": datetime("2016-01-04T10:00:13.000Z"), "id": 684071850977050624, "text": "Sunny this afternoon, high 55 (13 C). Low 38 (3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575204, "name": "New Orleans Weather", "screen_name": "NewOrleansWX", "lang": "en", "location": "New Orleans, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for New Orleans, LA", "followers_count": 2899, "friends_count": 92, "statues_count": 7888 }, "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-04T10:00:13.000Z"), "id": 684071851186872320, "text": "This is sickening. I'm so disappointed. https://t.co/tdmKnbPnxB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4199407294, "name": "Kallen J. Romero", "screen_name": "KallenRomero", "lang": "en", "location": "Tampa, FL", "create_at": date("2015-11-15"), "description": "null", "followers_count": 13, "friends_count": 61, "statues_count": 25 }, "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-04T10:00:13.000Z"), "id": 684071851249696768, "text": "Kicking off my first week of 2016 workouts with #MondayRunDay over lunch and #ManArmsMonday this afternoon! ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MondayRunDay", "ManArmsMonday" }}, "user": { "id": 268573461, "name": "Casey Winter", "screen_name": "casey_m_winter", "lang": "en", "location": "Manhattan, KS", "create_at": date("2011-03-18"), "description": "A Southern Girl in a Midwestern World, UNC Chapel Hill c/o 2013", "followers_count": 112, "friends_count": 202, "statues_count": 6745 }, "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-04T10:00:13.000Z"), "id": 684071851727822849, "text": "Sophie Lynn flashes a smile to her friend, Dozer https://t.co/AnHFxpv104", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 22934126, "name": "Bark Town USA", "screen_name": "BarkTownUSA", "lang": "en", "location": "Howell, MI", "create_at": date("2009-03-05"), "description": "BarkTown USA, your full service dog daycare and grooming facility in Howell, MI. Here, it's all about the dogs! #Puppies #Dog #Daycare #Grooming #PuppyPlaytime", "followers_count": 327, "friends_count": 113, "statues_count": 17555 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, MI", "id": "8cbcad86e88fd71a", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-84.006198,42.568064 -83.909455,42.651203") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26093, "countyName": "Livingston", "cityID": 2639540, "cityName": "Howell" } }
+{ "create_at": datetime("2016-01-04T10:00:13.000Z"), "id": 684071851740540928, "text": "Sports and politics Carly's big mistake.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 74837453, "name": "Maniere Carl Johnson", "screen_name": "ManiereTheGreat", "lang": "en", "location": "Bronx, NY", "create_at": date("2009-09-16"), "description": "I believe in truth peace and honesty, living by integrity and love, #Humanitarian!!! Law, Politics, Rotary, Debating my passions Leadership my end game!!!", "followers_count": 378, "friends_count": 1074, "statues_count": 27045 }, "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-04T10:00:13.000Z"), "id": 684071851803414528, "text": "Dealing with someone who has no patience or respect is something I probably won't ever be able 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": 755941556, "name": "haley †", "screen_name": "haley_gurtler", "lang": "en", "location": "Sioux Falls, SD // Sheldon, IA", "create_at": date("2012-08-13"), "description": "605 • 712 | @tstreek56 | University of Sioux Falls | Love Your Melon Ambassador | Radiologic Technican | no time for the irrelevant, arrogant people |", "followers_count": 293, "friends_count": 292, "statues_count": 5726 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sheldon, IA", "id": "2bc924d1d7f7604d", "name": "Sheldon", "place_type": "city", "bounding_box": rectangle("-95.862008,43.168433 -95.81949,43.196433") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19141, "countyName": "O'Brien", "cityID": 1972390, "cityName": "Sheldon" } }
+{ "create_at": datetime("2016-01-04T10:00:13.000Z"), "id": 684071851946029056, "text": "Now Hiring: Assistant Director of Admissions | The Ohio State University at Newark: US - OH - Newark | https://t.co/5II6zIBChh #jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.433333,40.116667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "jobs" }}, "user": { "id": 25106589, "name": "NACAC", "screen_name": "NACAC", "lang": "en", "location": "Arlington, VA", "create_at": date("2009-03-18"), "description": "NACAC is an org of more than 15,000 professionals from around the world dedicated to serving students as they make choices pursuing higher education. #nacac", "followers_count": 22898, "friends_count": 1456, "statues_count": 11261 }, "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-01-04T10:00:13.000Z"), "id": 684071851992190976, "text": "@NemoBradley he didn't even go to the game yesterday.. Didn't tell anyone he wasn't going.. Didn't answer his phone.. He was at a casino", "in_reply_to_status": 684071668248109057, "in_reply_to_user": 320612481, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 320612481 }}, "user": { "id": 576982482, "name": "Benny Ficaro⚾", "screen_name": "BennyFicaro", "lang": "en", "location": "null", "create_at": date("2012-05-10"), "description": "4⃣⚾️", "followers_count": 1096, "friends_count": 946, "statues_count": 24686 }, "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-04T10:00:13.000Z"), "id": 684071852076077056, "text": "Hips don't lie ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3201195509, "name": "le durlia", "screen_name": "itsSithBitch", "lang": "en", "location": "Triangle, VA", "create_at": date("2015-04-24"), "description": "hi, im durl. dark side. chipotle. bud heavy.", "followers_count": 167, "friends_count": 500, "statues_count": 2181 }, "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-01-04T10:00:13.000Z"), "id": 684071852084473856, "text": "Lol i see what you talking bout. �� must be my evil twin https://t.co/Y5QhOPNtq3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 367590846, "name": "Boats™ OTF", "screen_name": "FastLane_Ls", "lang": "en", "location": "SavingYaGirl ", "create_at": date("2011-09-03"), "description": "Sunny US Navy at its finest. Born Legendary. its cold outcha ClevelandBred LaiCal . On My Way Cali Commander in Chief of #NavyTwitter ✈️ OH||VA||ME", "followers_count": 742, "friends_count": 573, "statues_count": 39891 }, "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-01-04T10:00:13.000Z"), "id": 684071852319195136, "text": "Kelly Services #Accounting #Job: BOOK KEEPER (#Syracuse, NY) https://t.co/LCbBG2Q74h #KellyJobs #KellyServices #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.1841701,43.0513668"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Accounting", "Job", "Syracuse", "KellyJobs", "KellyServices", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 318736689, "name": "Syracuse Acct Jobs", "screen_name": "tmj_NYS_ACCT", "lang": "en", "location": "Syracuse, NY", "create_at": date("2011-06-16"), "description": "Follow this account for geo-targeted Accounting job tweets in Syracuse, NY from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 285, "friends_count": 274, "statues_count": 13 }, "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-04T10:00:13.000Z"), "id": 684071852327612416, "text": "Data Entry Clerk Admin Clerical Work From Home Customer... - applynowtoday: (#Providence, RI) https://t.co/41q9k5xItV #CustomerService #Job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.4290401,41.8574551"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Providence", "CustomerService", "Job" }}, "user": { "id": 71719032, "name": "TMJ-BOS CstSrv Jobs", "screen_name": "tmj_bos_cstsrv", "lang": "en", "location": "Boston, MA", "create_at": date("2009-09-04"), "description": "Follow this account for geo-targeted Customer Service job tweets in Boston, MA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 327, "friends_count": 287, "statues_count": 184 }, "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-04T10:00:13.000Z"), "id": 684071852667437061, "text": "Barberton Oh Temp:24.4°F Wind:3 mph Dir:N Baro:Rising Rain2day:0.03in Hum:75% UV:0.0 @ 13:00 01/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": 237, "friends_count": 228, "statues_count": 113669 }, "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-01-04T10:00:13.000Z"), "id": 684071852818493440, "text": "�� @ Metairie, Louisiana https://t.co/j6SONho4ey", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.1775,29.9978"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 273248077, "name": "Corrie Victory", "screen_name": "CorrieVictory", "lang": "en", "location": "New Orleans, LA", "create_at": date("2011-03-27"), "description": "| Independent Makeup Artist | Beauty | Bridal | Special Effects | bookings and inquiries: victoryartistry@gmail.com", "followers_count": 822, "friends_count": 381, "statues_count": 9648 }, "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-04T10:00:14.000Z"), "id": 684071853657178112, "text": "Sunny this afternoon, high 51 (11 C). Low 29 (-2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575255, "name": "Shreveport Weather", "screen_name": "ShreveportWX", "lang": "en", "location": "Shreveport, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Shreveport, LA", "followers_count": 698, "friends_count": 92, "statues_count": 7946 }, "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-04T10:00:14.000Z"), "id": 684071853758017537, "text": "@RyndersFiona ideas>>>>>", "in_reply_to_status": -1, "in_reply_to_user": 2528012677, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2528012677 }}, "user": { "id": 3256103258, "name": "ky", "screen_name": "reneekkyyy", "lang": "en", "location": "my own little world ", "create_at": date("2015-06-25"), "description": "keep it 8 more than 92", "followers_count": 545, "friends_count": 203, "statues_count": 415 }, "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-04T10:00:14.000Z"), "id": 684071854445690880, "text": "I need some breakfast", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2842961353, "name": "Humble Beast", "screen_name": "maywetherct", "lang": "en", "location": "In the gym ", "create_at": date("2014-10-06"), "description": "Humble, On my way to greatness", "followers_count": 951, "friends_count": 677, "statues_count": 16924 }, "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-04T10:00:14.000Z"), "id": 684071854584168448, "text": "Kobe Bryant #NBAVote", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "hashtags": {{ "NBAVote" }}, "user": { "id": 2272184119, "name": "Trent", "screen_name": "connor_trent", "lang": "en", "location": "Houston, TX", "create_at": date("2014-01-01"), "description": "null", "followers_count": 431, "friends_count": 309, "statues_count": 2666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-04T10:00:14.000Z"), "id": 684071854617817089, "text": "����. Grits with salt, pepper & cheese my guy https://t.co/V3daeNH2xB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 456426181, "name": "Tayyylor", "screen_name": "__Tayyybaee_", "lang": "en", "location": "Knox//Boro", "create_at": date("2012-01-05"), "description": "✨Good Girl|Bad Habits✨ MTSU'18 #RIP.Shaniah", "followers_count": 527, "friends_count": 510, "statues_count": 9561 }, "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-04T10:00:14.000Z"), "id": 684071854638784512, "text": "black parents be like https://t.co/MssrAEjEbB", "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": "=", "screen_name": "vijooul", "lang": "en", "location": "Arlington Heights, Milwaukee", "create_at": date("2012-11-23"), "description": "creative | she/they | Salaam | moon doggie | tongue aficionado | teaching artist", "followers_count": 2176, "friends_count": 2335, "statues_count": 85333 }, "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-04T10:00:14.000Z"), "id": 684071855406333952, "text": "@meeganwarner #actress @bellusmagazine #bellusmagazine #meeganeloise #modeling #joshuashultz @… https://t.co/6Pg1fq6zmk", "in_reply_to_status": -1, "in_reply_to_user": 2439125041, "favorite_count": 0, "coordinate": point("-118.3199692,34.1049385"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "actress", "bellusmagazine", "meeganeloise", "modeling", "joshuashultz" }}, "user_mentions": {{ 2439125041, 2220410166 }}, "user": { "id": 39374640, "name": "Joshua Shultz", "screen_name": "JoshuaShultz", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-05-11"), "description": "Director and Photographer Instagram @joshuashultz", "followers_count": 5576, "friends_count": 2983, "statues_count": 11668 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:14.000Z"), "id": 684071855607660546, "text": "lol same https://t.co/M9AjkXjOMA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2511351126, "name": "col10", "screen_name": "coltenfarley6", "lang": "en", "location": "jsu", "create_at": date("2014-05-20"), "description": "hey, go cocks", "followers_count": 495, "friends_count": 360, "statues_count": 4619 }, "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-04T10:00:14.000Z"), "id": 684071855842459648, "text": "All my homies live dummy far from me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2464157000, "name": "Mayateㄣ⃒", "screen_name": "Naythyn_", "lang": "en", "location": "SL", "create_at": date("2014-04-25"), "description": "steph ❤️", "followers_count": 464, "friends_count": 469, "statues_count": 17794 }, "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-04T10:00:14.000Z"), "id": 684071855855136768, "text": "Those 2 days just took me back ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 289564918, "name": "R.I.P. MARQUISE", "screen_name": "Simply_Tonii", "lang": "en", "location": "Miami Beach, FL", "create_at": date("2011-04-28"), "description": "Cancer ♋\nSnapchat: @simply_tonii \nIG: @simply_tonii", "followers_count": 770, "friends_count": 1050, "statues_count": 10206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Golden Glades, FL", "id": "dbdbdadee4fbdda9", "name": "Golden Glades", "place_type": "city", "bounding_box": rectangle("-80.229746,25.892265 -80.163078,25.929961") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1226375, "cityName": "Golden Glades" } }
+{ "create_at": datetime("2016-01-04T10:00:14.000Z"), "id": 684071855968227328, "text": "don't bitch and complain", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4326120493, "name": "Rebecca", "screen_name": "bexxxxxssss", "lang": "en", "location": "null", "create_at": date("2015-11-22"), "description": "la vida es un carnaval", "followers_count": 101, "friends_count": 107, "statues_count": 2640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:14.000Z"), "id": 684071856027111424, "text": "I 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": 1039241965, "name": "¢ɸυʀт ♡", "screen_name": "AaensonCourt", "lang": "en", "location": "WA ✈️ Johnson City, TN ", "create_at": date("2012-12-27"), "description": "somewhere under clouds of marijuana // RIP 8.18.15 //", "followers_count": 1540, "friends_count": 996, "statues_count": 30498 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Grove, TN", "id": "6c8191aa784849fa", "name": "Oak Grove", "place_type": "city", "bounding_box": rectangle("-82.449312,36.391883 -82.406242,36.445457") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47179, "countyName": "Washington", "cityID": 4754700, "cityName": "Oak Grove" } }
+{ "create_at": datetime("2016-01-04T10:00:14.000Z"), "id": 684071856110989312, "text": "������������ https://t.co/CbhFLfly99", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 125219692, "name": "IG : msjohnnisexo", "screen_name": "51shadesofJay_", "lang": "en", "location": "Miami Beach, FL", "create_at": date("2010-03-21"), "description": "SC: johnniser IG: msjohnnisexo", "followers_count": 254, "friends_count": 511, "statues_count": 6619 }, "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-04T10:00:14.000Z"), "id": 684071856198955008, "text": "Head to toe in black with a side of ombré. liketoknow.it https://t.co/tGItFk3DsM #liketkit // PC:… https://t.co/Y8WGOkjAqn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.1606528,32.71312672"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "liketkit" }}, "user": { "id": 1117765051, "name": "Olia Majd", "screen_name": "oliamajd", "lang": "en", "location": "California", "create_at": date("2013-01-24"), "description": "French/Persian Esthetician, Style Blogger & YouTuber!", "followers_count": 7170, "friends_count": 170, "statues_count": 4185 }, "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-04T10:00:14.000Z"), "id": 684071856295559168, "text": "@captbeardd ive already addressed this bb https://t.co/vrkgZpzfvX", "in_reply_to_status": 684071281751371777, "in_reply_to_user": 417283915, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 417283915 }}, "user": { "id": 975243546, "name": "voia", "screen_name": "voiamusic", "lang": "en", "location": "new york", "create_at": date("2012-11-27"), "description": "android-musician.vocalist.your friend // @pvpercrane @atkthemusic // booking: smoothietunesmusic@gmail.com collabs/remixes/etc: revise@attackthemusic.com", "followers_count": 1446, "friends_count": 1286, "statues_count": 9538 }, "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-04T10:00:14.000Z"), "id": 684071856303767552, "text": "Sunny this afternoon, high 58 (14 C). Low 32 (0 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575312, "name": "Lake Charles Weather", "screen_name": "LakeCharlesWX", "lang": "en", "location": "Lake Charles, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Lake Charles, LA", "followers_count": 540, "friends_count": 92, "statues_count": 7930 }, "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-04T10:00:14.000Z"), "id": 684071856379269124, "text": "#Nursing in #Beachwood, OH: State Tested Nursing Assistant - $500... at Menorah Park Center for Senior L... https://t.co/N1EZWlu20e #STNA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4899136,41.5011199"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "Beachwood", "STNA" }}, "user": { "id": 214122801, "name": "MenorahParkJobs", "screen_name": "MenorahParkJobs", "lang": "en", "location": "Beachwood, Ohio", "create_at": date("2010-11-10"), "description": "Menorah Park Center for Senior Living is one of the largest senior living communities in the state of Ohio.", "followers_count": 132, "friends_count": 82, "statues_count": 1380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beachwood, OH", "id": "fa5bd676191716ee", "name": "Beachwood", "place_type": "city", "bounding_box": rectangle("-81.532635,41.449669 -81.48037,41.501448") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3904500, "cityName": "Beachwood" } }
+{ "create_at": datetime("2016-01-04T10:00:14.000Z"), "id": 684071857100840961, "text": "@NickSherwood97 oh", "in_reply_to_status": 684065743089086464, "in_reply_to_user": 1029659809, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1029659809 }}, "user": { "id": 45563694, "name": "kat", "screen_name": "katxsmith", "lang": "en", "location": "the wild youth", "create_at": date("2009-06-08"), "description": "tell me you love me then come back to haunt me. njs.", "followers_count": 283, "friends_count": 96, "statues_count": 77497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Horseheads, NY", "id": "b38ed6e85a8d3a22", "name": "Horseheads", "place_type": "city", "bounding_box": rectangle("-76.861348,42.135019 -76.748148,42.203132") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36015, "countyName": "Chemung", "cityID": 3635694, "cityName": "Horseheads" } }
+{ "create_at": datetime("2016-01-04T10:00:14.000Z"), "id": 684071857184706560, "text": "Like all three joints out VA got shut down and we got in a accident �� ask @SolbiatoKing3 https://t.co/wvuhuDTddy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 502999446 }}, "user": { "id": 210919588, "name": "#⃣5⃣1⃣", "screen_name": "Flip_Flop52", "lang": "en", "location": "Maryland ", "create_at": date("2010-11-01"), "description": "I Don't Believe in Hype I Believe in Work. Livingstone College Football Player #LC'18", "followers_count": 1215, "friends_count": 1298, "statues_count": 23781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Peppermill Village, MD", "id": "0070136b61ad854c", "name": "Peppermill Village", "place_type": "city", "bounding_box": rectangle("-76.89967,38.887402 -76.87726,38.902155") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2460950, "cityName": "Peppermill Village" } }
+{ "create_at": datetime("2016-01-04T10:00:14.000Z"), "id": 684071857268461568, "text": "#Quincy, IL #Job: Pat Excellence Rep Float at Quincy Medical Group https://t.co/ZeyG1AQVVF #medical #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-91.4098726,39.9356016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Quincy", "Job", "medical", "Jobs", "Hiring" }}, "user": { "id": 2166774096, "name": "Quincy MedGroup", "screen_name": "QuincyMedGroup", "lang": "en", "location": "Quincy, IL", "create_at": date("2013-10-31"), "description": "null", "followers_count": 109, "friends_count": 62, "statues_count": 47 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Quincy, IL", "id": "833d51949e9c0bfb", "name": "Quincy", "place_type": "city", "bounding_box": rectangle("-91.42088,39.879982 -91.315192,39.972396") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17001, "countyName": "Adams", "cityID": 1762367, "cityName": "Quincy" } }
+{ "create_at": datetime("2016-01-04T10:00:14.000Z"), "id": 684071857537040384, "text": "No https://t.co/J8UXxrgofO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3249902922, "name": "Li' Shawt ❣", "screen_name": "miraclelashayy", "lang": "en", "location": "@financer_ ", "create_at": date("2015-06-19"), "description": "SC: adoresmiracle || fineeassk ❤️ @prettyasanna long_liveexx", "followers_count": 667, "friends_count": 473, "statues_count": 26707 }, "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-04T10:00:14.000Z"), "id": 684071857549541377, "text": "Yuhhhh ������ https://t.co/wYxmncP190", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 270620112, "name": "Neddy Wap", "screen_name": "Ned_Flander_", "lang": "en", "location": "crosstown ", "create_at": date("2011-03-22"), "description": "She don't want pets but Ima dog yeah yeah.. #freetae #smang", "followers_count": 1693, "friends_count": 989, "statues_count": 100359 }, "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-04T10:00:14.000Z"), "id": 684071857637568512, "text": "Tonight's forecast is clear and a low of 20°F. #WorldHypnotismDay #TomThumbDay #DivorceMonday #WorldBrailleDay #quadrantidsMeteorShower", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.5333,41.5167"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WorldHypnotismDay", "TomThumbDay", "DivorceMonday", "WorldBrailleDay", "quadrantidsMeteorShower" }}, "user": { "id": 42666285, "name": "Quad Cities Weather", "screen_name": "qcweather", "lang": "en", "location": "Quad Cities", "create_at": date("2009-05-26"), "description": "Is it going to be warm, wet or cold in the Quad Cities today? You can also find us on Facebook.", "followers_count": 740, "friends_count": 0, "statues_count": 11461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rock Island Arsenal, IL", "id": "8f966c3da836a196", "name": "Rock Island Arsenal", "place_type": "city", "bounding_box": rectangle("-90.570223,41.509992 -90.511303,41.523635") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17161, "countyName": "Rock Island", "cityID": 1765104, "cityName": "Rock Island Arsenal" } }
+{ "create_at": datetime("2016-01-04T10:00:14.000Z"), "id": 684071857784406017, "text": "��������❤️❤️���� https://t.co/NU2bPQxkb9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3311904649, "name": "little madi", "screen_name": "ohhthatmadi", "lang": "en", "location": "santiago asb", "create_at": date("2015-08-10"), "description": "psalms 28:7 • do good always", "followers_count": 250, "friends_count": 192, "statues_count": 517 }, "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-04T10:00:14.000Z"), "id": 684071857784537089, "text": "fried egg https://t.co/XE9TXPaIrc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1369768069, "name": "president isaac", "screen_name": "isaacmeep", "lang": "en", "location": "Maryland, USA", "create_at": date("2013-04-21"), "description": "capital letters overrated. ❤10-14-15. running for president #isaac2036. college sports are trash.", "followers_count": 1529, "friends_count": 1175, "statues_count": 51166 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenbelt, MD", "id": "30f7f566cd810031", "name": "Greenbelt", "place_type": "city", "bounding_box": rectangle("-76.915629,38.971715 -76.858787,39.015363") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2434775, "cityName": "Greenbelt" } }
+{ "create_at": datetime("2016-01-04T10:00:15.000Z"), "id": 684071857906135040, "text": "Now Hiring: Director of Counseling | Hinsdale District 86: US - IL - Hinsdale | https://t.co/UjInuDm79p #jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.94813,41.790281"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "jobs" }}, "user": { "id": 25106589, "name": "NACAC", "screen_name": "NACAC", "lang": "en", "location": "Arlington, VA", "create_at": date("2009-03-18"), "description": "NACAC is an org of more than 15,000 professionals from around the world dedicated to serving students as they make choices pursuing higher education. #nacac", "followers_count": 22898, "friends_count": 1456, "statues_count": 11262 }, "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("2016-01-04T10:00:15.000Z"), "id": 684071857969074177, "text": "@_captnm0rgan @MirandaSings Friday... I can't wait!!! ��", "in_reply_to_status": 684071624757252096, "in_reply_to_user": 450402336, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 450402336, 39596022 }}, "user": { "id": 59320951, "name": "Jonathan Cano", "screen_name": "jonathancano", "lang": "en", "location": "Spring, TX", "create_at": date("2009-07-22"), "description": "It Never Gets Easier... You Just Get Better!!!", "followers_count": 21, "friends_count": 17, "statues_count": 101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-04T10:00:15.000Z"), "id": 684071858421907457, "text": "life is a learning process", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1658841228, "name": "King JMart", "screen_name": "kingjmart", "lang": "en", "location": "The Top", "create_at": date("2013-08-09"), "description": "Often Imitated Never Duplicated. || RHS '17 || RHS ASL Club VP 2015-2016 || JV/Varsity #83 || pfl✊ || avid superhero enthusiast ||", "followers_count": 268, "friends_count": 391, "statues_count": 6624 }, "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-04T10:00:15.000Z"), "id": 684071858463879168, "text": "Just had lunch and this is how I feel lmao cannot breath �� I need to… https://t.co/cMOWnbCQoc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0064,40.7142"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 55216714, "name": "EVHT", "screen_name": "Cande_Mor", "lang": "en", "location": "null", "create_at": date("2009-07-09"), "description": "Destiny :Follow my heart and pursuit my happiness even if it's the last thing I do", "followers_count": 109, "friends_count": 88, "statues_count": 6311 }, "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-04T10:00:15.000Z"), "id": 684071858505908225, "text": "@JLMendoza_ tan verdadero como la el amor musulman de donald trump", "in_reply_to_status": 684069399968350208, "in_reply_to_user": 193862468, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 193862468 }}, "user": { "id": 112358584, "name": "Ernesto Paulino", "screen_name": "ernesto022", "lang": "es", "location": "Miami, FL", "create_at": date("2010-02-07"), "description": "Aprendiendo a vivir, quizás nunca lo logre... #RealMadrid, #Chelsea, #RedSox, #licey... 24/5/2014", "followers_count": 206, "friends_count": 497, "statues_count": 8405 }, "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-01-04T10:00:15.000Z"), "id": 684071858560446464, "text": "Tomorrows 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": 198698602, "name": "Nessi", "screen_name": "Vnessa_Mariee", "lang": "en", "location": "Surf City ", "create_at": date("2010-10-04"), "description": "Michael Avila♡", "followers_count": 965, "friends_count": 365, "statues_count": 15765 }, "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-04T10:00:15.000Z"), "id": 684071858564562944, "text": "Wind 4 mph SSE. Barometer 1009.8 hPa, Falling quickly. Temperature 86.4 °F. Rain today 0.00 in. Humidity 10%", "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": 87, "friends_count": 265, "statues_count": 140207 }, "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-04T10:00:15.000Z"), "id": 684071858795233280, "text": "Mind Yo Business Bitch!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1059756260, "name": "Q.", "screen_name": "QamarCTHS", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2013-01-03"), "description": "|FloridaMade954|\n|R.I.P Lil Kenny| |R.I.P Akee| #FreeTony", "followers_count": 814, "friends_count": 418, "statues_count": 20416 }, "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-04T10:00:15.000Z"), "id": 684071858845532161, "text": "#Oakland, CA #Retail #Job: Retail Sales Merchandiser Full Time at Advantage Solutions https://t.co/UfXb5FvFcl #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.2711137,37.8043637"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Oakland", "Retail", "Job", "Jobs", "Hiring" }}, "user": { "id": 28720800, "name": "Advantage Careers", "screen_name": "ASolutionsJobs", "lang": "en", "location": "North America", "create_at": date("2009-04-03"), "description": "Advantage Solutions has a career for you! Come Join Our Team Today!", "followers_count": 2022, "friends_count": 1147, "statues_count": 29881 }, "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-04T10:00:15.000Z"), "id": 684071859193692160, "text": "Sunny this afternoon, high 56 (13 C). Low 32 (0 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575370, "name": "Lafayette Weather", "screen_name": "LafayetteWX", "lang": "en", "location": "Lafayette, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Lafayette, LA", "followers_count": 470, "friends_count": 92, "statues_count": 7969 }, "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-04T10:00:15.000Z"), "id": 684071859223015425, "text": "@_xcolbsx mornin' colb", "in_reply_to_status": 684062132430843905, "in_reply_to_user": 3067290843, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3067290843 }}, "user": { "id": 375405944, "name": "phil", "screen_name": "PaulHerschel", "lang": "en", "location": "Minneapolis, Minnesota", "create_at": date("2011-09-17"), "description": "@townshipwi // @casualmpls", "followers_count": 393, "friends_count": 220, "statues_count": 7840 }, "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-04T10:00:15.000Z"), "id": 684071859285934081, "text": "Want to work at Mandarin Oriental Hotel Las Vegas? We're #hiring in #LasVegas, NV! Click for details: https://t.co/HR0TQezyhT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.1745805,36.1062236"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "LasVegas" }}, "user": { "id": 3089847157, "name": "Mandarin Las Vegas", "screen_name": "MO_LVJobs", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-03-16"), "description": "Nevada's only Triple Forbes Five-Star luxury destination located on the world-famous #LasVegas Strip. Apply here to open #jobs.", "followers_count": 84, "friends_count": 59, "statues_count": 44 }, "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-04T10:00:15.000Z"), "id": 684071859453857792, "text": "@TODAYshow ok so today marks #90days2vaca and I'm thinking of documenting my transformation if any so here it goes. https://t.co/YHHPzv8HkG", "in_reply_to_status": -1, "in_reply_to_user": 7744592, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "90days2vaca" }}, "user_mentions": {{ 7744592 }}, "user": { "id": 32641996, "name": "Audra True", "screen_name": "audratrue", "lang": "en", "location": "South Carolina", "create_at": date("2009-04-17"), "description": "null", "followers_count": 13, "friends_count": 84, "statues_count": 133 }, "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-01-04T10:00:15.000Z"), "id": 684071859525042176, "text": "Kelly Services #Clerical #Job: Administrative Assistant II (#Beaverton, OR) https://t.co/OY608mpkwY #KellyJobs #KellyServices #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.8029665,45.4923824"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Clerical", "Job", "Beaverton", "KellyJobs", "KellyServices", "Jobs", "Hiring" }}, "user": { "id": 23022397, "name": "TMJ-PDX Cleric. Jobs", "screen_name": "tmj_pdx_cler", "lang": "en", "location": "Portland, OR", "create_at": date("2009-03-05"), "description": "Follow this account for geo-targeted Office/Clerical job tweets in Portland, OR. Need help? Tweet us at @CareerArc!", "followers_count": 291, "friends_count": 201, "statues_count": 85 }, "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-04T10:00:15.000Z"), "id": 684071859525169152, "text": "Give no fucks yeah we don't give no fucks yeah ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 205588654, "name": "Pistol P", "screen_name": "Paris_HUH", "lang": "en", "location": "174th Rocblock ", "create_at": date("2010-10-20"), "description": "IG: Pistol__P174 // sc: pistolp174", "followers_count": 622, "friends_count": 333, "statues_count": 27502 }, "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-04T10:00:15.000Z"), "id": 684071859692802048, "text": "That's why Ehub >>>> �� https://t.co/vBzpaBCdAY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 145273454, "name": "Jabril X ", "screen_name": "FeedTheSheed", "lang": "en", "location": "Houston, TX", "create_at": date("2010-05-18"), "description": "Go Birds", "followers_count": 2296, "friends_count": 1374, "statues_count": 143275 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-04T10:00:15.000Z"), "id": 684071859738918912, "text": "...What? https://t.co/JzLWCcHDfG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 415012178, "name": "SPURS 29-6", "screen_name": "brenden_i", "lang": "en", "location": "San Marcos, Texas", "create_at": date("2011-11-17"), "description": "Arab trap representative. #txst #Spurs #Raiders", "followers_count": 505, "friends_count": 750, "statues_count": 52730 }, "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-04T10:00:15.000Z"), "id": 684071859860578305, "text": "Only if they knew!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 193442206, "name": "Jalicia usher", "screen_name": "IS_A_Bombshell", "lang": "en", "location": "ÜT: 41.8923779,-87.6153839", "create_at": date("2010-09-21"), "description": "I am a young black women that loves being independent. I love to make my own money$$ overall I am just a real bitch that like to provide for me and my child....", "followers_count": 131, "friends_count": 229, "statues_count": 1933 }, "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-04T10:00:15.000Z"), "id": 684071859910905856, "text": "LOL https://t.co/7YnOJJz4oF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "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": 11333, "friends_count": 1008, "statues_count": 338857 }, "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-04T10:00:15.000Z"), "id": 684071860670218241, "text": "���� https://t.co/T0K5jggH34", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 381094734, "name": "Bailey James", "screen_name": "bjmcafkjl", "lang": "en", "location": "null", "create_at": date("2011-09-27"), "description": "#FreeAndy #FuckIt", "followers_count": 418, "friends_count": 397, "statues_count": 16484 }, "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-01-04T10:00:15.000Z"), "id": 684071860682625025, "text": "@CStalling �� dude wtf I've barely started it..you're evil ��", "in_reply_to_status": 684071488404598784, "in_reply_to_user": 3245569350, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3245569350 }}, "user": { "id": 2596793167, "name": "Jendaya Christmas", "screen_name": "jayyclaus", "lang": "en", "location": "null", "create_at": date("2014-06-30"), "description": "null", "followers_count": 229, "friends_count": 213, "statues_count": 2113 }, "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-01-04T10:00:15.000Z"), "id": 684071861454532610, "text": "Boy you �� https://t.co/bt7WsYzMQY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 409015116, "name": "Sir Dee", "screen_name": "DeeTruthIsaac", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2011-11-09"), "description": "TMT TBE Long LiveDMiles", "followers_count": 1044, "friends_count": 219, "statues_count": 55169 }, "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-04T10:00:15.000Z"), "id": 684071861483737088, "text": "If you're a #Retail professional in #SHALLOTTE, NC, check out this #job: https://t.co/yH06UBwSl5 #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.3716526,33.9781567"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "SHALLOTTE", "job", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 59967889, "name": "TMJ-NC Retail Jobs", "screen_name": "tmj_nc_retail", "lang": "en", "location": "North Carolina", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in North Carolina. Need help? Tweet us at @CareerArc!", "followers_count": 394, "friends_count": 305, "statues_count": 588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shallotte, NC", "id": "013127b30aaf3731", "name": "Shallotte", "place_type": "city", "bounding_box": rectangle("-78.410893,33.910755 -78.357798,34.00227") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37019, "countyName": "Brunswick", "cityID": 3760800, "cityName": "Shallotte" } }
+{ "create_at": datetime("2016-01-04T10:00:15.000Z"), "id": 684071861525721088, "text": "Wind 4.0 mph N. Barometer 1036.71 mb, Steady. Temperature 30.4 °F. Rain today 0.00 in. Humidity 81%", "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": 25, "friends_count": 124, "statues_count": 11602 }, "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-04T10:00:15.000Z"), "id": 684071861748023299, "text": "seriously... how does this shit happen", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 56484541, "name": "christina lovermi", "screen_name": "christinaajoan", "lang": "en", "location": "null", "create_at": date("2009-07-13"), "description": "|| keep the faith || ♡✨ stockton '16", "followers_count": 429, "friends_count": 990, "statues_count": 13283 }, "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-04T10:00:15.000Z"), "id": 684071861810892802, "text": "Yup that's right #preach my love!! �� https://t.co/PpTMts9OIr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "preach" }}, "user": { "id": 2991228338, "name": "Lovelykel92", "screen_name": "delgado_racquel", "lang": "en", "location": "Cali", "create_at": date("2015-01-20"), "description": "#CureMS #Chargersnation⚡️ #Dance #TeamCarlito ❤️", "followers_count": 754, "friends_count": 1204, "statues_count": 1941 }, "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-04T10:00:15.000Z"), "id": 684071861848690688, "text": "Sunny this afternoon, high 46 (8 C). Low 25 (-4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575409, "name": "Fort Smith Weather", "screen_name": "FortSmithWX", "lang": "en", "location": "Fort Smith, AR", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Fort Smith, AR", "followers_count": 442, "friends_count": 92, "statues_count": 8018 }, "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-04T10:00:16.000Z"), "id": 684071862029135875, "text": "Huge bday S/O to my man @JohnRensel4 ���� have a good day brotha! https://t.co/qFZoR07CgY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 401084740 }}, "user": { "id": 592807339, "name": "Derek Dalton", "screen_name": "ducieduce98", "lang": "en", "location": "null", "create_at": date("2012-05-28"), "description": "Sydney Schertzinger is an absolute babe", "followers_count": 605, "friends_count": 493, "statues_count": 3500 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallmadge, OH", "id": "cdb98b233892e8af", "name": "Tallmadge", "place_type": "city", "bounding_box": rectangle("-81.465754,41.060128 -81.373636,41.135788") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3976106, "cityName": "Tallmadge" } }
+{ "create_at": datetime("2016-01-04T10:00:16.000Z"), "id": 684071862356328448, "text": "Only 24 hours left to enter our #swplanner #giveaway! \n\nWe will be giving away one Southern… https://t.co/YKvJopB3HN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.643461,41.943636"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "swplanner", "giveaway" }}, "user": { "id": 2946077511, "name": "StorybookChicago", "screen_name": "StorybookChi", "lang": "en", "location": "Chicago, Illinois", "create_at": date("2014-12-28"), "description": "Storybook Weddings & Events is a boutique wedding planning company servicing Chicago and the Chicagoland area. Contact us at info@storybookchicago.com", "followers_count": 217, "friends_count": 522, "statues_count": 603 }, "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-04T10:00:16.000Z"), "id": 684071862448439298, "text": "���������� my patience is running so thin ya'll ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 31134514, "name": "VanellopeVonShweetz", "screen_name": "dreface", "lang": "en", "location": "Spring, TX", "create_at": date("2009-04-14"), "description": "25. IG: dreface Snapchat: Andrea Davis", "followers_count": 606, "friends_count": 558, "statues_count": 40678 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-04T10:00:16.000Z"), "id": 684071862477959168, "text": "The weather is boring. 16ºF and fair. #MorrisWeather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.9,45.59"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MorrisWeather" }}, "user": { "id": 1931463259, "name": "Morris Weather", "screen_name": "MorrisMNWeather", "lang": "en", "location": "Morris, MN", "create_at": date("2013-10-03"), "description": "A place to find the current weather in Morris, MN. Run by @bman4789\n\nScheduled tweets are at 7 AM, 12 PM, 3 PM, 6 PM, and 10 PM. Daily forecast at 6 AM.", "followers_count": 21, "friends_count": 1, "statues_count": 1782 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morris, MN", "id": "dc943406a248a125", "name": "Morris", "place_type": "city", "bounding_box": rectangle("-95.933929,45.564348 -95.883709,45.607334") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27149, "countyName": "Stevens", "cityID": 2744242, "cityName": "Morris" } }
+{ "create_at": datetime("2016-01-04T10:00:16.000Z"), "id": 684071862553415681, "text": "I don't mess with a lot of ppl because they not on my level.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2354172188, "name": "Marcus Carter", "screen_name": "iamthndr", "lang": "en", "location": "Chicago, IL", "create_at": date("2014-02-20"), "description": "DJ @SpiritNavyPier @OdysseyChicago @MysticNavyPier + Creative Editor @ReactPresents + Apart of @LightEmUpChi & @DynastyPodcasts + CEO of @HyeDemand", "followers_count": 741, "friends_count": 669, "statues_count": 2142 }, "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-04T10:00:16.000Z"), "id": 684071862859501568, "text": "reeling with the feeling don't stop continue", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 571216123, "name": "jasmin blue", "screen_name": "jblucifer", "lang": "en", "location": "atx", "create_at": date("2012-05-04"), "description": "she's in love with the world #SEU19", "followers_count": 533, "friends_count": 280, "statues_count": 17952 }, "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-04T10:00:16.000Z"), "id": 684071862947610626, "text": "Homie paying 3 child supports & only making 8.50/hr lmaooo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 114123868, "name": "Joe", "screen_name": "joeinthecut_", "lang": "en", "location": "Houston ", "create_at": date("2010-02-13"), "description": "#UHAlumnus. Blessed.", "followers_count": 2013, "friends_count": 1063, "statues_count": 151669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-04T10:00:16.000Z"), "id": 684071863023091712, "text": "Time to get focused and take inventory of plans for 2016. #mondaymotivation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "mondaymotivation" }}, "user": { "id": 278316381, "name": "Jonathan Moseley Sr", "screen_name": "drjay2354", "lang": "en", "location": "So Cal", "create_at": date("2011-04-06"), "description": "Motivational speaker and Consultant. CEO Moseley Enterprise, VP Nation Action Network LA, aka The Godfather. ole skool 2 da bone. I love music!", "followers_count": 451, "friends_count": 1080, "statues_count": 5427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:16.000Z"), "id": 684071863052582912, "text": "I'm forever embarrassing myself in front of the guy I got a crush on. FOREVER.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 57953843, "name": "taylor", "screen_name": "AfiaOnTheRun", "lang": "en", "location": "nc", "create_at": date("2009-07-18"), "description": "pro-black, intersectional feminist, college student", "followers_count": 3509, "friends_count": 742, "statues_count": 105195 }, "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-04T10:00:16.000Z"), "id": 684071863056773121, "text": "Last shopping stop (at @Walgreens in Guaynabo, PR) https://t.co/YugFHnOjnL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-66.10459991,18.40956187"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 46177695 }}, "user": { "id": 14667213, "name": "Ivette Johnson", "screen_name": "ivettej", "lang": "en", "location": "Lima, Peru", "create_at": date("2008-05-05"), "description": "Mom, marketer, tech maniac. Not neccesarily in that order | Living in Perú, trying to see the world | Connecting the dots since 1974 | Working @LHHDBMPERU", "followers_count": 898, "friends_count": 1881, "statues_count": 9169 }, "place": { "country": "Puerto Rico", "country_code": "Puerto Rico", "full_name": "Guaynabo, Puerto Rico", "id": "b1be5b7cdf83c424", "name": "Guaynabo", "place_type": "city", "bounding_box": rectangle("-66.142021,18.332759 -66.088351,18.440542") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72061, "countyName": "Guaynabo", "cityID": 7232522, "cityName": "Guaynabo" } }
+{ "create_at": datetime("2016-01-04T10:00:16.000Z"), "id": 684071863559913473, "text": "@jon_c_saldana Me this season �� https://t.co/ktckrrFU9s", "in_reply_to_status": -1, "in_reply_to_user": 2481053641, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2481053641 }}, "user": { "id": 1011348781, "name": "Guillermo Vasquez", "screen_name": "fiverstud_", "lang": "en", "location": "3B & The Bump", "create_at": date("2012-12-14"), "description": "null", "followers_count": 535, "friends_count": 1449, "statues_count": 12455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:16.000Z"), "id": 684071863614504960, "text": "Literally every school in Oregon is closed but Mcminnville", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1636549999, "name": "kenny", "screen_name": "kenddiiee", "lang": "en", "location": "McMinnville", "create_at": date("2013-07-31"), "description": "high key bitch// snapchat: kenddiee", "followers_count": 583, "friends_count": 564, "statues_count": 7845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McMinnville, OR", "id": "2a732a9dca7f3c87", "name": "McMinnville", "place_type": "city", "bounding_box": rectangle("-123.25428,45.185598 -123.128954,45.239298") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41071, "countyName": "Yamhill", "cityID": 4145000, "cityName": "McMinnville" } }
+{ "create_at": datetime("2016-01-04T10:00:16.000Z"), "id": 684071863631261696, "text": "I'm sorry.. But I'm not the kind of female whose gonna beg U2 search thru ur phone. If I don't trust u we don't need to be together. PERIOD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 336706210, "name": "Bryley Holmes", "screen_name": "BRYLEYHOLMES", "lang": "en", "location": "Nebraska, USA", "create_at": date("2011-07-16"), "description": "I win, you lose", "followers_count": 2016, "friends_count": 1482, "statues_count": 58797 }, "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-04T10:00:16.000Z"), "id": 684071863635464192, "text": "I don't ever wanna wake up, looking into someone else's eyes ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1141488764, "name": "Jacqueline King", "screen_name": "jacquelinek19", "lang": "en", "location": "null", "create_at": date("2013-02-01"), "description": "null", "followers_count": 1318, "friends_count": 1163, "statues_count": 10825 }, "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-04T10:00:16.000Z"), "id": 684071863757225984, "text": "I got hella sweat outfits", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1220050592, "name": "Boonie ™", "screen_name": "Boonieflexinx2", "lang": "en", "location": "4⃣1⃣9⃣ ⭕️H❗️⭕️", "create_at": date("2013-02-25"), "description": "IG:@_boonieflexin2 Tyrin Boone Snapchat - bacshot_boonex2 Dreams worth more than money R.I.P likko", "followers_count": 1073, "friends_count": 1135, "statues_count": 22814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mansfield, OH", "id": "f136163002bd51f6", "name": "Mansfield", "place_type": "city", "bounding_box": rectangle("-82.572489,40.681009 -82.378653,40.815657") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39139, "countyName": "Richland", "cityID": 3947138, "cityName": "Mansfield" } }
+{ "create_at": datetime("2016-01-04T10:00:16.000Z"), "id": 684071863903899648, "text": "[Mon Jan 4th, 12:00pm] Temperature: 65.3 F, 18.5 C; Humidity: 37.5%; Light: 23.54%. https://t.co/UI5qWnQ21d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 269430133, "name": "Shoemaker Sandbox", "screen_name": "shoe_sandbox", "lang": "en", "location": "Earth (for now). ", "create_at": date("2011-03-20"), "description": "I am a bot. Developer account for @shoemaker.", "followers_count": 7, "friends_count": 1, "statues_count": 24477 }, "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-04T10:00:16.000Z"), "id": 684071864084254721, "text": "Now Stitch telling Ashley to take care of herself. I'm just waiting on Abby and Ashley to do Mother Daughter porn with stitch. #yr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "yr" }}, "user": { "id": 366636001, "name": "Luke Pegues", "screen_name": "Lukeduke75", "lang": "en", "location": "null", "create_at": date("2011-09-02"), "description": "TV fanatic, Y&R junkie, JR Ewing Lover, Cookie Lyons #1 Supporter", "followers_count": 373, "friends_count": 354, "statues_count": 14857 }, "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-04T10:00:16.000Z"), "id": 684071864168087554, "text": "Personally I'd much rather be healthy and at school than sick and in bed.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1682153106, "name": "keen", "screen_name": "keenanissocool", "lang": "en", "location": "check out my boys soundcloud⤵️", "create_at": date("2013-08-18"), "description": "16. Blessed.", "followers_count": 627, "friends_count": 460, "statues_count": 11800 }, "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-04T10:00:16.000Z"), "id": 684071864302305280, "text": "How to make my morning: Ask me if I'd like 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": 426384857, "name": "الياسمين", "screen_name": "enimsajthing", "lang": "en", "location": "null", "create_at": date("2011-12-01"), "description": "null", "followers_count": 234, "friends_count": 310, "statues_count": 16841 }, "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-04T10:00:16.000Z"), "id": 684071864382042112, "text": "Mostly sunny this afternoon, high 41 (5 C). Low 26 (-3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575444, "name": "Little Rock Weather", "screen_name": "LittleRockWX", "lang": "en", "location": "Little Rock, AR", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Little Rock, AR", "followers_count": 1835, "friends_count": 92, "statues_count": 7892 }, "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-04T10:00:16.000Z"), "id": 684071864696745984, "text": "@Semscure you are mine https://t.co/OBL7k6fBdR", "in_reply_to_status": -1, "in_reply_to_user": 371523135, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 371523135 }}, "user": { "id": 2861865872, "name": "Supra", "screen_name": "Supraisbae", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2014-10-17"), "description": "null", "followers_count": 200, "friends_count": 215, "statues_count": 6222 }, "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-04T10:00:16.000Z"), "id": 684071864784687104, "text": "@FlatChestAtBest so true.", "in_reply_to_status": 684020395993792512, "in_reply_to_user": 211366809, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 211366809 }}, "user": { "id": 602489616, "name": "Danielleohelle", "screen_name": "daniellemdallas", "lang": "en", "location": "Long Beach, California", "create_at": date("2012-06-07"), "description": "I need less hot cheetos and more ab crunches •08/??/10• Long Beach State '18•", "followers_count": 198, "friends_count": 176, "statues_count": 12363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kingsbury, NV", "id": "3ade6caf4df367b0", "name": "Kingsbury", "place_type": "city", "bounding_box": rectangle("-119.937258,38.959023 -119.884492,38.986512") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32005, "countyName": "Douglas", "cityID": 3238000, "cityName": "Kingsbury" } }
+{ "create_at": datetime("2016-01-04T10:00:16.000Z"), "id": 684071864918913024, "text": "Join the Ghirardelli team! See our latest #BusinessMgmt #job opening here: https://t.co/ahIzul8b0W #management #Anaheim, CA #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.9145036,33.8352932"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BusinessMgmt", "job", "management", "Anaheim", "Hiring" }}, "user": { "id": 2228943546, "name": "Ghirardelli Careers", "screen_name": "ghirardellijobs", "lang": "en", "location": "null", "create_at": date("2013-12-03"), "description": "Ghirardelli invites you to explore our open #jobs and discover a place where your ideas come to life, as part of our delicious legacy.", "followers_count": 51, "friends_count": 1, "statues_count": 84 }, "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-04T10:00:16.000Z"), "id": 684071865023905793, "text": "almost everyday since break has started I have woken up at noon.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2588362327, "name": "hayla hubbard", "screen_name": "Haylahubbard", "lang": "en", "location": "ΔΒΣ", "create_at": date("2014-06-25"), "description": "OHS", "followers_count": 168, "friends_count": 189, "statues_count": 845 }, "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-04T10:00:16.000Z"), "id": 684071865065836545, "text": "I'm at Boston Logan International Airport (BOS) in Boston, MA https://t.co/97Y7mhYDKr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.01667643,42.36650312"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 40370520, "name": "José Antonio", "screen_name": "jantonioadac", "lang": "pt", "location": "Boston-MA", "create_at": date("2009-05-15"), "description": "null", "followers_count": 9, "friends_count": 65, "statues_count": 1598 }, "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-04T10:00:16.000Z"), "id": 684071865091010560, "text": "1 month till I get my license back ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2197974282, "name": "❃ ❀ ✿ ❁ ✾", "screen_name": "DaneChyna", "lang": "en", "location": "Powell, OH", "create_at": date("2013-11-16"), "description": "null", "followers_count": 579, "friends_count": 65, "statues_count": 3746 }, "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-04T10:00:16.000Z"), "id": 684071865233485825, "text": "Woke up to the sound of my dog barfing on my bed :) you know it's going to be 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": 1631923322, "name": "Supreme Dictator", "screen_name": "ybagherzadeh", "lang": "en", "location": "null", "create_at": date("2013-07-29"), "description": "hi Check me on Go90 in ATV's LA Story", "followers_count": 412, "friends_count": 331, "statues_count": 6887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:16.000Z"), "id": 684071865254457344, "text": "What happen to all the taz angels", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 477939151, "name": "babygirljuicy", "screen_name": "htxbaddiest", "lang": "en", "location": "screwston", "create_at": date("2012-01-29"), "description": "im ya plugs plug bitch.", "followers_count": 651, "friends_count": 286, "statues_count": 44816 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-04T10:00:16.000Z"), "id": 684071865829208065, "text": "@LisaVanderpump he is so handsome. Not to mention to be able 2 live in a beautiful palace like Villa Rosa. Lucky fellow!", "in_reply_to_status": 684039656640389120, "in_reply_to_user": 110840720, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 110840720 }}, "user": { "id": 443398842, "name": "martine pirozzi", "screen_name": "MartinePirozzi", "lang": "en", "location": "Toledo, Ohio", "create_at": date("2011-12-21"), "description": "Birthday is 2/24/1965. I have had Sarcoidosis and ITP for many years. Disabled. My dream is to meet Lisa Vanderpump and Kyle Richards.", "followers_count": 274, "friends_count": 1009, "statues_count": 7201 }, "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-04T10:00:17.000Z"), "id": 684071866219282432, "text": "YOU KNOWWWWWW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 380274514, "name": "Miss Sugars", "screen_name": "VanillaFlaca", "lang": "en", "location": "Probably in a kitchen", "create_at": date("2011-09-26"), "description": "Life is uncertain, eat dessert first. NY", "followers_count": 764, "friends_count": 348, "statues_count": 52598 }, "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-01-04T10:00:17.000Z"), "id": 684071866651164672, "text": "OfficeTeam: Customer Service Representative (#Pewaukee, WI) https://t.co/IU5qYtIjaN #CustomerService #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.2169027,43.0542058"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Pewaukee", "CustomerService", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 71948502, "name": "TMJ-MKE CstSrv Jobs", "screen_name": "tmj_mke_cstsrv", "lang": "en", "location": "Milwaukee, WI", "create_at": date("2009-09-05"), "description": "Follow this account for geo-targeted Customer Service job tweets in Milwaukee, WI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 305, "friends_count": 276, "statues_count": 126 }, "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-01-04T10:00:17.000Z"), "id": 684071866672091137, "text": "I can honestly say I came a long way!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1532049180, "name": "✨Bryan Trimble✨", "screen_name": "TheReal_BT21", "lang": "en", "location": "Kansas City, MO", "create_at": date("2013-06-19"), "description": "philippians 4:13", "followers_count": 653, "friends_count": 364, "statues_count": 4136 }, "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.752215 -97.244569,37.78135") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20173, "countyName": "Sedgwick", "cityID": 2005337, "cityName": "Bel Aire" } }
+{ "create_at": datetime("2016-01-04T10:00:17.000Z"), "id": 684071866785374208, "text": "Partly cloudy this afternoon, high 38 (3 C). Low 24 (-4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26195145, "name": "Wichita Weather", "screen_name": "WichitaWeather", "lang": "en", "location": "Wichita, KS", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Wichita, KS", "followers_count": 1847, "friends_count": 88, "statues_count": 7747 }, "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-04T10:00:17.000Z"), "id": 684071866831646725, "text": "reuniting with my luvbugggg today ������������ get too pick him later tonight ❤️ Im sooo happy wth ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 521883744, "name": "YaYa✨", "screen_name": "ylw_xo", "lang": "en", "location": "941 ✈️ 850", "create_at": date("2012-03-11"), "description": "She's a women of substance, a beauty and a beast❤️ #FAMU19", "followers_count": 1903, "friends_count": 2010, "statues_count": 28644 }, "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-04T10:00:17.000Z"), "id": 684071866911318017, "text": "Can't wait for my hair to grow back ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 401080385, "name": "KP", "screen_name": "kristenpray20", "lang": "en", "location": "United States", "create_at": date("2011-10-29"), "description": "Looking forward to what's to come. RobertMorrisSB", "followers_count": 873, "friends_count": 564, "statues_count": 6661 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Royal, PA", "id": "52823121f24097ac", "name": "Port Royal", "place_type": "city", "bounding_box": rectangle("-77.399511,40.52641 -77.380529,40.542642") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42067, "countyName": "Juniata", "cityID": 4262304, "cityName": "Port Royal" } }
+{ "create_at": datetime("2016-01-04T10:00:17.000Z"), "id": 684071867053785088, "text": "@blacknasty869", "in_reply_to_status": -1, "in_reply_to_user": 3348596363, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3348596363 }}, "user": { "id": 1198507452, "name": "pretty&petty✨", "screen_name": "xotezyah", "lang": "en", "location": "dirty jerzzz✨", "create_at": date("2013-02-19"), "description": "Jarell E.❤️", "followers_count": 1574, "friends_count": 1130, "statues_count": 35506 }, "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-04T10:00:17.000Z"), "id": 684071867280314368, "text": "why lie��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 79302696, "name": "fatfat❣", "screen_name": "__KhyMari", "lang": "en", "location": "w| the shits. ♥️", "create_at": date("2009-10-02"), "description": "pending ...", "followers_count": 1308, "friends_count": 876, "statues_count": 34565 }, "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-04T10:00:17.000Z"), "id": 684071867435585536, "text": "���������������������������������������������� https://t.co/2SbiM8Igwt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 277298044, "name": "LightskinJimmyButler", "screen_name": "Terence_Huie", "lang": "en", "location": "Tallahassee, FL", "create_at": date("2011-04-04"), "description": "Accountant by day, sports junkie by night. Sometimes I film basketball. Oh and I tweet a lot. I do it all.", "followers_count": 2039, "friends_count": 739, "statues_count": 188371 }, "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-04T10:00:17.000Z"), "id": 684071867569672192, "text": "I'm extremely sick, i have class at 10:55, and I'm struggling to eat my breakfast.", "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": 509, "friends_count": 220, "statues_count": 54387 }, "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-04T10:00:17.000Z"), "id": 684071867754364928, "text": "@sd_phillips hey my man, what are y'all's plans for the trip? Flying to Phoenix or y'all going the Vegas route?", "in_reply_to_status": -1, "in_reply_to_user": 368313066, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 368313066 }}, "user": { "id": 2745022510, "name": "Dustin", "screen_name": "dustyhicks2", "lang": "en", "location": "Mountain Brook, AL", "create_at": date("2014-08-16"), "description": "I love my wife, daughter and Alabama sports! I am a scout/writer for BamaMixTape. I try to always have a good time no matter the situation. I also work for CTSN", "followers_count": 350, "friends_count": 2017, "statues_count": 4079 }, "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-04T10:00:17.000Z"), "id": 684071867934748672, "text": "0 https://t.co/5npGfpJF75", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2411388691, "name": "Jamaya Dandridge", "screen_name": "Mayad__", "lang": "en", "location": "null", "create_at": date("2014-03-25"), "description": "IG\\\\ @mayad__ ✨", "followers_count": 768, "friends_count": 278, "statues_count": 8734 }, "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-04T10:00:17.000Z"), "id": 684071868387704833, "text": "@nbcdays #dool #Days50 \nI believe days has officially just jumped the shark. \n#jumpedtheshark #daysofourlives #whatweretheythinking", "in_reply_to_status": -1, "in_reply_to_user": 29991832, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "dool", "Days50", "jumpedtheshark", "daysofourlives", "whatweretheythinking" }}, "user_mentions": {{ 29991832 }}, "user": { "id": 169728716, "name": "BeAmy", "screen_name": "beamy40", "lang": "en", "location": "PA", "create_at": date("2010-07-22"), "description": "Ramblings of Randomness ~ old soul with a young heart...Texan, Christian, Sarcasm Expert and Self Proclaimed Dilly-Dallier...", "followers_count": 628, "friends_count": 2016, "statues_count": 1880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coatesville, PA", "id": "56346d67bcd19861", "name": "Coatesville", "place_type": "city", "bounding_box": rectangle("-75.841424,39.974835 -75.797618,40.005378") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42029, "countyName": "Chester", "cityID": 4214712, "cityName": "Coatesville" } }
+{ "create_at": datetime("2016-01-04T10:00:17.000Z"), "id": 684071868697976832, "text": ".@USPATRIOT5 @BreitbartNews $13,000,000 \"fail\" for #HillaryClinton? What about the $770M that Ken Timmerman writes of in #DarkForces?", "in_reply_to_status": 684070452398968832, "in_reply_to_user": 166455908, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HillaryClinton", "DarkForces" }}, "user_mentions": {{ 166455908, 457984599 }}, "user": { "id": 265666449, "name": "mark wonderful", "screen_name": "markwonderful", "lang": "en", "location": "Phoenix, Az", "create_at": date("2011-03-13"), "description": "Christian, Ex-1%er, Divorced 2003, Asperger's Az-Born-Boomer Conservative Pro-Israel Fotos Actor Writer Reader 13yrs protecting 1%ers. RETIRED", "followers_count": 1862, "friends_count": 1544, "statues_count": 130265 }, "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-04T10:00:17.000Z"), "id": 684071869239013376, "text": "Some serious throwback https://t.co/ueReTaKSpT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 591286118, "name": "Anthony Ortega", "screen_name": "AnthonyOrtega20", "lang": "en", "location": "Texas, USA", "create_at": date("2012-05-26"), "description": "NFHS •Goalkeeper• Todos vosotros tenéis amor, aprendán como compartirlo con los demás.Snapchat:anthonydegea", "followers_count": 418, "friends_count": 400, "statues_count": 3656 }, "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-04T10:00:17.000Z"), "id": 684071869301952512, "text": "Morning clouds/afternoon sun this afternoon, high 27 (-3 C). Low 21 (-6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26199170, "name": "Omaha Weather", "screen_name": "OmahaWX", "lang": "en", "location": "Omaha, Nebraska", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Omaha, Nebraska", "followers_count": 2589, "friends_count": 88, "statues_count": 7740 }, "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-04T10:00:17.000Z"), "id": 684071869675249664, "text": "im getting a bong next weekend", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2689022580, "name": "melanye b", "screen_name": "wtfmelanye", "lang": "en", "location": "null", "create_at": date("2014-07-28"), "description": "bad girl with alotta hoes", "followers_count": 197, "friends_count": 384, "statues_count": 1754 }, "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-04T10:00:17.000Z"), "id": 684071869994020864, "text": "@Opiumrevolution jj you are amazing", "in_reply_to_status": 684071462068568064, "in_reply_to_user": 87288635, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 87288635 }}, "user": { "id": 1850475632, "name": "shake junt", "screen_name": "crushvolcano", "lang": "en", "location": "Olympia, WA", "create_at": date("2013-09-09"), "description": "fern///INFP http://foxfern.itch.io *BlackLivesMatter *FuckCops #ACAB #AHAG icon by @Opiumrevolution", "followers_count": 761, "friends_count": 211, "statues_count": 15333 }, "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-04T10:00:17.000Z"), "id": 684071870031880192, "text": "a novinha experiente já nasceu c esse dom", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 449288862, "name": "alexandre m ⚡️", "screen_name": "marchesan_iapi", "lang": "pt", "location": "Porto Alegre", "create_at": date("2011-12-28"), "description": "Hello, I'm skater, I have 15 years and do funny videos. ⚡️ SKATE TODO SANTO DIA ⚡️ @justinbieber", "followers_count": 1124, "friends_count": 456, "statues_count": 13981 }, "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-04T10:00:17.000Z"), "id": 684071870140829696, "text": "Ima money getter yo bitch is a pussy popper", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 297736550, "name": "Rell", "screen_name": "yeahterell", "lang": "en", "location": "Dallas, Tx", "create_at": date("2011-05-12"), "description": "Never gone lose. | SC: yeah_terell | #MCE", "followers_count": 845, "friends_count": 388, "statues_count": 68658 }, "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-04T10:00:17.000Z"), "id": 684071870170296324, "text": "Jk CAUSE NOBODY WANTS YOU. Lying 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": 411864180, "name": "Jessica", "screen_name": "jessicapatex", "lang": "en", "location": "CUAA", "create_at": date("2011-11-13"), "description": "Go Cardinals ⚫️❤️⚪️ #CUAABOWLING", "followers_count": 844, "friends_count": 396, "statues_count": 37130 }, "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-01-04T10:00:17.000Z"), "id": 684071870186954752, "text": "temperature up 57°F -> 61°F\nhumidity down 48% -> 42%\nwind 5mph -> 9mph\npressure 29.91in rising", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.18628,33.53858"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 139549542, "name": "Glendale Weather", "screen_name": "_GlendaleAZ", "lang": "en", "location": "Glendale, AZ", "create_at": date("2010-05-02"), "description": "Weather updates, forecast, warnings and information for Glendale, AZ. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 79, "friends_count": 1, "statues_count": 25298 }, "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": 427820, "cityName": "Glendale" } }
+{ "create_at": datetime("2016-01-04T10:00:17.000Z"), "id": 684071870187081728, "text": ".@eyeonannapolis Surprisingly, I went to .@Factors_Row many times, paid w/cc, no one batted an eye! Sign would look great in my media room!", "in_reply_to_status": 684071273677352960, "in_reply_to_user": 30227658, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 30227658, 2310536052 }}, "user": { "id": 54382606, "name": "Factor One", "screen_name": "dafactor1", "lang": "en", "location": "USA", "create_at": date("2009-07-06"), "description": "Husband. Conservative. Former Military Officer. Occasional Barista. Consultant and Advisor for hire.", "followers_count": 201, "friends_count": 186, "statues_count": 7078 }, "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-01-04T10:00:17.000Z"), "id": 684071870312919040, "text": "If You're Reading This You Changed My Life. @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": 448292065, "name": "Renaeee", "screen_name": "TigerRenae", "lang": "en", "location": "Oklahoma", "create_at": date("2011-12-27"), "description": "Festivals make the world happy. I love God and Trap Music. I'm alright and I'm always getting better.", "followers_count": 455, "friends_count": 401, "statues_count": 4276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morris, OK", "id": "0113c3c316159f41", "name": "Morris", "place_type": "city", "bounding_box": rectangle("-95.890341,35.595859 -95.850986,35.617195") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40111, "countyName": "Okmulgee", "cityID": 4049400, "cityName": "Morris" } }
+{ "create_at": datetime("2016-01-04T10:00:17.000Z"), "id": 684071870392569858, "text": "#Treasury Do Something Today by Gail from RockSpringsVintage https://t.co/4P4wCJHTpF via @Etsy #etsyspecialt #art https://t.co/m8j08KRZ9p", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Treasury", "etsyspecialt", "art" }}, "user_mentions": {{ 11522502 }}, "user": { "id": 1268368296, "name": "SylCameoJewelsStore", "screen_name": "SOCaldwell2", "lang": "en", "location": "Cape Coral, Florida", "create_at": date("2013-03-14"), "description": "Come visit our Etsy #Gift Store we sell & buy #Vintage #Jewelry #Collectibles #painting #artwork #handmade @cameojewels13 https://t.co/4fUXIbgGfS", "followers_count": 1402, "friends_count": 1575, "statues_count": 18137 }, "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-04T10:00:18.000Z"), "id": 684071870447013888, "text": "#HR #Job alert: Human Resources Manager - Corporate | ZF TRW | #Livonia, MI https://t.co/XYxXH2Zbdl #Veterans #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.3527097,42.36837"), "retweet_count": 0, "lang": "de", "is_retweet": false, "hashtags": {{ "HR", "Job", "Livonia", "Veterans", "Jobs", "Hiring" }}, "user": { "id": 23130397, "name": "TMJ-DTW HR Jobs", "screen_name": "tmj_dtw_hr", "lang": "en", "location": "Detroit, MI", "create_at": date("2009-03-06"), "description": "Follow this account for geo-targeted Human Resources job tweets in Detroit, MI. Need help? Tweet us at @CareerArc!", "followers_count": 387, "friends_count": 294, "statues_count": 19 }, "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-01-04T10:00:18.000Z"), "id": 684071870476324864, "text": "https://t.co/x2IJ45hnv2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2323196605, "name": "Sonny C.", "screen_name": "scheung0317", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-02-01"), "description": "Liberal Foodie, Helping the less fortunate, You must be shapeless, formless, like water. When you put water in a cup, it become the cup.", "followers_count": 8, "friends_count": 33, "statues_count": 83 }, "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-04T10:00:18.000Z"), "id": 684071870488969216, "text": "@TarunBarnes I called Pie not you", "in_reply_to_status": 684071748195627008, "in_reply_to_user": 520608600, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 520608600 }}, "user": { "id": 306290907, "name": "theylovebugg", "screen_name": "BuggMan96", "lang": "en", "location": "Bastrop, LA", "create_at": date("2011-05-27"), "description": "snapchat:buggman96", "followers_count": 1110, "friends_count": 791, "statues_count": 7271 }, "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-04T10:00:18.000Z"), "id": 684071870656806912, "text": "can't promise that things won't be broken but I swear that I will never leavee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 406603592, "name": "kay", "screen_name": "k_scott5", "lang": "en", "location": "hell", "create_at": date("2011-11-06"), "description": "the killer in me is the killer in you", "followers_count": 372, "friends_count": 315, "statues_count": 17216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Griffith, IN", "id": "0b980b7cee278114", "name": "Griffith", "place_type": "city", "bounding_box": rectangle("-87.454444,41.506007 -87.392243,41.567101") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18089, "countyName": "Lake", "cityID": 1830042, "cityName": "Griffith" } }
+{ "create_at": datetime("2016-01-04T10:00:18.000Z"), "id": 684071870988181504, "text": "Log off https://t.co/5XMwWaw0V9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3326004867, "name": "AJ", "screen_name": "OBJumperr", "lang": "en", "location": "Champaign, IL", "create_at": date("2015-06-14"), "description": "RIP Uncle Josh, you were the realist. #IDontAnswerToFeminists.", "followers_count": 1050, "friends_count": 185, "statues_count": 415 }, "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-04T10:00:18.000Z"), "id": 684071871063588865, "text": "I feel bad for every last person that fucked me over then and now.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 286989287, "name": "tiara...❣", "screen_name": "__lovetiara", "lang": "en", "location": "Baltimore, MD", "create_at": date("2011-04-23"), "description": "proud muva❤️ your Nubian Queen", "followers_count": 2636, "friends_count": 1828, "statues_count": 117845 }, "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-04T10:00:18.000Z"), "id": 684071871084625920, "text": "It's crazy how the people you known from middle school are going to jail for homicide...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3243780945, "name": "arriii✨", "screen_name": "arrisvfri", "lang": "en", "location": "null", "create_at": date("2015-05-09"), "description": "null", "followers_count": 140, "friends_count": 142, "statues_count": 2020 }, "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-04T10:00:18.000Z"), "id": 684071871197782016, "text": "Awe yay!!!! I can't wait. ��☺️���� https://t.co/6ViU6fs0KM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 256234554, "name": "NAJ", "screen_name": "YessIts_Moi", "lang": "en", "location": "null", "create_at": date("2011-02-22"), "description": "#OU18 Aspiring Sports Agent", "followers_count": 1072, "friends_count": 963, "statues_count": 23085 }, "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-04T10:00:18.000Z"), "id": 684071871197933568, "text": "@Marc_Bertrand did you hear sims throw plaudits at Peyton for taking a sack?? ����", "in_reply_to_status": -1, "in_reply_to_user": 56886358, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 56886358 }}, "user": { "id": 22533571, "name": "Chris Pollone", "screen_name": "ChrisPollone", "lang": "en", "location": "40.7501, -73.9698", "create_at": date("2009-03-02"), "description": "National TV News Correspondent | Love the Peacock | Cuse | Bleed Dirty Water & Orange | I tweet about sports. A lot. | BOS ~ SYR~ JAN ~ BHM ~ NYC", "followers_count": 3417, "friends_count": 3312, "statues_count": 50205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midtown East, Manhattan", "id": "780479db3d3354b2", "name": "Midtown East", "place_type": "neighborhood", "bounding_box": rectangle("-73.976074,40.747134 -73.964383,40.755859") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-04T10:00:18.000Z"), "id": 684071871231307776, "text": "I forgot my head phones ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 406201086, "name": "Prototype ☯", "screen_name": "_Trinara", "lang": "en", "location": " forest hill dr, Cadillactica ", "create_at": date("2011-11-06"), "description": "proverbs 31 unapologetic carefree spiritual being RIH lil snupe oh and I block #freetrony #justice4six\n#blacklivesmater #wewillshootback", "followers_count": 969, "friends_count": 743, "statues_count": 27716 }, "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.069786,31.639828") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4876000, "cityName": "Waco" } }
+{ "create_at": datetime("2016-01-04T10:00:18.000Z"), "id": 684071871503990785, "text": "@KaitlynHunter7 Thank You �� Kaitlyn", "in_reply_to_status": -1, "in_reply_to_user": 923744232, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 923744232 }}, "user": { "id": 22189163, "name": "Dr Carl Metzger", "screen_name": "ClearMindedness", "lang": "en", "location": "Bend, OR", "create_at": date("2009-02-27"), "description": "ClearMindedness phone talking/books-917.675.0781 http://ClearMindedness.com", "followers_count": 38917, "friends_count": 33259, "statues_count": 230192 }, "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-01-04T10:00:18.000Z"), "id": 684071871952752642, "text": "Morning clouds/afternoon sun this afternoon, high 34 (1 C). Low 23 (-5 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433220, "name": "St. Louis Weather", "screen_name": "StLouisWeather", "lang": "en", "location": "St. Louis Metro Area", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for St. Louis Metro Area", "followers_count": 3677, "friends_count": 88, "statues_count": 7713 }, "place": { "country": "United States", "country_code": "United States", "full_name": "St. Louis, MO", "id": "60e6df5778ff9dac", "name": "St. Louis", "place_type": "city", "bounding_box": rectangle("-90.320515,38.531852 -90.166409,38.774346") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29510, "countyName": "St. Louis", "cityID": 2965000, "cityName": "St. Louis" } }
+{ "create_at": datetime("2016-01-04T10:00:18.000Z"), "id": 684071872179245056, "text": "Can you recommend anyone for this #job? RN - https://t.co/BZXSXT05NC #RN #Westfield, MA #Nursing #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.7592109,42.11807"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "RN", "Westfield", "Nursing", "Hiring" }}, "user": { "id": 2688167298, "name": "Noble Hospital Jobs", "screen_name": "JobsatNobleHosp", "lang": "en", "location": "Westfield, MA", "create_at": date("2014-07-28"), "description": "Noble Hospital is a regional community hospital, providing a broad range of expert services to Greater Westfield and surrounding areas.", "followers_count": 17, "friends_count": 0, "statues_count": 94 }, "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-01-04T10:00:18.000Z"), "id": 684071872225349632, "text": "Temp 33.8°F Rising, Pressure 30.563in Falling slowly, Dew point 20.6°, Wind NE 1mph, Rain today 0.00in https://t.co/gPtnr2dDtV #STLwx #MOwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.35555556,38.47694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "STLwx", "MOwx" }}, "user": { "id": 1191304230, "name": "Oakville Weather", "screen_name": "OakvilleWx", "lang": "en", "location": "Oakville, Missouri, USA", "create_at": date("2013-02-17"), "description": "Current conditions on the hour brought to you by @DaveStLou", "followers_count": 104, "friends_count": 80, "statues_count": 24600 }, "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-01-04T10:00:18.000Z"), "id": 684071872535728128, "text": "@tsunamichelle You're the damn best.", "in_reply_to_status": 684053551136419841, "in_reply_to_user": 216966248, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 216966248 }}, "user": { "id": 2257133250, "name": "Jay", "screen_name": "thejesusmanzano", "lang": "en", "location": "UCLA", "create_at": date("2013-12-21"), "description": "We are government. UCLA '19. Hillary 2016", "followers_count": 205, "friends_count": 165, "statues_count": 10735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:18.000Z"), "id": 684071872825147392, "text": "Nathan Deal rescinds order attempting to ban Syrian refugees from Georgia: https://t.co/T7HVY1mkB7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 63251803, "name": "ARC", "screen_name": "AtlantaRegional", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-08-05"), "description": "The Atlanta Regional Commission (ARC) is the planning agency for the 10 county Atlanta metropolitan region.", "followers_count": 5803, "friends_count": 1800, "statues_count": 7473 }, "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-04T10:00:18.000Z"), "id": 684071872988774400, "text": "*BM https://t.co/3EfE65kydu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 305055117, "name": "SkinnyTheLongShaft", "screen_name": "DopeOG_Skinny", "lang": "en", "location": "infinite tsukuyomi ", "create_at": date("2011-05-25"), "description": "R.I.P. Braylon Dion King I•VII•MMXII SC: brandon_skinny @whataburger Loves Me... best friend is @eismerci", "followers_count": 1132, "friends_count": 755, "statues_count": 112865 }, "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-04T10:00:18.000Z"), "id": 684071873009684480, "text": "I lost my mom in the store. I guess I'm an orphan now.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 750355345, "name": "Ali", "screen_name": "Ali_Wright22", "lang": "en", "location": "Laquey", "create_at": date("2012-08-10"), "description": "Professional napper and dinosaur enthusiast", "followers_count": 431, "friends_count": 398, "statues_count": 3539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Leonard Wood, MO", "id": "6cccdaa30057d3d1", "name": "Fort Leonard Wood", "place_type": "city", "bounding_box": rectangle("-92.156058,37.692152 -92.081902,37.796725") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29169, "countyName": "Pulaski", "cityID": 2925264, "cityName": "Fort Leonard Wood" } }
+{ "create_at": datetime("2016-01-04T10:00:18.000Z"), "id": 684071873009717248, "text": "THE BACHELOR STARTS TONIGHT I REPEAR THE BACHELOR STARTS TONIGHT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.0089056,38.59189651"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 379470591, "name": "Tierra Fulton", "screen_name": "_tierrafulton", "lang": "en", "location": "IG + SC: tierra.fulton ", "create_at": date("2011-09-24"), "description": "God first | veni vidi vici | love yourself, I dare you", "followers_count": 630, "friends_count": 609, "statues_count": 62096 }, "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-01-04T10:00:18.000Z"), "id": 684071873676640256, "text": "JTFO���� a whole buck em up fr https://t.co/Ux5kkIu2qT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155587586, "name": "cee.", "screen_name": "carolynb__", "lang": "en", "location": "somewhere between joy and pain", "create_at": date("2010-06-14"), "description": "I love u... #Pocahontaswashere❤️", "followers_count": 751, "friends_count": 604, "statues_count": 29800 }, "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-04T10:00:18.000Z"), "id": 684071873798258689, "text": "Eric is already growing a mustache and he's 5 like dang took Mario almost to middle school/high school to grow one", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 450837671, "name": "karina lizeth", "screen_name": "Liz_Medina13", "lang": "en", "location": "chickasha, ok", "create_at": date("2011-12-30"), "description": "ig: karina_lizeth || sc: karina-lizeth", "followers_count": 1417, "friends_count": 791, "statues_count": 23863 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chickasha, OK", "id": "ba083f1f28634b26", "name": "Chickasha", "place_type": "city", "bounding_box": rectangle("-97.979369,34.992737 -97.917193,35.07315") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40051, "countyName": "Grady", "cityID": 4013950, "cityName": "Chickasha" } }
+{ "create_at": datetime("2016-01-04T10:00:18.000Z"), "id": 684071873823375360, "text": "@AllenandEnvy @DamagedRec @marksherry @ExchangeLA @jordansuckley Gunna be a good one lads!! ������", "in_reply_to_status": 684066662254010370, "in_reply_to_user": 236806784, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 236806784, 2295843860, 127476717, 126769527, 72912912 }}, "user": { "id": 331931554, "name": "The Red Ranger", "screen_name": "RangerDeRojo", "lang": "en", "location": "Los Angeles", "create_at": date("2011-07-08"), "description": "Ibiza Trance Family Resident::Underground Trance DJ::#SickFuckArmy::TranceHead::\nTranceFamily::Power Ranger::Ibiza Raver::Glover::Lover\n IG- Ronniejoe", "followers_count": 1436, "friends_count": 980, "statues_count": 43889 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:18.000Z"), "id": 684071874050048006, "text": "@ABO3UDAY @Aldowayesh فعلا سجل لاعبين لكن بنظام سجل عالحساب مافيه كاش هياطه هو الكاش فقط", "in_reply_to_status": 684069478087397378, "in_reply_to_user": 323226571, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user_mentions": {{ 323226571, 511868279 }}, "user": { "id": 613554907, "name": "خالد ١٦ النصر", "screen_name": "khalidALnassr", "lang": "ar", "location": "الرياض", "create_at": date("2012-06-20"), "description": "كل جماهير العالم تشجع إلا جمهور النصر فهو عاشق", "followers_count": 281, "friends_count": 105, "statues_count": 6901 }, "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-04T10:00:18.000Z"), "id": 684071874167353344, "text": "About to go spend some money ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2278507099, "name": "Monica Vint", "screen_name": "MonicaVint", "lang": "en", "location": "null", "create_at": date("2014-01-05"), "description": "you do you boo", "followers_count": 717, "friends_count": 775, "statues_count": 10087 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rockwood, MI", "id": "2cac8a296639933d", "name": "Rockwood", "place_type": "city", "bounding_box": rectangle("-83.269999,42.042057 -83.20333,42.083904") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2669180, "cityName": "Rockwood" } }
+{ "create_at": datetime("2016-01-04T10:00:18.000Z"), "id": 684071874234572800, "text": "I picked up my glass candle holder and it just shattered in my hands WHY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 553874647, "name": "mellie", "screen_name": "MELABOO98", "lang": "en", "location": "the nawfside", "create_at": date("2012-04-14"), "description": "my heart is gold and my hands are cold", "followers_count": 600, "friends_count": 584, "statues_count": 4245 }, "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-01-04T10:00:18.000Z"), "id": 684071874322550786, "text": "Yup. Weird af. �� I'm just jelly cos I can't swim for shiet ���� https://t.co/nEefOTj9Rd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1097403349, "name": "The 4th Hokage ⚡️", "screen_name": "JalizScanlan", "lang": "en", "location": "Utah", "create_at": date("2013-01-16"), "description": "One wife | One son | Samoan | Don't mistake my kindness for weakness. I'm just being nice.", "followers_count": 558, "friends_count": 531, "statues_count": 8895 }, "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-04T10:00:18.000Z"), "id": 684071874373005314, "text": "I hate everyone who wished away the warm weather because I hate the cold", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 513547130, "name": "Erin Garris", "screen_name": "eringarris", "lang": "en", "location": "Columbus, OH", "create_at": date("2012-03-03"), "description": "null", "followers_count": 556, "friends_count": 351, "statues_count": 3129 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, OH", "id": "bec4732d440b4c03", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-82.670558,39.681924 -82.522696,39.792026") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39045, "countyName": "Fairfield", "cityID": 3941720, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-01-04T10:00:18.000Z"), "id": 684071874528063488, "text": "Partly cloudy this afternoon, high 33 (1 C). Low 25 (-4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433608, "name": "Kansas City Weather", "screen_name": "KansasCityWX", "lang": "en", "location": "Kansas City Metro Area", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Kansas City Metro Area", "followers_count": 1932, "friends_count": 88, "statues_count": 7833 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "ddabe75a8f44fe5a", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.765917,38.827176 -94.385522,39.356662") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-01-04T10:00:19.000Z"), "id": 684071874649702401, "text": "@RickyBaby321 @darren_dazmav The U.S.A. even more so :-(", "in_reply_to_status": 684070782033592320, "in_reply_to_user": 218246836, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 218246836, 2903638510 }}, "user": { "id": 1631877504, "name": "Melanie", "screen_name": "JustMel1963", "lang": "en", "location": "Santa Rosa Beach, FL", "create_at": date("2013-07-29"), "description": "#WorldPeaceAdvocate #AllChildrenMatter #ChristConsciousness #Ascension #AddictionRecovery #GlobalAwakening #SpiritualRevolution #NESARA", "followers_count": 8683, "friends_count": 8525, "statues_count": 110092 }, "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-01-04T10:00:19.000Z"), "id": 684071874779688960, "text": "Can you recommend anyone for this #SupplyChain #job? https://t.co/MGjBG5PK56 #NOVI, MI #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.4754913,42.48059"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupplyChain", "job", "NOVI", "Hiring" }}, "user": { "id": 126380751, "name": "Ryder Jobs", "screen_name": "Ryderjobs", "lang": "en", "location": "null", "create_at": date("2010-03-25"), "description": "Ryder System, Inc. is a FORTUNE 500 provider of transportation and supply chain management solutions. View our job openings and Get All You Need to Succeed.", "followers_count": 750, "friends_count": 8, "statues_count": 5606 }, "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-01-04T10:00:19.000Z"), "id": 684071874951811072, "text": "I wonder what's gonna happen when black twitter gets ahold of this lol https://t.co/AdPseNN2EE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 111934148, "name": "M", "screen_name": "biebsusreedus", "lang": "en", "location": "Indiana", "create_at": date("2010-02-06"), "description": "Justin followed 8.12.14 | Justin followed on shots 9.26.14 | I kinda have a thing for norman reedus", "followers_count": 1607, "friends_count": 2004, "statues_count": 22562 }, "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-04T10:00:19.000Z"), "id": 684071875161407488, "text": "take care of your lady and she gonna take care 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": 2375829356, "name": "Santana Gomez", "screen_name": "santanamusicc", "lang": "en", "location": "Glendale,AZ", "create_at": date("2014-03-06"), "description": "Music Artist, Music Producer, Vocalist, Performer email: santanamusic04@gmail.com", "followers_count": 456, "friends_count": 285, "statues_count": 4687 }, "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-04T10:00:19.000Z"), "id": 684071875195105281, "text": "https://t.co/CyNz18NvBZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2799256175, "name": "Kim", "screen_name": "kimschafferrr", "lang": "en", "location": "traw❤️", "create_at": date("2014-10-01"), "description": "NJ/21/insta-kimschafferr", "followers_count": 262, "friends_count": 421, "statues_count": 7653 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roselle Park, NJ", "id": "bcab414422facfa5", "name": "Roselle Park", "place_type": "city", "bounding_box": rectangle("-74.285711,40.657943 -74.24661,40.67251") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3464650, "cityName": "Roselle Park" } }
+{ "create_at": datetime("2016-01-04T10:00:19.000Z"), "id": 684071875329261569, "text": "Ok https://t.co/ztSPBmJCHb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 382844392, "name": "Lola morzan", "screen_name": "HENNYwork_", "lang": "en", "location": "NYC", "create_at": date("2011-09-30"), "description": "IG: Lolamorzan #FrenchMontana.", "followers_count": 319, "friends_count": 283, "statues_count": 10837 }, "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-04T10:00:19.000Z"), "id": 684071875639558144, "text": "@Sophie_FosterXx i thought you were going to uni???", "in_reply_to_status": 684070919959113729, "in_reply_to_user": 234550364, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 234550364 }}, "user": { "id": 2448240463, "name": "Mary☀️", "screen_name": "marydavidow", "lang": "en", "location": "Minnesota", "create_at": date("2014-04-16"), "description": "Sometimes I pretend to be normal.It gets boring. So i go back to being me. 15yr old photographer,artist, professinal fangirl.", "followers_count": 321, "friends_count": 286, "statues_count": 3498 }, "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-01-04T10:00:19.000Z"), "id": 684071875740319744, "text": ":)))) https://t.co/w8uwZlB04P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1022336245, "name": "Allison Hayes", "screen_name": "AKayHay", "lang": "en", "location": "Seymour, IN ", "create_at": date("2012-12-19"), "description": "Indiana University Women's Rowing, SHSSD alum & daughter of the one true King", "followers_count": 732, "friends_count": 561, "statues_count": 4192 }, "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-04T10:00:19.000Z"), "id": 684071875819995138, "text": "I need to get my shit together", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 351136763, "name": "Autumn Lockwood", "screen_name": "autumn_lockwood", "lang": "en", "location": "southgate", "create_at": date("2011-08-08"), "description": "null", "followers_count": 749, "friends_count": 374, "statues_count": 25224 }, "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-01-04T10:00:19.000Z"), "id": 684071875861987328, "text": "������������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 494573891, "name": "Caitlin Stewart", "screen_name": "caitlinmstew", "lang": "en", "location": "Demopolis, AL", "create_at": date("2012-02-16"), "description": "Colby's mommy", "followers_count": 580, "friends_count": 629, "statues_count": 6154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Demopolis, AL", "id": "5b362c677d8f87fb", "name": "Demopolis", "place_type": "city", "bounding_box": rectangle("-87.86969,32.487703 -87.795508,32.525519") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1091, "countyName": "Marengo", "cityID": 120296, "cityName": "Demopolis" } }
+{ "create_at": datetime("2016-01-04T10:00:19.000Z"), "id": 684071875920683008, "text": "So glad I got to celebrate kirbs birthday with some cool people. Last night was rad. Hopefully… https://t.co/c3dTtUnSsH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.94283657,35.73526979"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 177719317, "name": "Marcelyn", "screen_name": "ItsJustMarcelyn", "lang": "en", "location": "Franklin, TN ", "create_at": date("2010-08-12"), "description": "not your typical black girl| Isaiah 42:16 gets me through each day|", "followers_count": 471, "friends_count": 381, "statues_count": 13581 }, "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-01-04T10:00:19.000Z"), "id": 684071876277092352, "text": "@samsabri Lolz. That makes it less weird. Happy flying ✈️✈️✈️", "in_reply_to_status": 684071708827844608, "in_reply_to_user": 14969511, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14969511 }}, "user": { "id": 14662086, "name": "Amanda V. Duncan", "screen_name": "avduncan", "lang": "en", "location": "Born Montanan. Live in Seattle", "create_at": date("2008-05-05"), "description": "Media relations for @Microsoft | Very into 3D Printing | New Media Geek | Always looking for small batch bourbon recommendations", "followers_count": 1018, "friends_count": 624, "statues_count": 4227 }, "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-04T10:00:19.000Z"), "id": 684071876759400448, "text": "Holidays are over, back to the school routine��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 101641190, "name": "Val", "screen_name": "ValerieMacias24", "lang": "en", "location": "null", "create_at": date("2010-01-03"), "description": "null", "followers_count": 900, "friends_count": 502, "statues_count": 27289 }, "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-04T10:00:19.000Z"), "id": 684071877166276608, "text": "Now I gotta go to Reagan", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 40495544, "name": "K $", "screen_name": "KirstinKianaaa", "lang": "en", "location": "HTX", "create_at": date("2009-05-16"), "description": "sparkle & shine on dez hoes", "followers_count": 1800, "friends_count": 1797, "statues_count": 48344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-04T10:00:19.000Z"), "id": 684071877359239168, "text": "Baby and slim still point guard and center...so much money on my mind it's all I remember...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 286454615, "name": "Justin Henderson", "screen_name": "2Love_4Life", "lang": "en", "location": "null", "create_at": date("2011-04-22"), "description": "College Park Born and Raised...Proud Pops...I do it for Kaidyn and Sahni...Rip to my lil bro Tre Tre✌️❤️ Family First...#Justo...#MilesCollege Student Athlete", "followers_count": 788, "friends_count": 804, "statues_count": 16085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Union City, GA", "id": "d1683f3c5c646d61", "name": "Union City", "place_type": "city", "bounding_box": rectangle("-84.597615,33.542538 -84.485509,33.633988") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1378324, "cityName": "Union City" } }
+{ "create_at": datetime("2016-01-04T10:00:19.000Z"), "id": 684071877610848257, "text": "Protection 1 #Utilities #Job: Residential Installer I (#Portland, ME) https://t.co/xRonPpvnA3 #Jobs #Hiring https://t.co/Oideo9Bl43", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.3243182,43.6949452"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Utilities", "Job", "Portland", "Jobs", "Hiring" }}, "user": { "id": 2194610792, "name": "Protection 1 Careers", "screen_name": "Pro1Careers", "lang": "en", "location": "null", "create_at": date("2013-11-14"), "description": "All of our open positions will be tweeted here! Let's chat about how you can be part of the P1 Family. You can also follow us at @ProtectionOne", "followers_count": 152, "friends_count": 86, "statues_count": 297 }, "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-01-04T10:00:19.000Z"), "id": 684071877694754816, "text": "Want to work in #Lewisville, TX? View our latest opening: https://t.co/l1cggC8wWJ #Hospitality #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.9686,33.009336"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Lewisville", "Hospitality", "Job", "Jobs", "Hiring" }}, "user": { "id": 1648169364, "name": "Raising Cane's Jobs", "screen_name": "CaniacCareers", "lang": "en", "location": "null", "create_at": date("2013-08-05"), "description": "null", "followers_count": 145, "friends_count": 40, "statues_count": 480 }, "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-04T10:00:19.000Z"), "id": 684071877732601856, "text": "@ParksByDoom @HorizonsOne very doubtful. Won't be that impressive. Also, that would require Park Ops to contribute to the cost", "in_reply_to_status": 684071687730630656, "in_reply_to_user": 636211811, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 636211811, 220689884 }}, "user": { "id": 799381514, "name": "ThisFloridaLife", "screen_name": "thisfloridalife", "lang": "en", "location": "Orlando, FL", "create_at": date("2012-09-02"), "description": "Dani. Native Floridian. Love my homestate and exploring my city. Contributing writer @TouringPlans & @OrlandoInformer. Hotelier/ Hospitality geek. #Parrothead", "followers_count": 2659, "friends_count": 1257, "statues_count": 22524 }, "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-04T10:00:19.000Z"), "id": 684071877799575552, "text": "Morning clouds/afternoon sun this afternoon, high 38 (3 C). Low 26 (-3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433972, "name": "Memphis Weather", "screen_name": "MemphisWX", "lang": "en", "location": "Memphis, TN", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Memphis, TN", "followers_count": 1688, "friends_count": 88, "statues_count": 7745 }, "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-04T10:00:19.000Z"), "id": 684071877938024448, "text": "This #Healthcare #job might be a great fit for you: Speech Therapist-Speech Therapy - https://t.co/2oPrW3eveS https://t.co/D4PebZIuxm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.9131935,36.5787155"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job" }}, "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": 54, "friends_count": 27, "statues_count": 50 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monterey, CA", "id": "49af5b43d4963f4c", "name": "Monterey", "place_type": "city", "bounding_box": rectangle("-121.927024,36.573741 -121.830974,36.618826") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 648872, "cityName": "Monterey" } }
+{ "create_at": datetime("2016-01-04T10:00:19.000Z"), "id": 684071878298853376, "text": "soda https://t.co/tztVJ50p0D", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1369768069, "name": "president isaac", "screen_name": "isaacmeep", "lang": "en", "location": "Maryland, USA", "create_at": date("2013-04-21"), "description": "capital letters overrated. ❤10-14-15. running for president #isaac2036. college sports are trash.", "followers_count": 1529, "friends_count": 1175, "statues_count": 51167 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenbelt, MD", "id": "30f7f566cd810031", "name": "Greenbelt", "place_type": "city", "bounding_box": rectangle("-76.915629,38.971715 -76.858787,39.015363") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2434775, "cityName": "Greenbelt" } }
+{ "create_at": datetime("2016-01-04T10:00:19.000Z"), "id": 684071878432956416, "text": "@Migos @brinnabbyy RT bby", "in_reply_to_status": 683917199191310336, "in_reply_to_user": 258601468, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 353029369, 3388069324 }}, "user": { "id": 258601468, "name": "$$", "screen_name": "YazzySays", "lang": "en", "location": "where the bands at $$$$", "create_at": date("2011-02-27"), "description": "Ya main bitch wanna be me... | Freak hoe prez | IG : _bazzyyyyy", "followers_count": 2239, "friends_count": 1329, "statues_count": 104144 }, "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-04T10:00:20.000Z"), "id": 684071878852362240, "text": "���������� https://t.co/mI2os3EOFD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "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": 257, "friends_count": 182, "statues_count": 11072 }, "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-04T10:00:20.000Z"), "id": 684071879108378624, "text": "#Slapbak #truliodisgracias #truliodisgracious #DaMuttTour #MizzBrooks #octavernsanclemente… https://t.co/S6VLwDyM1c", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.6003799,33.4129066"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Slapbak", "truliodisgracias", "truliodisgracious", "DaMuttTour", "MizzBrooks", "octavernsanclemente" }}, "user": { "id": 58980889, "name": "SlapBak", "screen_name": "SlapBak", "lang": "en", "location": "Orange County, California", "create_at": date("2009-07-21"), "description": "New Single, Heartbreaker, out now http://www.youtube.com/watch?v=0iG5CjQmD6o", "followers_count": 316, "friends_count": 722, "statues_count": 729 }, "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-01-04T10:00:20.000Z"), "id": 684071879263424512, "text": "This #security #job might be a great fit for you: Security Consultant - https://t.co/GhuvXAiNsm #infosec #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": {{ "security", "job", "infosec", "Chicago", "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": "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-04T10:00:20.000Z"), "id": 684071879276163072, "text": "Daisy's an eager-to-please pup who LOVES belly rubs! Meet her at our NEast adoption center! https://t.co/UKJ0Gk4CNq https://t.co/ihcNdEhsuu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21013686, "name": "PAWS", "screen_name": "phillypaws", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2009-02-16"), "description": "Dedicated to saving Philadelphia's homeless, abandoned, and unwanted animals. Join us! They won't make it without you. Adopt. Donate. Foster. Volunteer.", "followers_count": 6451, "friends_count": 487, "statues_count": 9064 }, "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-04T10:00:20.000Z"), "id": 684071879326355458, "text": "I can't get over this photo.. It makes me laugh every time I see it.. Which is why it's on my work desk. #auntielife https://t.co/Om0MQAPaKL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "auntielife" }}, "user": { "id": 192615461, "name": "Jillian Anderson", "screen_name": "Jillie_Alexis", "lang": "en", "location": "Colorado Springs, CO", "create_at": date("2010-09-19"), "description": "I make TV & sometimes I'm on it: Host @sococw @fox21news The Bachelor 19 & BIP 2. Former Fox-Newser & Redskin Ambasador | IG: @JillieAlexis Snap: @Jillie_Alexis", "followers_count": 19734, "friends_count": 1257, "statues_count": 15192 }, "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-04T10:00:20.000Z"), "id": 684071879406174208, "text": "@Its_Danny_G @Purse_ you have rainbow six for Xbox one?", "in_reply_to_status": 684071064629080066, "in_reply_to_user": 977220504, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 977220504, 544476139 }}, "user": { "id": 427639959, "name": "Justin", "screen_name": "JayMintz96", "lang": "en", "location": "null", "create_at": date("2011-12-03"), "description": "Park Central\nGuitarist/singer/songwriter, Manhattanville 18' Music Education\nH.K.W ❤", "followers_count": 170, "friends_count": 190, "statues_count": 1817 }, "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-04T10:00:20.000Z"), "id": 684071879573807104, "text": "@tweetria pataka chai. \"Gazab ka swaad \" ������", "in_reply_to_status": 684071713991168001, "in_reply_to_user": 68629264, "favorite_count": 0, "retweet_count": 0, "lang": "hi", "is_retweet": false, "user_mentions": {{ 68629264 }}, "user": { "id": 235845512, "name": "Saurabh", "screen_name": "FloydMalhar", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-01-08"), "description": "Transportation Design Student | Art Center College of Design | loves pun but not a pun-Jabi | good fingering skills on guitar. okayish actually.", "followers_count": 602, "friends_count": 399, "statues_count": 57948 }, "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-04T10:00:20.000Z"), "id": 684071879586365440, "text": "CVS Health: Retail Store Positions (#Madison, MS) https://t.co/MQutNNRM1C #Retail #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.1103716,32.4633647"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Madison", "Retail", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 59967434, "name": "TMJ-MS Retail Jobs", "screen_name": "tmj_ms_retail", "lang": "en", "location": "Mississippi", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in Mississippi Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 449, "friends_count": 299, "statues_count": 654 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madison, MS", "id": "0041984014acb103", "name": "Madison", "place_type": "city", "bounding_box": rectangle("-90.205184,32.426057 -90.062166,32.522579") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28089, "countyName": "Madison", "cityID": 2844520, "cityName": "Madison" } }
+{ "create_at": datetime("2016-01-04T10:00:20.000Z"), "id": 684071879594930181, "text": "@DrewMTips @baseballcrank Meh. No one cares.", "in_reply_to_status": 684071064729686016, "in_reply_to_user": 29090846, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 29090846, 38021361 }}, "user": { "id": 18034072, "name": "Don Surber", "screen_name": "donsurber", "lang": "en", "location": "Poca WV", "create_at": date("2008-12-10"), "description": "Retired. Author of EXCEPTIONAL AMERICANS 1 http://t.co/VVsXoqxSM9 EXCEPTIONAL AMERICANS 2: THE CAPITALISTS http://t.co/IQYwFlhQnO", "followers_count": 2807, "friends_count": 82, "statues_count": 30348 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Poca, WV", "id": "001a9f0813d7a82f", "name": "Poca", "place_type": "city", "bounding_box": rectangle("-81.825749,38.43513 -81.799353,38.490068") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54079, "countyName": "Putnam", "cityID": 5464516, "cityName": "Poca" } }
+{ "create_at": datetime("2016-01-04T10:00:20.000Z"), "id": 684071879615864832, "text": "Well good fucking morning ha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 464656712, "name": "Jules", "screen_name": "bakerbuddy1", "lang": "en", "location": "null", "create_at": date("2012-01-15"), "description": "You can't sit with us, you can't hit the blunt.", "followers_count": 463, "friends_count": 203, "statues_count": 15530 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rochester, IN", "id": "cf7036581a451292", "name": "Rochester", "place_type": "city", "bounding_box": rectangle("-86.24189,41.038581 -86.159562,41.080211") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18049, "countyName": "Fulton", "cityID": 1865214, "cityName": "Rochester" } }
+{ "create_at": datetime("2016-01-04T10:00:20.000Z"), "id": 684071879720566784, "text": "I was at the airport the other day & mfs was walking around with no fucking jacket !!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 162846385, "name": "Lil Lexus #GAState17", "screen_name": "CarameLexx", "lang": "en", "location": "︻╦╤─ChoppaCity 2ATL longliveJR", "create_at": date("2010-07-04"), "description": "@trustn0__one ❤️ Petite . 5'3 . Rarity . Nursing Student . #GreenFetish .", "followers_count": 2098, "friends_count": 1385, "statues_count": 55607 }, "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-01-04T10:00:20.000Z"), "id": 684071880291160065, "text": "When the MSU ticket office refunds your ticket reservation for the championship game and the L becomes more real https://t.co/gaWdpqboFD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1043279756, "name": "Lauren L. Zanger", "screen_name": "LaurenZanger", "lang": "en", "location": "null", "create_at": date("2012-12-28"), "description": "BA in Communication from Michigan State University. Ann Arbor native. BTN StudentU. Sports fanatic, human rights activist.", "followers_count": 188, "friends_count": 314, "statues_count": 2570 }, "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-04T10:00:20.000Z"), "id": 684071880526004225, "text": "@shaydeez_ pure facts", "in_reply_to_status": 684071561415016448, "in_reply_to_user": 2203938523, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2203938523 }}, "user": { "id": 286964296, "name": "BLM", "screen_name": "___TC_", "lang": "en", "location": "Drexel University ", "create_at": date("2011-04-23"), "description": "RIP Johnny Wyche Sunset: 8/9/2015", "followers_count": 1036, "friends_count": 909, "statues_count": 97028 }, "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-04T10:00:20.000Z"), "id": 684071880530067456, "text": "I've Been Binge Watching Supernatural For About 4 Days Now & I'm Only On Season 4 & I Have 7 Seasons 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": 2936152083, "name": "Marshall Foee", "screen_name": "LethalBanger10", "lang": "en", "location": "Decatur Il.", "create_at": date("2014-12-21"), "description": "Free NumbaSeven , Quann, & Tank Love Peace ✌ & Chicken Grease R.I.P Marshall William 1930-2009 ☝ SC.. Tunechijr_10", "followers_count": 317, "friends_count": 367, "statues_count": 1857 }, "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-01-04T10:00:20.000Z"), "id": 684071880685432832, "text": "It smells great down here now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 517162013, "name": "carlos soriano", "screen_name": "cscharles89", "lang": "en", "location": "null", "create_at": date("2012-03-06"), "description": "null", "followers_count": 116, "friends_count": 224, "statues_count": 7228 }, "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-01-04T10:00:20.000Z"), "id": 684071880748314625, "text": "Wind 3.0 mph WNW. Barometer 30.015 in, Falling. Temperature 31.9F. Rain today 0.00 in. Humidity 42% | 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": 574, "friends_count": 773, "statues_count": 35274 }, "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-04T10:00:20.000Z"), "id": 684071880844677120, "text": "@bootywranglers if you pay I will lol", "in_reply_to_status": 684070803411869696, "in_reply_to_user": 2236746318, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2236746318 }}, "user": { "id": 191684786, "name": "kynsley☾", "screen_name": "KayyKattt", "lang": "en", "location": "Laguna Beach, CA", "create_at": date("2010-09-16"), "description": "null", "followers_count": 293, "friends_count": 299, "statues_count": 11275 }, "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-04T10:00:20.000Z"), "id": 684071880979013633, "text": "I'm having a mental breakdown", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 321466977, "name": "Nic", "screen_name": "NiCool_Kam", "lang": "en", "location": "null", "create_at": date("2011-06-21"), "description": "7-6-15 Sergey P ❤️ 8-5-15 JT", "followers_count": 845, "friends_count": 865, "statues_count": 12937 }, "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-04T10:00:20.000Z"), "id": 684071881419436032, "text": ".@Theguy923 19. Climate change denial, Daesh/Al Qaeda/the Taliban, and people who don't check their email regularly", "in_reply_to_status": 684067925699899394, "in_reply_to_user": 3310928604, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3310928604 }}, "user": { "id": 605008217, "name": "HECK YEAH", "screen_name": "cptngarlock", "lang": "en", "location": "mechE hell", "create_at": date("2012-06-10"), "description": "rx-0 sj gundam, wannabe bougie, low-art connoisseur pretending it's high-art. not a dog on the internet, i swear.", "followers_count": 415, "friends_count": 396, "statues_count": 30334 }, "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-01-04T10:00:20.000Z"), "id": 684071881670922240, "text": "@ArianaGrande @TheAMAs john lennon hear til wrd amr how aids vic goga mid finger asin heil nirt knod can curry key allahu cor dak rid", "in_reply_to_status": 668218214036312065, "in_reply_to_user": 34507480, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 34507480, 4441580542 }}, "user": { "id": 2830705318, "name": "john lennon heil cur", "screen_name": "wachzitu15", "lang": "en", "location": "nel judith tattonka today ", "create_at": date("2014-10-14"), "description": "little big horn dave bury my hart at wnd knee blackhart wachzitu bigfoot uss wachzitu whittney gila applelusa maghezeick sandknoval scar lodge uss knod cor", "followers_count": 1, "friends_count": 185, "statues_count": 2240 }, "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-04T10:00:20.000Z"), "id": 684071882228953088, "text": "Finally got our apartment looking like it should https://t.co/kw2Ek7UKS4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 223637850, "name": "rhiannon☽", "screen_name": "lauren_sweetser", "lang": "en", "location": "Maine", "create_at": date("2010-12-06"), "description": "•• Free spirit, adventurer, Maine, tattoo addict, rock lover, cactus mother, coffee master, and soul searcher •• Blood and Wine Designs ••", "followers_count": 407, "friends_count": 368, "statues_count": 15340 }, "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-01-04T10:00:20.000Z"), "id": 684071882342174721, "text": "He was fine tho. ��", "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": ". February 12th❤️", "screen_name": "___Shannonnn", "lang": "en", "location": "662 ✈️ 601 Hattiesburg, MS ", "create_at": date("2011-03-10"), "description": "#USM18 .", "followers_count": 1377, "friends_count": 841, "statues_count": 47201 }, "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-01-04T10:00:20.000Z"), "id": 684071882342199296, "text": "Your most difficult times can often lead you to the greatest moments 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": 369593609, "name": "Darrell Logan", "screen_name": "DarrellLogan", "lang": "en", "location": "Niles, Ohio", "create_at": date("2011-09-07"), "description": "Life of ME.. I'm just a fun Lovin',crazy, cool, single.. Happy go lucky kind of guy. Oh yes, with a Heart of Gold! . 21+Only", "followers_count": 267, "friends_count": 473, "statues_count": 6409 }, "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-04T10:00:20.000Z"), "id": 684071882400768000, "text": "�� https://t.co/qgWAy39Cwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1912291542, "name": "...", "screen_name": "SariaJadee", "lang": "en", "location": "null", "create_at": date("2013-09-27"), "description": "null", "followers_count": 1854, "friends_count": 905, "statues_count": 28348 }, "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-04T10:00:20.000Z"), "id": 684071882560286720, "text": "When Flavias been watching Greys a little too much... https://t.co/rcYgJ9F04b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 327178606, "name": "kate mcmanus", "screen_name": "_kateemcmanus", "lang": "en", "location": "null", "create_at": date("2011-06-30"), "description": "• uconn '18 • Αφ • ig & snapchat: katee.mcmanus •", "followers_count": 632, "friends_count": 488, "statues_count": 5905 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Derby, CT", "id": "c2231ed99cad982e", "name": "Derby", "place_type": "city", "bounding_box": rectangle("-73.123799,41.301402 -73.039142,41.352876") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 919480, "cityName": "Derby" } }
+{ "create_at": datetime("2016-01-04T10:00:20.000Z"), "id": 684071882799353856, "text": "THAT FELLY https://t.co/UZqZoZMG2w", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 167705672, "name": "Will Bekolo", "screen_name": "WillyBekk", "lang": "en", "location": "charlotte, nc", "create_at": date("2010-07-17"), "description": "pchs'15 | #uncc19 | Finessin' since '97", "followers_count": 1182, "friends_count": 584, "statues_count": 27216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morrisville, NC", "id": "0122292c5bc9ff29", "name": "Morrisville", "place_type": "city", "bounding_box": rectangle("-78.881022,35.796927 -78.786799,35.870756") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3744520, "cityName": "Morrisville" } }
+{ "create_at": datetime("2016-01-04T10:00:21.000Z"), "id": 684071883109568512, "text": "Join the OhioHealth team! See our latest #Nursing #job opening here: https://t.co/GvQsych0gw #Columbus, OH #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.025396,40.0480476"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "Columbus", "Hiring" }}, "user": { "id": 2923178940, "name": "OhioHealth Jobs", "screen_name": "OhioHealthJobs", "lang": "en", "location": "null", "create_at": date("2014-12-08"), "description": "OhioHealth is a not-for-profit, faith-based health system that has served central Ohio since 1891. Join us! Together WE can make a difference.", "followers_count": 130, "friends_count": 56, "statues_count": 1102 }, "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-04T10:00:21.000Z"), "id": 684071883197698048, "text": "@jjauthor https://t.co/7dimvOW14u", "in_reply_to_status": 683882852430168064, "in_reply_to_user": 177584156, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 177584156 }}, "user": { "id": 1400837041, "name": "carole robinson", "screen_name": "crobinson8279", "lang": "en", "location": "California", "create_at": date("2013-05-03"), "description": "about empowerment , self-reliance. Opportunity for all favoritism to none. Heritage action sentinel. Fortunate to be on the same wave length as hubby", "followers_count": 355, "friends_count": 423, "statues_count": 6357 }, "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-01-04T10:00:21.000Z"), "id": 684071883277385728, "text": "About a half-dozen kids are onstage and a band from @RDGschools is waiting on the sidewalk. https://t.co/Fq8ANG2tLq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2409334200 }}, "user": { "id": 803344344, "name": "Adam Richter/REagle", "screen_name": "AdamRichterRE", "lang": "en", "location": "Reading, PA", "create_at": date("2012-09-04"), "description": "Digital news editor for @ReadingEagle. Also: Writer, #LOTR fan, #runner, dad. RT ≠ endorsement. http://bit.ly/digitalwatchblog", "followers_count": 877, "friends_count": 512, "statues_count": 19619 }, "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-01-04T10:00:21.000Z"), "id": 684071883591946240, "text": "Morning clouds/afternoon sun this afternoon, high 38 (3 C). Low 25 (-4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119697197, "name": "Huntsville Weather", "screen_name": "HuntsvilleWX", "lang": "en", "location": "null", "create_at": date("2010-03-04"), "description": "null", "followers_count": 669, "friends_count": 65, "statues_count": 6056 }, "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-04T10:00:21.000Z"), "id": 684071884099432448, "text": "If I delete my Instagram can I reuse the same name I once had????", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1855043856, "name": "Elyssa Fouts", "screen_name": "elyssaflores", "lang": "en", "location": "Houston, TX", "create_at": date("2013-09-11"), "description": "Snap: elyssaroxkbye", "followers_count": 383, "friends_count": 617, "statues_count": 4023 }, "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-04T10:00:21.000Z"), "id": 684071884296720384, "text": "let's switch �� https://t.co/olgbiw9LNJ", "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": 1990, "friends_count": 997, "statues_count": 228525 }, "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-04T10:00:21.000Z"), "id": 684071884363808774, "text": "@StarringMrPenn @ToriKelly @seventeen omg I just just died and came back to life��������", "in_reply_to_status": 684044563137343489, "in_reply_to_user": 189151074, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 189151074, 19017237, 16824090 }}, "user": { "id": 953520084, "name": "Sermontee Brown", "screen_name": "Montee_SoAwesom", "lang": "en", "location": "null", "create_at": date("2012-11-17"), "description": "Singer, dancer and actress. Also a friend, sister and daughter IG: montee_solovely Snapchat: sab_me2014", "followers_count": 310, "friends_count": 349, "statues_count": 1332 }, "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-04T10:00:21.000Z"), "id": 684071884502228992, "text": "Happy #mancrushmonday! Who wants to make @pablohernandez7 and @draeaxtellxxx their first #mcm of… https://t.co/PWDEtm2T1M", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.3853912,34.0842209"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "mancrushmonday", "mcm" }}, "user_mentions": {{ 105672148, 3300762094 }}, "user": { "id": 17478928, "name": "Andrew Christian", "screen_name": "andrewchristian", "lang": "en", "location": "Los Angeles", "create_at": date("2008-11-18"), "description": "#Designer Andrew Christian: Creator of #Luxury #Sportswear, #Swimwear & #Underwear for Men as seen in Details Magazine, Men's Fitness and Playboy.", "followers_count": 1403806, "friends_count": 214074, "statues_count": 30914 }, "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-04T10:00:21.000Z"), "id": 684071884867170304, "text": "\"Go like 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": 4610208201, "name": "ŁÔ7Ë", "screen_name": "mirandakalyn", "lang": "en", "location": "Marysville, OH", "create_at": date("2015-12-26"), "description": "This is my new account.. ❄️☪✝", "followers_count": 89, "friends_count": 189, "statues_count": 123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marysville, OH", "id": "79e69adde79a23b4", "name": "Marysville", "place_type": "city", "bounding_box": rectangle("-83.401316,40.199677 -83.298207,40.274078") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39159, "countyName": "Union", "cityID": 3948160, "cityName": "Marysville" } }
+{ "create_at": datetime("2016-01-04T10:00:21.000Z"), "id": 684071884950913025, "text": "@Skreenlooker89 LAWWWWD ��������", "in_reply_to_status": 684071748510167040, "in_reply_to_user": 608721020, "favorite_count": 0, "retweet_count": 0, "lang": "cy", "is_retweet": false, "user_mentions": {{ 608721020 }}, "user": { "id": 147983000, "name": "Berenice", "screen_name": "B_Loves_Tiesto", "lang": "en", "location": "Arlington, TX", "create_at": date("2010-05-25"), "description": "➡️LAN | 21 | IG: berenice_rmz94", "followers_count": 1154, "friends_count": 591, "statues_count": 30680 }, "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-04T10:00:21.000Z"), "id": 684071885106184192, "text": "BOYS HOOPS: Wilbert, The Woodlands pull away from Conroe, 61-47 https://t.co/gW6ChKIj41", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.49669647,30.17720032"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2875734329, "name": "The Woodlands News", "screen_name": "TheWoodlandsBL", "lang": "en", "location": "The Woodlands, Texas", "create_at": date("2014-11-13"), "description": "The Woodlands BubbleLife features community news, photos and events. Share your business, organization or personal news and events at https://t.co/q9Nc0oyUti.", "followers_count": 134, "friends_count": 122, "statues_count": 12685 }, "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-04T10:00:21.000Z"), "id": 684071885114585088, "text": "Someone bring me food I will pay you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3312340960, "name": "Romina Martin", "screen_name": "romimartin1303", "lang": "en", "location": "null", "create_at": date("2015-06-07"), "description": "null", "followers_count": 187, "friends_count": 161, "statues_count": 1702 }, "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-04T10:00:21.000Z"), "id": 684071885185798144, "text": "I can be 8 am or 8 pm and I'm still going to turn up to \"679\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 343852842, "name": "ᴮ", "screen_name": "beverlyngarza", "lang": "en", "location": "ohs", "create_at": date("2011-07-27"), "description": "null", "followers_count": 335, "friends_count": 29, "statues_count": 4271 }, "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-04T10:00:21.000Z"), "id": 684071885194293250, "text": "In DR �� eating mangoes https://t.co/8HDcTA2Mgu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 595778791, "name": "CVS Thug", "screen_name": "PinasNPalmTrees", "lang": "en", "location": "null", "create_at": date("2012-05-31"), "description": "a legend dead or alive #CRC", "followers_count": 550, "friends_count": 351, "statues_count": 28302 }, "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-04T10:00:21.000Z"), "id": 684071885290782720, "text": "I think Food was God's gift to man. Well. That and sex.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2283841056, "name": "halcyon", "screen_name": "areisaryan", "lang": "en", "location": "trinidad ・indy ・huntsville", "create_at": date("2014-01-09"), "description": "im here. deal. #futurehive", "followers_count": 789, "friends_count": 528, "statues_count": 16150 }, "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-04T10:00:21.000Z"), "id": 684071885307547648, "text": "The best thing about being back at work is I don't have time 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": 18806820, "name": "lindseyb.", "screen_name": "lblevs", "lang": "en", "location": "never never land.", "create_at": date("2009-01-09"), "description": "realist. optimist. overly self-indulgent. free spirit. proverbs 31:25. blessed. southern. roll tide.", "followers_count": 419, "friends_count": 446, "statues_count": 6458 }, "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-04T10:00:21.000Z"), "id": 684071885534048256, "text": "OBAMA'S BLOODY HANDS: 6 AIR FORCE DEAD - https://t.co/slYwEydJ0D .@RosieOnTheRight @Ruffiansea @Daria_gav @gadiadelman @IDFSpokesperson #USA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "USA" }}, "user_mentions": {{ 24600120, 1414423532, 850660369, 120836404, 18576537 }}, "user": { "id": 174618314, "name": "Montgomery Granger", "screen_name": "mjgranger1", "lang": "en", "location": "Long Island, New York", "create_at": date("2010-08-04"), "description": "Author: SAVING GRACE AT GUANTANAMO BAY; MAJ #USArmy (Ret.); Husband, Father, Educator, Veteran, @TheBlaze Contributor, @UofAlabama @TeachersCollege @stonybrooku", "followers_count": 58256, "friends_count": 50051, "statues_count": 333204 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Grove, NY", "id": "43545465269e5d09", "name": "Lake Grove", "place_type": "city", "bounding_box": rectangle("-73.13217,40.835174 -73.102764,40.883089") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3640530, "cityName": "Lake Grove" } }
+{ "create_at": datetime("2016-01-04T10:00:21.000Z"), "id": 684071885584351232, "text": "�� https://t.co/MBCq8H7SHL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1321739850, "name": "LongLiveKnuccless", "screen_name": "troy2timez", "lang": "en", "location": "on the 8", "create_at": date("2013-04-01"), "description": "Guard (6'0) |Humble ™ | The Real Duo @WardKhyree | Baltimore , MD | RIP Trenny | sc:teeroynumba10 KDM", "followers_count": 1218, "friends_count": 1143, "statues_count": 30044 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Essex, MD", "id": "8a8002ffef902f59", "name": "Essex", "place_type": "city", "bounding_box": rectangle("-76.489916,39.270065 -76.381559,39.330308") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2426600, "cityName": "Essex" } }
+{ "create_at": datetime("2016-01-04T10:00:21.000Z"), "id": 684071885680689153, "text": "@EverettJohnMor um fam, we need to get one of these at the house https://t.co/N05S4BHZkt", "in_reply_to_status": -1, "in_reply_to_user": 298242606, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 298242606 }}, "user": { "id": 233402395, "name": "Xhris®", "screen_name": "Chris_Archey", "lang": "en", "location": "Milpitas~San Jose~Fremont", "create_at": date("2011-01-02"), "description": "Realtor | Business: chrisarcheyRE@gmail.com", "followers_count": 1057, "friends_count": 485, "statues_count": 31287 }, "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-04T10:00:21.000Z"), "id": 684071885794086913, "text": "Só saudades. #winterbreak @ Madame Tussauds New York https://t.co/aDYmofpyXB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.98822679,40.75691406"), "retweet_count": 0, "lang": "pt", "is_retweet": false, "hashtags": {{ "winterbreak" }}, "user": { "id": 123376237, "name": "mich", "screen_name": "mchlmoreira", "lang": "en", "location": "null", "create_at": date("2010-03-15"), "description": "the gunslinger extraordinaire walking contradictions", "followers_count": 1648, "friends_count": 459, "statues_count": 29200 }, "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-04T10:00:21.000Z"), "id": 684071885882134528, "text": "Technically he is not wrong, Christmas Times last until Kings day… https://t.co/Rtu6gx7iUB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.00925129,40.71327642"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34735464, "name": "Fernando S.Cobourg", "screen_name": "saxecobourg", "lang": "en", "location": "New York, USA", "create_at": date("2009-04-23"), "description": "Name,Fernando Saxe Cobourg und Gotha,born in France,married,PHD,Partner at NY's brokerage company.Love,Polo,Tennis,Sail,Scubadive and travel.", "followers_count": 1284, "friends_count": 2056, "statues_count": 39679 }, "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-04T10:00:21.000Z"), "id": 684071885940731904, "text": "Welcome! https://t.co/h23JqepUoF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 780264936, "name": "VCheng", "screen_name": "thevcheng", "lang": "en", "location": "Boston, MA", "create_at": date("2012-08-25"), "description": "Proud Asian American, Follower of Christ, & Lover of Life. BU Hospitality, ASU PR, Donut Enthusiast, Everyday Explorer, & Go-getter.", "followers_count": 466, "friends_count": 512, "statues_count": 3559 }, "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-04T10:00:21.000Z"), "id": 684071886028849152, "text": "Sunny this afternoon, high 43 (6 C). Low 27 (-3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119698223, "name": "Birmingham Weather", "screen_name": "BirminghamWX", "lang": "en", "location": "Birmingham, AL", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Birmingham, AL", "followers_count": 503, "friends_count": 68, "statues_count": 6068 }, "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-04T10:00:21.000Z"), "id": 684071886221881344, "text": "Jill Malpass named new principal of The Woodlands High School Ninth Grade Campus https://t.co/5SRBYuZNRr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.49669647,30.17720032"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2875734329, "name": "The Woodlands News", "screen_name": "TheWoodlandsBL", "lang": "en", "location": "The Woodlands, Texas", "create_at": date("2014-11-13"), "description": "The Woodlands BubbleLife features community news, photos and events. Share your business, organization or personal news and events at https://t.co/q9Nc0oyUti.", "followers_count": 134, "friends_count": 122, "statues_count": 12686 }, "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-04T10:00:21.000Z"), "id": 684071886230261760, "text": "My singing >>> Tank's singing https://t.co/skT06xQV4w", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 243859653, "name": "Mad Swami", "screen_name": "ThinzelWashin10", "lang": "en", "location": "null", "create_at": date("2011-01-27"), "description": "whatever. #ODU class of eventually", "followers_count": 1527, "friends_count": 1388, "statues_count": 96652 }, "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-04T10:00:21.000Z"), "id": 684071886314180612, "text": "Just posted a photo @ Kohl's https://t.co/23hWGbJi7e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.98483038,41.64697412"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 32551073, "name": "Kennith gantz", "screen_name": "Kennyg66", "lang": "en", "location": "New Bedford MA", "create_at": date("2009-04-17"), "description": "Going to Hell in a hand basket!!!!!", "followers_count": 73, "friends_count": 59, "statues_count": 765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Smith Mills, MA", "id": "db0dcea533e078a2", "name": "Smith Mills", "place_type": "city", "bounding_box": rectangle("-71.047745,41.619498 -70.958622,41.680953") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25005, "countyName": "Bristol", "cityID": 2562300, "cityName": "Smith Mills" } }
+{ "create_at": datetime("2016-01-04T10:00:21.000Z"), "id": 684071886448275456, "text": "@LoLoJT @DWILKGOTDAPOLO on me I do too for the l", "in_reply_to_status": 684071515743105024, "in_reply_to_user": 275332748, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 275332748, 351426973 }}, "user": { "id": 260041116, "name": "Ken Penn ✈️", "screen_name": "BornWitWings", "lang": "en", "location": "Little Rock, Ar ✈️ Ks ", "create_at": date("2011-03-02"), "description": "You can do anything you put your mind to just put God first. R.I.P. Altee Tenpenny", "followers_count": 1170, "friends_count": 804, "statues_count": 43910 }, "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-04T10:00:21.000Z"), "id": 684071886645506048, "text": "i can't look at any of maddie's pictures without my self esteem dropping like 500 points", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1329250440, "name": "josh", "screen_name": "intrenets", "lang": "en", "location": "null", "create_at": date("2013-04-05"), "description": "if anyone asks you how the perfect satisfaction of all our sexual wanting will look, lift your face and say, like this", "followers_count": 538, "friends_count": 55, "statues_count": 53808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Daytona, FL", "id": "66bc82f162d83f01", "name": "South Daytona", "place_type": "city", "bounding_box": rectangle("-81.029402,29.148366 -80.989419,29.185724") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1267325, "cityName": "South Daytona" } }
+{ "create_at": datetime("2016-01-04T10:00:21.000Z"), "id": 684071886657986561, "text": "LMAO https://t.co/7YnOJJz4oF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "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": 11333, "friends_count": 1008, "statues_count": 338858 }, "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-04T10:00:21.000Z"), "id": 684071887127883776, "text": "Trojans use tourney to adjust to absence of standout https://t.co/5WO37LcNJ3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.08640289,32.85390091"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1327215439, "name": "Euless, TX News", "screen_name": "BLifeEuless", "lang": "en", "location": "Euless, Texas", "create_at": date("2013-04-04"), "description": "Euless BubbleLife features community news, photos and events. Share your business, organization or personal news and events at http://euless.bubblelife.com.", "followers_count": 259, "friends_count": 261, "statues_count": 7825 }, "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-04T10:00:22.000Z"), "id": 684071887211638785, "text": "@jesss_canterr we need to hang in the city soon", "in_reply_to_status": 684070520359370753, "in_reply_to_user": 2875658626, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2875658626 }}, "user": { "id": 346330964, "name": "Jarrett Livingston", "screen_name": "Unsexual_Cocoa", "lang": "en", "location": "Virginia", "create_at": date("2011-07-31"), "description": "Lime-Cucumber Gatorade", "followers_count": 348, "friends_count": 187, "statues_count": 25142 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chester, VA", "id": "8a8fb2a0babe3b8a", "name": "Chester", "place_type": "city", "bounding_box": rectangle("-77.480516,37.307356 -77.400126,37.391516") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51041, "countyName": "Chesterfield", "cityID": 5116096, "cityName": "Chester" } }
+{ "create_at": datetime("2016-01-04T10:00:22.000Z"), "id": 684071887232741377, "text": "@TyyMyShoes on your neck?��", "in_reply_to_status": 684021526937624576, "in_reply_to_user": 980574595, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 980574595 }}, "user": { "id": 3434879944, "name": "êm", "screen_name": "emilynotjimi", "lang": "en", "location": "864", "create_at": date("2015-08-21"), "description": "sc: emilycathryn_98", "followers_count": 67, "friends_count": 73, "statues_count": 1169 }, "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-01-04T10:00:22.000Z"), "id": 684071887471652864, "text": "used to think i was tall, peak", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 334852964, "name": "perpetual g", "screen_name": "JasLPH", "lang": "en", "location": "lph", "create_at": date("2011-07-13"), "description": "stop dat, start dat, get dat, what", "followers_count": 500, "friends_count": 265, "statues_count": 43042 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgetown, SC", "id": "3eaba204289a7cfc", "name": "Georgetown", "place_type": "city", "bounding_box": rectangle("-79.349456,33.291995 -79.26754,33.408833") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45043, "countyName": "Georgetown", "cityID": 4528870, "cityName": "Georgetown" } }
+{ "create_at": datetime("2016-01-04T10:00:22.000Z"), "id": 684071887618478080, "text": "There are flurries outside!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1063478748, "name": "UVA 12-1", "screen_name": "brianfrance22", "lang": "en", "location": "Chesapeake, VA", "create_at": date("2013-01-05"), "description": "18. One Goal.", "followers_count": 261, "friends_count": 361, "statues_count": 4748 }, "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-04T10:00:22.000Z"), "id": 684071887668928512, "text": "@Hm89Kkb \nまじかい 笑", "in_reply_to_status": 684038030680719360, "in_reply_to_user": 613456419, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user_mentions": {{ 613456419 }}, "user": { "id": 1427596352, "name": "寺井 雅人", "screen_name": "l723f2ced777k6n", "lang": "ja", "location": "null", "create_at": date("2013-05-14"), "description": "Indiana University Purdue University FW/Baseball ⚾️/ Windsurfing /Studying abroad http://instagram.com/masatoterai No pain No gain", "followers_count": 631, "friends_count": 636, "statues_count": 14064 }, "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-04T10:00:22.000Z"), "id": 684071887685550080, "text": "@wickedwitch_km Also, we're doing Taco Tuesday! https://t.co/uJ2U8smxAP", "in_reply_to_status": 684070061322059776, "in_reply_to_user": 97329837, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2370875653 }}, "user": { "id": 97329837, "name": "Andrew Sibulsky", "screen_name": "AndrewSibulsky", "lang": "en", "location": "Coeur d' Alene, Idaho", "create_at": date("2009-12-16"), "description": "I should be #composing right now. Am I?", "followers_count": 118, "friends_count": 378, "statues_count": 4337 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coeur d'Alene, ID", "id": "da6f5e66f73b8f7f", "name": "Coeur d'Alene", "place_type": "city", "bounding_box": rectangle("-116.855401,47.65878 -116.732088,47.744794") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16055, "countyName": "Kootenai", "cityID": 1616750, "cityName": "Coeur d'Alene" } }
+{ "create_at": datetime("2016-01-04T10:00:22.000Z"), "id": 684071887966605312, "text": "can't wait for all the chisme! my other half is coming home ������ @_Aphroditeee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 769554614 }}, "user": { "id": 68539447, "name": "jocelynee", "screen_name": "Jocelyne_Ojeda", "lang": "en", "location": "Cienega de Escobar, Durango", "create_at": date("2009-08-24"), "description": "Santa Maria Del Oro, Durango", "followers_count": 913, "friends_count": 675, "statues_count": 14572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Addison, IL", "id": "92e012db736d72c5", "name": "Addison", "place_type": "city", "bounding_box": rectangle("-88.063656,41.903958 -87.949502,41.958634") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1700243, "cityName": "Addison" } }
+{ "create_at": datetime("2016-01-04T10:00:22.000Z"), "id": 684071887983398913, "text": "Hey @buffer community! Interested in signing up as a Beta user for a super-secret product?! :) Feel free to DM me!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 197962366 }}, "user": { "id": 24089653, "name": "Marc Anthony Rosa", "screen_name": "Marc_Rosa", "lang": "en", "location": "Houston, TX", "create_at": date("2009-03-12"), "description": "Product @buffer, Passionate about product design and technology. Advocate for gummy bears.", "followers_count": 1485, "friends_count": 990, "statues_count": 4899 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-04T10:00:22.000Z"), "id": 684071888201515009, "text": "@NCPANTHERS1 @Dougielarge @goldieonice To all. I have thanked you for the commitment to the team and us. We haven't won much but we LOVE our", "in_reply_to_status": 683978547740934148, "in_reply_to_user": 37239551, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 37239551, 303151202, 36581980 }}, "user": { "id": 3087447410, "name": "John Mockridge", "screen_name": "MyZoreck2", "lang": "en", "location": "Florida, USA", "create_at": date("2015-03-15"), "description": "Age 63 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": 53, "friends_count": 84, "statues_count": 4335 }, "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-01-04T10:00:22.000Z"), "id": 684071888205787136, "text": "Days like these are why wine was invented ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2597678212, "name": "❤ Ohio_Girl ❤", "screen_name": "614_amber", "lang": "en", "location": "Columbus, OH", "create_at": date("2014-06-10"), "description": "Dare to be something more #GoBucks #GoCavs", "followers_count": 481, "friends_count": 429, "statues_count": 3957 }, "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-04T10:00:22.000Z"), "id": 684071888218419200, "text": "#mondayblues #sailtosable #lovemyjacks #jackrogers #preppy #littleblackdresscamphill… https://t.co/DHvMoCoel3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.915396,40.23993657"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "mondayblues", "sailtosable", "lovemyjacks", "jackrogers", "preppy", "littleblackdresscamphill" }}, "user": { "id": 2756415823, "name": "LittleBlackDress", "screen_name": "lbd_camphill", "lang": "en", "location": "null", "create_at": date("2014-08-22"), "description": "One is never over-dressed or underdressed with a little black dress...Karl Lagerfeld", "followers_count": 72, "friends_count": 110, "statues_count": 535 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Camp Hill, PA", "id": "a93638d9055b5624", "name": "Camp Hill", "place_type": "city", "bounding_box": rectangle("-76.944227,40.229813 -76.906489,40.25942") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42041, "countyName": "Cumberland", "cityID": 4211000, "cityName": "Camp Hill" } }
+{ "create_at": datetime("2016-01-04T10:00:22.000Z"), "id": 684071888281190400, "text": "@bruhitszach if I would send u a DM would u read and reply to it ��", "in_reply_to_status": -1, "in_reply_to_user": 1912839878, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1912839878 }}, "user": { "id": 2946806214, "name": "Torrance Bean Skye", "screen_name": "torry__henry", "lang": "en", "location": "1/5", "create_at": date("2014-12-28"), "description": "met Daniel Skye 8/29/15 & 12/13/15 Daniel and Nick are my lifesavers !NICK BEAN IS MINE (: just to make that clear .", "followers_count": 533, "friends_count": 635, "statues_count": 19442 }, "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-04T10:00:22.000Z"), "id": 684071888314863618, "text": "This is so cute ��❤️ https://t.co/cH6Pq2OllI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 415208998, "name": "↠ mandy ↞", "screen_name": "thecommonmandy", "lang": "en", "location": "null", "create_at": date("2011-11-17"), "description": "|| #HailSouthern || Psalm 23:4 ||", "followers_count": 1709, "friends_count": 947, "statues_count": 101809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Evans, GA", "id": "01af006c3a91a348", "name": "Evans", "place_type": "city", "bounding_box": rectangle("-82.228501,33.479096 -82.098825,33.593268") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13073, "countyName": "Columbia", "cityID": 1328044, "cityName": "Evans" } }
+{ "create_at": datetime("2016-01-04T10:00:22.000Z"), "id": 684071888339910656, "text": "I need Low Life on iTunes, I hate streaming it from soundcloud ���� @theweeknd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 255388236 }}, "user": { "id": 535520351, "name": "Juanito", "screen_name": "JuaanTooRaaw", "lang": "en", "location": "Long Beach, CA", "create_at": date("2012-03-24"), "description": "The Juan for you .", "followers_count": 200, "friends_count": 230, "statues_count": 9886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:22.000Z"), "id": 684071888453287936, "text": "Just sneezed into my elbow and I think people in the airport think I'm \"dabbing\" ...nobody even blessed me��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.284057,34.014641"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3103890948, "name": "Katelin Resta", "screen_name": "projectcheers", "lang": "en", "location": "Austin, Texas", "create_at": date("2015-03-22"), "description": "associate @techstars '15 | CM @FreeTextbooks", "followers_count": 167, "friends_count": 366, "statues_count": 253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta Georgia", "id": "07d9e040a8482000", "name": "Atlanta Georgia", "place_type": "poi", "bounding_box": rectangle("-84.2840571,34.014640899999996 -84.284057,34.014641") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1342425, "cityName": "Johns Creek" } }
+{ "create_at": datetime("2016-01-04T10:00:22.000Z"), "id": 684071888482623489, "text": "@Cahri12 On my way princess��", "in_reply_to_status": 683859390605225985, "in_reply_to_user": 458681734, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 458681734 }}, "user": { "id": 53541971, "name": "Chanel Judd", "screen_name": "chanel_judd", "lang": "en", "location": "null", "create_at": date("2009-07-03"), "description": "Everything happens for a reason⚢ IG:chaneljudd", "followers_count": 891, "friends_count": 566, "statues_count": 2508 }, "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-04T10:00:22.000Z"), "id": 684071888545423360, "text": "Sunny this afternoon, high 53 (12 C). Low 33 (1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119698549, "name": "Mobile Weather", "screen_name": "MobileWX", "lang": "en", "location": "Mobile, Al", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Mobile, Al", "followers_count": 403, "friends_count": 68, "statues_count": 6143 }, "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-04T10:00:22.000Z"), "id": 684071888616816640, "text": "Sixty naked bitches, no exaggeration. https://t.co/tWpUsWvOFv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 61546390, "name": "️", "screen_name": "httpscottie", "lang": "en", "location": "Michigan", "create_at": date("2009-07-30"), "description": "...gfg", "followers_count": 7138, "friends_count": 392, "statues_count": 233605 }, "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-04T10:00:22.000Z"), "id": 684071888960659456, "text": "Remote https://t.co/yxwVgj4MhK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3220647008, "name": "alligator hunter", "screen_name": "senortatertot", "lang": "en", "location": "Tejas", "create_at": date("2015-05-19"), "description": "southern advocate", "followers_count": 411, "friends_count": 395, "statues_count": 3750 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rendon, TX", "id": "00386da25bfa7106", "name": "Rendon", "place_type": "city", "bounding_box": rectangle("-97.30462,32.551138 -97.190905,32.610434") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4861568, "cityName": "Rendon" } }
+{ "create_at": datetime("2016-01-04T10:00:22.000Z"), "id": 684071888998510592, "text": "I don't want t", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160693280, "name": "amanda sessions", "screen_name": "AmandaSessions9", "lang": "en", "location": "Huntsville, Alabama ", "create_at": date("2010-06-28"), "description": "I'm 18. C/O '15 Mad Cow. 2 Corinthians 1:3-4 RIP Juwan SC: amanda_sessions", "followers_count": 501, "friends_count": 533, "statues_count": 8670 }, "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-04T10:00:22.000Z"), "id": 684071889040314368, "text": "@MrTLexify when you feel the Margwa (Child Services) teleport behind you https://t.co/WqCFxK9MtC", "in_reply_to_status": -1, "in_reply_to_user": 1282055606, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1282055606 }}, "user": { "id": 3019964666, "name": "Angel Garcia", "screen_name": "AngelAngelg5", "lang": "en", "location": "Oxnard, CA", "create_at": date("2015-02-14"), "description": "My name is Angel and I love YouTube, YouTube is life and love.", "followers_count": 56, "friends_count": 141, "statues_count": 298 }, "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-04T10:00:22.000Z"), "id": 684071889115983873, "text": "@willparty was very busy over the Holiday Break! No better way to start off 2016 than with a… https://t.co/xzRKcCDcOp", "in_reply_to_status": -1, "in_reply_to_user": 19791116, "favorite_count": 0, "coordinate": point("-121.9058,37.32625"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19791116 }}, "user": { "id": 19791116, "name": "WilliamsPartyRentals", "screen_name": "willparty", "lang": "en", "location": "San Jose, CA", "create_at": date("2009-01-30"), "description": "Entertaining Made Easy!", "followers_count": 173, "friends_count": 263, "statues_count": 2075 }, "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-04T10:00:22.000Z"), "id": 684071889422045186, "text": "Accident on I-71 NB near Greenlawn Ave #traffic https://t.co/hylMo0EFh8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.00963,39.93954"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "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": 1052, "friends_count": 42, "statues_count": 50881 }, "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-04T10:00:22.000Z"), "id": 684071889636044800, "text": "@lvws_ Cred: ❤️❤️❤️ https://t.co/Cd3DGMDv8N", "in_reply_to_status": -1, "in_reply_to_user": 2767113154, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2767113154 }}, "user": { "id": 56423679, "name": "Cherae Leri", "screen_name": "Cheraeleri", "lang": "en", "location": "International", "create_at": date("2009-07-13"), "description": "#Singer #Model #Sapiosexual Click the Link --|| http://youtu.be/QvMqX1cZbuA Cherae Leri X Kehlani-The Way ❤️", "followers_count": 1787, "friends_count": 1192, "statues_count": 23237 }, "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-04T10:00:22.000Z"), "id": 684071890277646336, "text": "worst Monday ever", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 460612639, "name": "dyedwool", "screen_name": "veronicasuniga", "lang": "en", "location": "puppy snuggles ", "create_at": date("2012-01-10"), "description": "rain in July", "followers_count": 449, "friends_count": 391, "statues_count": 16293 }, "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-04T10:00:22.000Z"), "id": 684071890596573184, "text": "@marshinizedxRoe thank you for sharing #MP", "in_reply_to_status": -1, "in_reply_to_user": 495451694, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MP" }}, "user_mentions": {{ 495451694 }}, "user": { "id": 1494430362, "name": "Mel Jones the BEAST", "screen_name": "motivationphil", "lang": "en", "location": "Williamsburg, Virginia", "create_at": date("2013-06-08"), "description": "Motivational Philosopher, Public Speaker, Personal Trainer, and Life Coach. #MotivationalPhilosopher #MP #MotivationalSpeaker #Entrepreneur", "followers_count": 1242, "friends_count": 501, "statues_count": 5883 }, "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-01-04T10:00:22.000Z"), "id": 684071890743353344, "text": "I'm really hoping this was a joke https://t.co/Bzv6r4UXPB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 427848943, "name": "LaDainian", "screen_name": "SaugyAugi", "lang": "en", "location": "Mansfield, TX", "create_at": date("2011-12-03"), "description": "My name is Alex, most call me Augi. 19. San Diego, California ✈️ Mansfield, Texas. LeBron fan. MY LEG!!! Q, Sal, Murr, Joe. #MFFL #BoltUp⚡️@paytenvinson ❤️", "followers_count": 1337, "friends_count": 799, "statues_count": 43244 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beachwood, NJ", "id": "77a79bea3dffdff3", "name": "Beachwood", "place_type": "city", "bounding_box": rectangle("-74.224657,39.910712 -74.178612,39.946012") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean", "cityID": 3404180, "cityName": "Beachwood" } }
+{ "create_at": datetime("2016-01-04T10:00:22.000Z"), "id": 684071890747551744, "text": "That's racist lol https://t.co/5FikXHxhcz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 208955958, "name": "Pelumi 'A", "screen_name": "MichaelAdeyoju_", "lang": "en", "location": "null", "create_at": date("2010-10-28"), "description": "USF '18 - Instagram : Michael_Adeyoju #2. Soccer . (❤❤Sade.S.A❤❤) 09.18.2014", "followers_count": 342, "friends_count": 105, "statues_count": 22103 }, "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-04T10:00:22.000Z"), "id": 684071890760130562, "text": "@sarahmccammon @nprpolitics @ChrisChristie Oh God...this is way too easy...", "in_reply_to_status": 684060248202723328, "in_reply_to_user": 21104257, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21104257, 5741722, 1347285918 }}, "user": { "id": 3105025619, "name": "American St. Nick", "screen_name": "AmericanStNick", "lang": "en", "location": "An uplifting MUST READ.", "create_at": date("2015-03-22"), "description": "“One more memorable and inspiring story from the greatest generation...It will touch your heart and make you proud.”\n\n– Tom Brokaw", "followers_count": 168, "friends_count": 475, "statues_count": 3014 }, "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-04T10:00:22.000Z"), "id": 684071891045199872, "text": "Morning snow showers this afternoon, high 31 (-1 C). Low 22 (-6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119751854, "name": "Chicago Weather", "screen_name": "Chicago_WX", "lang": "en", "location": "Chicago, IL", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Chicago, IL", "followers_count": 1060, "friends_count": 68, "statues_count": 6425 }, "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-04T10:00:22.000Z"), "id": 684071891120840704, "text": "Too real https://t.co/u3hF2HlxXZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 42464346, "name": "grahamborghini", "screen_name": "gkilaaaa", "lang": "en", "location": "New Bedford, MA", "create_at": date("2009-05-25"), "description": "I been a terror since the public school era. SC:Gkilaaaa", "followers_count": 1063, "friends_count": 1432, "statues_count": 10735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nashua, NH", "id": "db6cee8d55fe4a29", "name": "Nashua", "place_type": "city", "bounding_box": rectangle("-71.5749,42.700085 -71.433334,42.805852") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33011, "countyName": "Hillsborough", "cityID": 3350260, "cityName": "Nashua" } }
+{ "create_at": datetime("2016-01-04T10:00:22.000Z"), "id": 684071891141791744, "text": "��GET YOUR $5 EARLY BIRD TICKETS NOW��\nlevelsmarketing @_dafatboy @1JazzyJeff @selectivesrvc… https://t.co/zPMKeA2Owg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.192749,25.7845192"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2517779879, 97774874, 2795158475 }}, "user": { "id": 60697749, "name": "#Aftershock", "screen_name": "radiojamez", "lang": "en", "location": "Miami", "create_at": date("2009-07-27"), "description": "SoFlo Hottest DJ / Host /Event Planner ... #MrWildLYFE #PacJamPresident #4ThQuarterKing Booking: (786)202-8085 RadioJamez@gmail.com", "followers_count": 3545, "friends_count": 666, "statues_count": 83322 }, "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-04T10:00:23.000Z"), "id": 684071891418521600, "text": "Why do I feel like every black guy in Silverado is going to steal 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": 1260691410, "name": "❄️Alexis❄️", "screen_name": "ItsAlexisssa", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2013-03-11"), "description": "18/single/vegetarian", "followers_count": 300, "friends_count": 272, "statues_count": 6190 }, "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-04T10:00:23.000Z"), "id": 684071891439628288, "text": "@matutex0 we in 2016 now , no time to go back ��", "in_reply_to_status": 683861620452405249, "in_reply_to_user": 1155408242, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1155408242 }}, "user": { "id": 505407438, "name": "αηgιε", "screen_name": "Angie_Tamay", "lang": "en", "location": "null", "create_at": date("2012-02-26"), "description": "null", "followers_count": 180, "friends_count": 144, "statues_count": 1451 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ossining, NY", "id": "462c2d50fa4b9bb9", "name": "Ossining", "place_type": "city", "bounding_box": rectangle("-73.876223,41.136627 -73.813587,41.199919") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3655530, "cityName": "Ossining" } }
+{ "create_at": datetime("2016-01-04T10:00:23.000Z"), "id": 684071891460464640, "text": "Apollo International: Security Officer (#Bloomington, MN) https://t.co/YepnEpB0qO #security #Veterans #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.2982799,44.840798"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Bloomington", "security", "Veterans", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 118914397, "name": "TMJ-MSP Security Job", "screen_name": "tmj_MSP_secure", "lang": "en", "location": "St. Paul, MN", "create_at": date("2010-03-01"), "description": "Follow this account for geo-targeted Security job tweets in St. Paul, MN from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 292, "friends_count": 266, "statues_count": 15 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomington, MN", "id": "080b8d8543aab399", "name": "Bloomington", "place_type": "city", "bounding_box": rectangle("-93.399443,44.78542 -93.203245,44.863519") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2706616, "cityName": "Bloomington" } }
+{ "create_at": datetime("2016-01-04T10:00:23.000Z"), "id": 684071891607269377, "text": "@realDonaldTrump @HillaryClinton @MichaelCohen212 https://t.co/RpW3f8UOWH", "in_reply_to_status": -1, "in_reply_to_user": 25073877, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 25073877, 1339835893, 52136185 }}, "user": { "id": 3193870345, "name": "neilsaavy7099", "screen_name": "714_319", "lang": "en", "location": "Lake Forest, CA", "create_at": date("2015-05-12"), "description": "God\nFamily\nCountry \nMy yella Labrador \nGlock .40 ., AR's 2nd A.\n L A KINGS Fan. GKG !\nHate PC & if I hurt ur feelings because I'm not, 2 !$@! bad.", "followers_count": 224, "friends_count": 110, "statues_count": 5762 }, "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-04T10:00:23.000Z"), "id": 684071891657601024, "text": "I make my own , what I need some guys money for. ����, \nOnly broke bitches do that. https://t.co/Ji3rAtngR3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1852094994, "name": "BabyyGirll ☀", "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": 1902, "friends_count": 1424, "statues_count": 16992 }, "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-04T10:00:23.000Z"), "id": 684071891720470529, "text": "I love it when u look at me baby", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 317946368, "name": "Maresa Lettig ♈", "screen_name": "marryleeanne", "lang": "en", "location": "null", "create_at": date("2011-06-15"), "description": "If an army moves against me, my mind will not be afraid - Psalm 27:3", "followers_count": 400, "friends_count": 523, "statues_count": 3311 }, "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-04T10:00:23.000Z"), "id": 684071892605661184, "text": "Damn right especially a month https://t.co/p8JGpbUYXj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 204149181, "name": "MIA", "screen_name": "I_Be_Sleep", "lang": "en", "location": "kalamazoo", "create_at": date("2010-10-17"), "description": "Low end or no end family first r.i.p treezy free shooter", "followers_count": 1213, "friends_count": 1104, "statues_count": 26803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westwood, MI", "id": "5ef83697018ccdd9", "name": "Westwood", "place_type": "city", "bounding_box": rectangle("-85.649338,42.288789 -85.608758,42.336593") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26077, "countyName": "Kalamazoo", "cityID": 2686380, "cityName": "Westwood" } }
+{ "create_at": datetime("2016-01-04T10:00:23.000Z"), "id": 684071892639076352, "text": "@NSAEncounters she's fine I'd smash", "in_reply_to_status": -1, "in_reply_to_user": 2493844699, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2493844699 }}, "user": { "id": 148640111, "name": "George R. Jenkins Jr", "screen_name": "hndsomdevl", "lang": "en", "location": "Plano Tx.", "create_at": date("2010-05-26"), "description": "I'm a grown ass man that don't have time for bullshit or people who r full of it. I'm a handsome devil,and a Marine so don't push me! Let's drink n b merry...1", "followers_count": 55, "friends_count": 261, "statues_count": 605 }, "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-04T10:00:23.000Z"), "id": 684071892718727168, "text": "Can you recommend anyone for this #job? Sales Associate - https://t.co/OisaeQ1Nbf #Omaha, NE #Retail #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.0674861,41.2654737"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Omaha", "Retail", "Hiring" }}, "user": { "id": 3248774184, "name": "Kay Jewelers", "screen_name": "JobsAtKay", "lang": "en", "location": "null", "create_at": date("2015-06-18"), "description": "Your Future Starts Here.", "followers_count": 5, "friends_count": 0, "statues_count": 1110 }, "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-04T10:00:23.000Z"), "id": 684071892806926340, "text": "My top things are my relationship with God, losing weight, making/saving money, traveling and enjoying my young years ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24097310, "name": "Kaison Tiller", "screen_name": "BigKai_", "lang": "en", "location": "University of Toledo", "create_at": date("2009-03-12"), "description": "the goal is to inspire.", "followers_count": 2516, "friends_count": 2110, "statues_count": 158171 }, "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-04T10:00:23.000Z"), "id": 684071892890697729, "text": "Love that I have practice later ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 796511011, "name": "ITS MY BIRTHDAY!", "screen_name": "Payton_Roberts0", "lang": "en", "location": "Newton, KS", "create_at": date("2012-09-01"), "description": "Iowa Central Track Commit • just a light skin livin' lavish •", "followers_count": 624, "friends_count": 374, "statues_count": 8545 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newton, KS", "id": "fa8690b2c9ddb059", "name": "Newton", "place_type": "city", "bounding_box": rectangle("-97.377877,37.999177 -97.309268,38.071764") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20079, "countyName": "Harvey", "cityID": 2050475, "cityName": "Newton" } }
+{ "create_at": datetime("2016-01-04T10:00:23.000Z"), "id": 684071893138337793, "text": "We stayed up until the wee hours of the night packing, making a necessary coffee pit-stop. Even… https://t.co/hIZs5jZtYu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.984726,39.7292099"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24721921, "name": "Trisha Saturday", "screen_name": "TrishaSaturday", "lang": "en", "location": "Denver, CO", "create_at": date("2009-03-16"), "description": "null", "followers_count": 295, "friends_count": 298, "statues_count": 1672 }, "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-04T10:00:23.000Z"), "id": 684071893377265664, "text": "Thank GOD for blessing me to see another day ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1214333604, "name": "DRΣШ", "screen_name": "UncleSaltyyy", "lang": "en", "location": "Shawnee, OK", "create_at": date("2013-02-23"), "description": "|20 years old| |insta: drewchapline5 |GodFirst| RIP Raphael, RIP Little Joe", "followers_count": 845, "friends_count": 686, "statues_count": 15787 }, "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-01-04T10:00:23.000Z"), "id": 684071893469642752, "text": "Cool temps for once!!! (@ Boca Raton, FL in Boca Raton, FL) https://t.co/hy1V4wubxh https://t.co/PEB39dnhiV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.08329391,26.35899667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 31164534, "name": "Morton Levitt", "screen_name": "levitm1", "lang": "en", "location": "Boca Raton, FL", "create_at": date("2009-04-14"), "description": "Dysfunctional Dungeon Denizen, Master of the Sub tweet, GOW guru, and Social Media Harnesser. Also, in my spare time, Food Reviewer at He&SheFood blog.", "followers_count": 1394, "friends_count": 1329, "statues_count": 61809 }, "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-04T10:00:23.000Z"), "id": 684071893654097921, "text": "Mostly sunny this afternoon, high 47 (8 C). Low 29 (-2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119753180, "name": "Jackson Weather", "screen_name": "JacksonWX", "lang": "en", "location": "Jackson, MS", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Jackson, MS", "followers_count": 295, "friends_count": 68, "statues_count": 6374 }, "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-04T10:00:23.000Z"), "id": 684071893725347840, "text": "I'm ashamed lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 465221655, "name": "Brandy", "screen_name": "brandyyperez", "lang": "en", "location": "bay area", "create_at": date("2012-01-15"), "description": "maininsta:brandyyperez makeupinsta:brandyperezmua", "followers_count": 785, "friends_count": 416, "statues_count": 72372 }, "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-04T10:00:23.000Z"), "id": 684071893801037825, "text": "@__faithmariee hopefully", "in_reply_to_status": 684071790042329089, "in_reply_to_user": 286692444, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 286692444 }}, "user": { "id": 2216920408, "name": "B R O K E ™", "screen_name": "Jriichhh", "lang": "en", "location": "null", "create_at": date("2013-12-09"), "description": "Dont @ me cause i promise i dont care ‼️", "followers_count": 415, "friends_count": 235, "statues_count": 4387 }, "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-04T10:00:23.000Z"), "id": 684071893826064387, "text": "@mastaace @putyrdreams1st yo coconut oil is a life saver. You can do everything with that shit. I like using it in a preworkout", "in_reply_to_status": 684066279150481408, "in_reply_to_user": 34086767, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 34086767, 17488879 }}, "user": { "id": 1329532921, "name": "Tony SooWoo", "screen_name": "OgHasnt", "lang": "en", "location": "Chicago✈Los Angeles ", "create_at": date("2013-04-05"), "description": "Born on southside of Chicago, live in LA. Stay focused and keep going. Lifting is Life, so is music... And also Halo (Prop. 215 & S.B. 420) #RealHipHop", "followers_count": 582, "friends_count": 531, "statues_count": 10605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:23.000Z"), "id": 684071893859569664, "text": ":://", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2381555988, "name": "bella", "screen_name": "BellaReilley", "lang": "en", "location": "null", "create_at": date("2014-03-09"), "description": "mchs'19 | volleyball", "followers_count": 281, "friends_count": 254, "statues_count": 494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glen Ellyn, IL", "id": "d77ed7c9521c898d", "name": "Glen Ellyn", "place_type": "city", "bounding_box": rectangle("-88.089723,41.8152 -88.027005,41.903992") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1729756, "cityName": "Glen Ellyn" } }
+{ "create_at": datetime("2016-01-04T10:00:23.000Z"), "id": 684071893889085440, "text": "@AlterNet @POTUS always rely on getting info from the breaving abused before the assaulters for you 8/10 and under circumstances the best", "in_reply_to_status": 684068039017365504, "in_reply_to_user": 18851248, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18851248, 1536791610 }}, "user": { "id": 3292500478, "name": "TQSB5689", "screen_name": "tqsb56891", "lang": "en", "location": "United States ", "create_at": date("2015-05-20"), "description": "Cognitve Experimental Statistical NeuroScience Artificial Intelligence Robotics Physics Chemistry Design Retweet doesn't mean endorsement", "followers_count": 6, "friends_count": 353, "statues_count": 25803 }, "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-01-04T10:00:23.000Z"), "id": 684071894140637184, "text": "Yo 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": 3017847889, "name": "Jeikobu D.Kurosaki⚓️", "screen_name": "PublicHippie", "lang": "en", "location": "Pallet Town", "create_at": date("2015-02-12"), "description": "Wealth , Fame , Power ... @PublicHood Contact Directly : Drewskiessss@aol.com", "followers_count": 144, "friends_count": 87, "statues_count": 7299 }, "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-04T10:00:23.000Z"), "id": 684071894283350016, "text": "Lmaooooo...�������� https://t.co/dr36LNviIz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 1358322566, "name": "TRGAxBRZY✨", "screen_name": "fijiwatre__", "lang": "en", "location": "ᗩTᒪ,Gᗩ / Տҽղ16ɾ", "create_at": date("2013-04-16"), "description": "ᔕᑕ: ᕼOᒪᒪIᔕTEᖇᔕᗰᑌᖇᖴᖴ #44⚾ ⚾ |ᑕ/ᗝ'16| @_LuhMyya •R.I.P. GranGran, Reina, JuJu •Homecoming King @ TCHS. #myASU20", "followers_count": 1434, "friends_count": 1741, "statues_count": 28773 }, "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-04T10:00:23.000Z"), "id": 684071894442590208, "text": "@grayson_denny I'll take some when I get home", "in_reply_to_status": 684071328115220482, "in_reply_to_user": 2765202417, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2765202417 }}, "user": { "id": 54144322, "name": "~Sweet Heat~", "screen_name": "shittytittyx", "lang": "en", "location": "Arizona", "create_at": date("2009-07-06"), "description": "Tall broad. Drift bitch. Hair smith. @easychode Daily Frs. Drift rx7 fc.", "followers_count": 181, "friends_count": 105, "statues_count": 3308 }, "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-04T10:00:23.000Z"), "id": 684071894576840704, "text": "Accident with blockage in #OrangeCo on Landstreet Rd at Orange Blossom Trail #traffic https://t.co/YKhnnOyzxi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4047,28.436"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "OrangeCo", "traffic" }}, "user": { "id": 249853776, "name": "TT+WN Orlando", "screen_name": "TotalTrafficMCO", "lang": "en", "location": "Orlando, FL", "create_at": date("2011-02-09"), "description": "null", "followers_count": 172, "friends_count": 96, "statues_count": 38149 }, "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-01-04T10:00:23.000Z"), "id": 684071894589566976, "text": "@_MsKris French Bulldogs are nice. I'm of course looking for a pit bull mix. Something similar to this.. �� https://t.co/4Csoo3f6U0", "in_reply_to_status": 684071068647223296, "in_reply_to_user": 208280081, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 208280081 }}, "user": { "id": 1007706096, "name": "'Tun", "screen_name": "ilook_likeGucci", "lang": "en", "location": "University of Tennessee- Knox", "create_at": date("2012-12-12"), "description": "null", "followers_count": 346, "friends_count": 283, "statues_count": 12718 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Suwanee, GA", "id": "7b01e60536219287", "name": "Suwanee", "place_type": "city", "bounding_box": rectangle("-84.107404,34.001413 -84.024808,34.096528") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13135, "countyName": "Gwinnett", "cityID": 1374936, "cityName": "Suwanee" } }
+{ "create_at": datetime("2016-01-04T10:00:23.000Z"), "id": 684071894685880320, "text": "#CareerArc #Sales #Job alert: Part Time Sales | dressbarn | #Turlock, CA https://t.co/ImaFzXOXHu #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-120.8465941,37.4946568"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CareerArc", "Sales", "Job", "Turlock", "Jobs", "Hiring" }}, "user": { "id": 22577085, "name": "TMJ-CA-US Sales", "screen_name": "tmj_ca_sales", "lang": "en", "location": "California", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Sales job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 484, "friends_count": 314, "statues_count": 3754 }, "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-04T10:00:23.000Z"), "id": 684071894761340930, "text": "back to hell tomorrow ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3194827513, "name": "maria chammas", "screen_name": "mariachammmas", "lang": "en", "location": "Houston, TX", "create_at": date("2015-05-13"), "description": "good times never seemed so good", "followers_count": 83, "friends_count": 93, "statues_count": 240 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-04T10:00:23.000Z"), "id": 684071894887305216, "text": "@Annabelle__R https://t.co/pjBJeqNYTF", "in_reply_to_status": -1, "in_reply_to_user": 2344686354, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2344686354 }}, "user": { "id": 365232822, "name": "Kyrie", "screen_name": "ChaseSchertz", "lang": "en", "location": "null", "create_at": date("2011-08-30"), "description": "null", "followers_count": 1164, "friends_count": 985, "statues_count": 35387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallmadge, OH", "id": "cdb98b233892e8af", "name": "Tallmadge", "place_type": "city", "bounding_box": rectangle("-81.465754,41.060128 -81.373636,41.135788") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3976106, "cityName": "Tallmadge" } }
+{ "create_at": datetime("2016-01-04T10:00:23.000Z"), "id": 684071895218573312, "text": "Would you rather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27251039, "name": "Tabitha Rose", "screen_name": "raviolifreak", "lang": "en", "location": "Seattle", "create_at": date("2009-03-28"), "description": "are you ready to jingle your bells", "followers_count": 1466, "friends_count": 506, "statues_count": 25426 }, "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-04T10:00:23.000Z"), "id": 684071895285805056, "text": "En fait tout ce qu'on voit dans les film etc c'est la vrai vie c'est pas du mensonge I swear", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 809739811, "name": "Harlem", "screen_name": "Kev_joke", "lang": "fr", "location": "New York, NY", "create_at": date("2012-09-07"), "description": "Papa me reniera jamais , j'suis ni flic ni pd ! #Team237", "followers_count": 143, "friends_count": 54, "statues_count": 14074 }, "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-01-04T10:00:23.000Z"), "id": 684071895315030016, "text": "�������� https://t.co/hJ7fK3Qg9F", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1114945598, "name": "DJLiTEBRiTE", "screen_name": "DJLiTEBRiTE", "lang": "en", "location": "OKC", "create_at": date("2013-01-23"), "description": "Creating my own lane | Boa⚓️club | February 6th", "followers_count": 1543, "friends_count": 925, "statues_count": 11212 }, "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-04T10:00:24.000Z"), "id": 684071895575195649, "text": "No but I want to https://t.co/E7qdK7ifBc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3249902922, "name": "Li' Shawt ❣", "screen_name": "miraclelashayy", "lang": "en", "location": "@financer_ ", "create_at": date("2015-06-19"), "description": "SC: adoresmiracle || fineeassk ❤️ @prettyasanna long_liveexx", "followers_count": 667, "friends_count": 473, "statues_count": 26708 }, "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-04T10:00:24.000Z"), "id": 684071895679909888, "text": "School tomorrow. https://t.co/SjfFKAUbZC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1476138541, "name": "Meredith Novy", "screen_name": "MeredithNovy", "lang": "en", "location": "Longview, TX ", "create_at": date("2013-06-01"), "description": "don't you know that only fools are satisfied • tamu20", "followers_count": 343, "friends_count": 360, "statues_count": 3658 }, "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-01-04T10:00:24.000Z"), "id": 684071895696687104, "text": "������❗ https://t.co/VAEBW5GeBr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 310511805, "name": "SP", "screen_name": "shaylababby", "lang": "en", "location": "850fL", "create_at": date("2011-06-03"), "description": "1/1 gang I'm reckless ♐ IG:shaylaxofl", "followers_count": 588, "friends_count": 318, "statues_count": 14530 }, "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-01-04T10:00:24.000Z"), "id": 684071895843487744, "text": "@Josieposiee13 @CuteEmergency ����������you were doing this in October you're gonna die ����", "in_reply_to_status": 684071724565008385, "in_reply_to_user": 864823136, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 864823136, 568825492 }}, "user": { "id": 2171151751, "name": "Nikolette", "screen_name": "nikkidallich", "lang": "en", "location": "edwardsburg or east lansing ", "create_at": date("2013-11-02"), "description": "Michigan State University '18 (future accountant) #BallIsNotLife | #fngm | #SpartyOn @josieposiee13 is my fav", "followers_count": 988, "friends_count": 713, "statues_count": 16677 }, "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-04T10:00:24.000Z"), "id": 684071895927406593, "text": "@Hits93Toronto @selenagomez Please play #History by @onedirection", "in_reply_to_status": 684067918473261056, "in_reply_to_user": 796435297, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "History" }}, "user_mentions": {{ 796435297, 23375688, 209708391 }}, "user": { "id": 1135599187, "name": "HARRY'S FOOL", "screen_name": "Foolish4Harry", "lang": "en", "location": "stumbling through the dark ", "create_at": date("2013-01-30"), "description": "This is not the end", "followers_count": 4837, "friends_count": 2483, "statues_count": 299822 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:24.000Z"), "id": 684071896057384960, "text": "���� tbh I had the biggest crush on you last wrestling season and I kinda feel like you like me now but idk you're still cute tho", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 717363128, "name": "Kÿłâ", "screen_name": "KylaInglis", "lang": "en", "location": "Portland ➡️ Vancouver", "create_at": date("2012-07-25"), "description": "DDHS • Single • Employed • Philippians 4:13 ✨ Matthew 6:34", "followers_count": 502, "friends_count": 529, "statues_count": 12973 }, "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-04T10:00:24.000Z"), "id": 684071896183259136, "text": "Currently ���� @Kiereeee https://t.co/YOyIYVl4uK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2462281945 }}, "user": { "id": 1688974788, "name": "Queen Danielle", "screen_name": "__itsdanielle", "lang": "en", "location": "Your Grandaddy House ", "create_at": date("2013-08-21"), "description": "null", "followers_count": 267, "friends_count": 300, "statues_count": 584 }, "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-04T10:00:24.000Z"), "id": 684071896216809472, "text": "@PrometheusX1Z When will you know we have enough quality to try and win the league? Did you know in summer of 2013?", "in_reply_to_status": 684069107868774400, "in_reply_to_user": 1672578360, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1672578360 }}, "user": { "id": 20564407, "name": "42%", "screen_name": "JosephPepper", "lang": "en", "location": "Los Angeles", "create_at": date("2009-02-10"), "description": "This twitter account is devoted entirely to explaining why the Europa League is rubbish and we shouldn't bother with it.", "followers_count": 2422, "friends_count": 2058, "statues_count": 41215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:24.000Z"), "id": 684071896258719745, "text": "I've somehow chosen the slowest Walgreens ever. JUST GIVE ME MY DRUGS.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70612102, "name": "Crissy", "screen_name": "crissycano", "lang": "en", "location": "Dallas | Tx", "create_at": date("2009-08-31"), "description": "do you like cheese? my favorites gouda.", "followers_count": 755, "friends_count": 446, "statues_count": 24467 }, "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-04T10:00:24.000Z"), "id": 684071896418156544, "text": "Sunny this afternoon, high 52 (11 C). Low 31 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119754993, "name": "Waco Weather", "screen_name": "WacoWeather", "lang": "en", "location": "Waco, TX", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Waco, TX", "followers_count": 816, "friends_count": 68, "statues_count": 6355 }, "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.069786,31.639828") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4876000, "cityName": "Waco" } }
+{ "create_at": datetime("2016-01-04T10:00:24.000Z"), "id": 684071896447500288, "text": "@ApolloFuse breh he stimulating the economy, why u think birds fly south, cause its west side till i die, same reason i got 15% in art class", "in_reply_to_status": 684070763935170560, "in_reply_to_user": 286993706, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 286993706 }}, "user": { "id": 2354914670, "name": "DZNTZ", "screen_name": "GanjaManMark", "lang": "en", "location": "null", "create_at": date("2014-02-21"), "description": "#ETG #VerbalAssault #HHH #RawEmpire #AunthenticMobb gang gagang gang gang", "followers_count": 387, "friends_count": 148, "statues_count": 12231 }, "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-04T10:00:24.000Z"), "id": 684071896464240640, "text": "@GoodDayCourtney was w/ @SacramentoSPCA learning about the @LisaVanderpump Adoption story https://t.co/5UTEp4hsT4 https://t.co/YdSKHx1jfL", "in_reply_to_status": -1, "in_reply_to_user": 331759868, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 331759868, 19029562, 110840720 }}, "user": { "id": 27238125, "name": "Matt D", "screen_name": "dcap22803", "lang": "en", "location": "Elk Grove, CA", "create_at": date("2009-03-28"), "description": "Diehard @GoodDaySac @markatthemovies @buffalobilks fan with brain/eye tumors lover of life fun and football. Toy Soldier Collector", "followers_count": 1423, "friends_count": 2057, "statues_count": 76660 }, "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-04T10:00:24.000Z"), "id": 684071896602796032, "text": "@GamingRulesVids @HeavyCardboard @VacaBCK @ianoble @brbdc and that just makes these GOtY lists more confusing. :/", "in_reply_to_status": 684071427650224129, "in_reply_to_user": 2341927562, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2341927562, 2511542774, 19002469, 18646258, 10816192 }}, "user": { "id": 203365053, "name": "Justin Bowman", "screen_name": "Camelbo", "lang": "en", "location": "Apex, North Carolina", "create_at": date("2010-10-15"), "description": "Freelance photographer who loves people and loves interacting with their stories. capturing moments to last a lifetime. Also a dad and an avid Boardgamer!", "followers_count": 138, "friends_count": 220, "statues_count": 2817 }, "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.794651,35.767843") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3701520, "cityName": "Apex" } }
+{ "create_at": datetime("2016-01-04T10:00:24.000Z"), "id": 684071897106132992, "text": "Today!\n#firstdayback #fun #balls #toys #ugh #today #funpic #silly #play #outside #smile #smiles… https://t.co/1aoRpUJnAI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.0025318,33.65555564"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "firstdayback", "fun", "balls", "toys", "ugh", "today", "funpic", "silly", "play", "outside", "smile", "smiles" }}, "user": { "id": 45985345, "name": "@savannahboan", "screen_name": "thesavannahboan", "lang": "en", "location": "can't catch me", "create_at": date("2009-06-09"), "description": "anti-establishment, free spirit, livin' off the grid", "followers_count": 2869, "friends_count": 254, "statues_count": 4277 }, "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-04T10:00:24.000Z"), "id": 684071897433305088, "text": "Check out Mega Money Mixup 2016 from Chicago Sun-Times - I just entered here! https://t.co/C8Q9nMxsZ6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 194370687, "name": "Edgar Escobar", "screen_name": "Esco_lets_go", "lang": "en", "location": "CHICAGO", "create_at": date("2010-09-23"), "description": "Love. Live. Life", "followers_count": 96, "friends_count": 433, "statues_count": 3930 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Niles, IL", "id": "653aabd8c2d3e4d2", "name": "Niles", "place_type": "city", "bounding_box": rectangle("-87.855052,41.999951 -87.767299,42.062018") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1753000, "cityName": "Niles" } }
+{ "create_at": datetime("2016-01-04T10:00:24.000Z"), "id": 684071897949065218, "text": "sittin here sad as hell listening to Adele, i feel U baby", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2230016647, "name": "phil", "screen_name": "coldasfelipe", "lang": "en", "location": "MN", "create_at": date("2013-12-04"), "description": "human bean https://soundcloud.com/khemical/atnightkhem", "followers_count": 434, "friends_count": 158, "statues_count": 20601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willmar, MN", "id": "016e05c102e1ccc0", "name": "Willmar", "place_type": "city", "bounding_box": rectangle("-95.094207,45.08981 -94.986685,45.198324") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27067, "countyName": "Kandiyohi", "cityID": 2770420, "cityName": "Willmar" } }
+{ "create_at": datetime("2016-01-04T10:00:24.000Z"), "id": 684071898053910532, "text": "@winojanet @michaeljohns @realDonaldTrump Chged his view on this since seeing disaster.Continually advocates for private mkt. FACT, my dear.", "in_reply_to_status": 683782906783772672, "in_reply_to_user": 221173706, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 221173706, 14828860, 25073877 }}, "user": { "id": 1689920359, "name": "Robyn Ryan", "screen_name": "Reryan08", "lang": "en", "location": "null", "create_at": date("2013-08-21"), "description": "American Patriot", "followers_count": 579, "friends_count": 298, "statues_count": 15785 }, "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-04T10:00:24.000Z"), "id": 684071898280378368, "text": "the school district got me listening to trust issues this morning", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1677656617, "name": "zababe", "screen_name": "nataliezakaib", "lang": "en", "location": "null", "create_at": date("2013-08-17"), "description": "light skin arab. DADD // OVO . phat", "followers_count": 511, "friends_count": 243, "statues_count": 1788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McMinnville, OR", "id": "2a732a9dca7f3c87", "name": "McMinnville", "place_type": "city", "bounding_box": rectangle("-123.25428,45.185598 -123.128954,45.239298") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41071, "countyName": "Yamhill", "cityID": 4145000, "cityName": "McMinnville" } }
+{ "create_at": datetime("2016-01-04T10:00:24.000Z"), "id": 684071898355924992, "text": "Interested in a #Hospitality #job near #Camden, SC? This could be a great fit: https://t.co/a2w7P6PRwi #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.598,34.246604"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Camden", "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": 196, "friends_count": 39, "statues_count": 19098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Camden, SC", "id": "5f002f8289b93efa", "name": "Camden", "place_type": "city", "bounding_box": rectangle("-80.652572,34.21709 -80.536964,34.306286") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45055, "countyName": "Kershaw", "cityID": 4510855, "cityName": "Camden" } }
+{ "create_at": datetime("2016-01-04T10:00:24.000Z"), "id": 684071898431361024, "text": "@Xtrematis MOVER A MEXICO A LA EXTREMA POBREZA.", "in_reply_to_status": 683962377931341824, "in_reply_to_user": 512833326, "favorite_count": 0, "coordinate": point("-106.44693,31.6782175"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 512833326 }}, "user": { "id": 2203800090, "name": "Saul Mateos Jimenez", "screen_name": "saulma78", "lang": "es", "location": "Chihuahua ", "create_at": date("2013-11-19"), "description": "CIUDADANO ACTIVO E INCONFORME DE LA IMPUNIDAD Y CORRUPCION,IDEALISTA Y MIEMBRO DE MORENA.", "followers_count": 836, "friends_count": 1639, "statues_count": 7632 }, "place": { "country": "México", "country_code": "México", "full_name": "Juárez, Chihuahua", "id": "77d45fff09fe6f8a", "name": "Juárez", "place_type": "city", "bounding_box": rectangle("-106.950638,31.120681 -106.185378,31.783867") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-01-04T10:00:24.000Z"), "id": 684071898855034881, "text": "Road reopened in #Syracuse on Teall Ave Both NB/SB between James St and Peck Ave #traffic https://t.co/MBQubGTSI5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.12269,43.06502"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Syracuse", "traffic" }}, "user": { "id": 250330008, "name": "TTN Syracuse", "screen_name": "TotalTrafficSYR", "lang": "en", "location": "Syracuse, NY", "create_at": date("2011-02-10"), "description": "See a traffic problem, accident or delay? Call our Traffic Tip Line at 315-421-1190", "followers_count": 614, "friends_count": 344, "statues_count": 29812 }, "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-04T10:00:24.000Z"), "id": 684071899085733888, "text": "Partly cloudy this afternoon, high 38 (3 C). Low 24 (-4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119755728, "name": "Louisville Weather", "screen_name": "Louisville_WX", "lang": "en", "location": "Louisville, KY", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Louisville, KY", "followers_count": 1310, "friends_count": 68, "statues_count": 6381 }, "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-01-04T10:00:24.000Z"), "id": 684071899089993729, "text": "@JayVeeThree check out the last 6 18th picks. Pretty sweet", "in_reply_to_status": 684062584236974080, "in_reply_to_user": 586874504, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 586874504 }}, "user": { "id": 103043730, "name": "Alex Bucksot", "screen_name": "bucksotUSA", "lang": "en", "location": "Louisville, KY", "create_at": date("2010-01-08"), "description": "I sell beer and beer related accessories", "followers_count": 220, "friends_count": 575, "statues_count": 6749 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Galena, IN", "id": "c1a53eaeae2ad7a7", "name": "Galena", "place_type": "city", "bounding_box": rectangle("-85.957246,38.336151 -85.919741,38.36127") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18043, "countyName": "Floyd", "cityID": 1826206, "cityName": "Galena" } }
+{ "create_at": datetime("2016-01-04T10:00:24.000Z"), "id": 684071899165515776, "text": "I been sleep since 11:30 last 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": 314987607, "name": "G4", "screen_name": "_GH4", "lang": "en", "location": "Norview", "create_at": date("2011-06-10"), "description": "May the Lord be with us all", "followers_count": 1207, "friends_count": 831, "statues_count": 27527 }, "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-04T10:00:24.000Z"), "id": 684071899656159234, "text": "Lol stalker https://t.co/Fw9R6SbSss", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21543226, "name": "iAmOpiieeBee", "screen_name": "OpiieBee", "lang": "en", "location": "United States", "create_at": date("2009-02-21"), "description": "null", "followers_count": 3716, "friends_count": 1942, "statues_count": 102072 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:24.000Z"), "id": 684071899748503552, "text": "It don't gotta be that cold outside ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4180648126, "name": "NEW PAGE!", "screen_name": "Quitaaa___", "lang": "en", "location": "null", "create_at": date("2015-11-13"), "description": "Instagram @Quitaaa___ Snapchat @quitaquitaa", "followers_count": 250, "friends_count": 246, "statues_count": 866 }, "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-04T10:00:25.000Z"), "id": 684071899899412480, "text": "6. Sweden 2-1\n7. Lowell Green\n8. Zidane\n9. CEOs\n10. Ben Scrivens\n\n2016/1/4 11:53 CST #trndnl https://t.co/SHjd3tkINE", "in_reply_to_status": 684071898192281600, "in_reply_to_user": 1270239397, "favorite_count": 0, "coordinate": point("-98.309,56.9547"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "trndnl" }}, "user": { "id": 1270239397, "name": "Trendinalia Canada", "screen_name": "trendinaliaCA", "lang": "es", "location": "Canada", "create_at": date("2013-03-15"), "description": "Information about Twitter's Trending Topics in Canada", "followers_count": 1250, "friends_count": 7, "statues_count": 256849 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Division No. 23, Unorganized, Manitoba", "id": "654fd5617c7361d0", "name": "Division No. 23, Unorganized", "place_type": "city", "bounding_box": rectangle("-102.009382,55.158501 -88.989166,60.000283") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-04T10:00:25.000Z"), "id": 684071899903606784, "text": "@jessnvaldez yessss my mom says I do. You can tell if you look at my eyes long enough", "in_reply_to_status": 683894087137677312, "in_reply_to_user": 310956622, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 310956622 }}, "user": { "id": 307167024, "name": "⚜", "screen_name": "Danii_Giirl", "lang": "en", "location": "null", "create_at": date("2011-05-28"), "description": "✿ ღ SLTX. ✾ ✿ ღ HTX ✿ 1995 NBK. ⚜", "followers_count": 471, "friends_count": 254, "statues_count": 56158 }, "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-01-04T10:00:25.000Z"), "id": 684071900125908993, "text": "See our latest #Oakland, CA #job and click to apply: ER RN / Emergency Room Registered Nurse - https://t.co/uTFShbRQA5 #Healthcare #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.2143323,37.8015517"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Oakland", "job", "Healthcare", "Hiring" }}, "user": { "id": 20827274, "name": "SF Healthcare Jobs", "screen_name": "tmj_sfo_health", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-02-13"), "description": "Follow this account for geo-targeted Healthcare job tweets in San Francisco, CA. Need help? Tweet us at @CareerArc!", "followers_count": 786, "friends_count": 335, "statues_count": 1323 }, "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-04T10:00:25.000Z"), "id": 684071900331442177, "text": "I'm so excited for my punk music class", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3232979504, "name": "hannah", "screen_name": "hnnhkthln_", "lang": "en", "location": "null", "create_at": date("2015-06-01"), "description": "UCLA.", "followers_count": 99, "friends_count": 94, "statues_count": 1550 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:25.000Z"), "id": 684071900553867264, "text": "Yo man I'm hella pumped for the @saltines_ game, it's gonna be lit, ball out tonight dudes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4700687646 }}, "user": { "id": 2305428728, "name": "Brian Shaffer", "screen_name": "brian_shaffer12", "lang": "en", "location": "Westerville, OH", "create_at": date("2014-01-22"), "description": "WCHS 19'", "followers_count": 298, "friends_count": 314, "statues_count": 1740 }, "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-01-04T10:00:25.000Z"), "id": 684071900700524544, "text": "If that makes any sense", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2337316268, "name": "j rob", "screen_name": "jadadiane16", "lang": "en", "location": "wilding somewhere", "create_at": date("2014-02-10"), "description": "• call me J • DON'T be a wanna-be • learning is a permanent change in behavior • TRAPSOUL • angel with an attitude • 18 •", "followers_count": 1374, "friends_count": 761, "statues_count": 49023 }, "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-04T10:00:25.000Z"), "id": 684071901111693313, "text": "Don't remember putting my phone on dnd ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 342421404, "name": "N I N A", "screen_name": "_Svmxne", "lang": "en", "location": "Knightdale, NC", "create_at": date("2011-07-25"), "description": "SEN1⃣6⃣R #EHS #NAVY", "followers_count": 732, "friends_count": 744, "statues_count": 31532 }, "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-01-04T10:00:25.000Z"), "id": 684071901157666816, "text": "@LVPoetryArts Thank you for the follow.", "in_reply_to_status": -1, "in_reply_to_user": 4019367023, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4019367023 }}, "user": { "id": 2174567323, "name": "Ann Wilmer-Lasky", "screen_name": "awlasky", "lang": "en", "location": "Roswell, NM", "create_at": date("2013-11-04"), "description": "Strange old lady who lives in Roswell, New Mexico and writes science fiction, fantasy and HORROR. Did I mention poetry? Die-Hard Jimmy Fallon #FalPal", "followers_count": 8862, "friends_count": 9721, "statues_count": 64460 }, "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-01-04T10:00:25.000Z"), "id": 684071901350760448, "text": "I'm slick fine��", "in_reply_to_status": 683663397871861760, "in_reply_to_user": 41916656, "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": "email: cortavius14@gmail.com … GVO UAPB\n\n101.3 The Beat Intern", "followers_count": 2801, "friends_count": 1745, "statues_count": 50744 }, "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-04T10:00:25.000Z"), "id": 684071901363228672, "text": "Ramos Allup: La mayoría calificada no está en riesgo : Noticiero Digital https://t.co/8AFNqGgacA Es lo que espera la mayoría que votó MUD.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 1260976320, "name": "Franzo Marruffo", "screen_name": "lamis1712", "lang": "en", "location": "Caracas", "create_at": date("2013-03-11"), "description": "Venezolano, Ingeniero Químico MSc PhD. Actualmente jubilado.", "followers_count": 77, "friends_count": 79, "statues_count": 9556 }, "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-04T10:00:25.000Z"), "id": 684071901426143232, "text": "I was standing on my sidewalk thinking I used to be out standing on this corner everyday��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3514894463, "name": "One.Kay", "screen_name": "130thFinest", "lang": "en", "location": "null", "create_at": date("2015-09-01"), "description": "Show no love, love gets you killed", "followers_count": 85, "friends_count": 179, "statues_count": 724 }, "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-04T10:00:25.000Z"), "id": 684071901988175872, "text": "May not look like a lot of things that I have to do today, but it will fill my day up! :D @… https://t.co/8V6jfLHSAO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.901,48.5451"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2809161117, "name": "Matthew Ford", "screen_name": "FlameFord", "lang": "en", "location": "Kansas City, Missouri", "create_at": date("2014-10-05"), "description": "null", "followers_count": 25, "friends_count": 319, "statues_count": 869 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colville, WA", "id": "0963945c670b8732", "name": "Colville", "place_type": "city", "bounding_box": rectangle("-117.92569,48.521547 -117.857723,48.564636") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53065, "countyName": "Stevens", "cityID": 5314170, "cityName": "Colville" } }
+{ "create_at": datetime("2016-01-04T10:00:25.000Z"), "id": 684071902302699522, "text": "#Retail #Job in #GlenEllyn, IL: Customer Service Representative at Speedway https://t.co/sDpC4TYvU6 #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.0670118,41.8775293"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "Job", "GlenEllyn", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 59785313, "name": "TMJ-IL Retail Jobs", "screen_name": "tmj_il_retail", "lang": "en", "location": "Illinois", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in Illinois Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 464, "friends_count": 310, "statues_count": 1236 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glen Ellyn, IL", "id": "d77ed7c9521c898d", "name": "Glen Ellyn", "place_type": "city", "bounding_box": rectangle("-88.089723,41.8152 -88.027005,41.903992") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1729756, "cityName": "Glen Ellyn" } }
+{ "create_at": datetime("2016-01-04T10:00:25.000Z"), "id": 684071902365618178, "text": "I've been on Twitter all day and it's never been more dead", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 412740778, "name": "Bizzy Hendrix", "screen_name": "BizzyNextDoor", "lang": "en", "location": "CLE", "create_at": date("2011-11-14"), "description": "Alhamdillah", "followers_count": 586, "friends_count": 307, "statues_count": 61299 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westlake, OH", "id": "abcede5a546a90a3", "name": "Westlake", "place_type": "city", "bounding_box": rectangle("-81.970255,41.41104 -81.872904,41.478131") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3983622, "cityName": "Westlake" } }
+{ "create_at": datetime("2016-01-04T10:00:25.000Z"), "id": 684071902399311872, "text": "Temp 26.0°F RH 62% Wind 6.1 W Gust 16.0 W SLP 30.228 in Rising slowly Rain 0.00 in Solar 123 UV 0.8 #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": 102, "friends_count": 65, "statues_count": 32281 }, "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-04T10:00:25.000Z"), "id": 684071902495637504, "text": "KTP Graduate #Education #Job: MCAT INSTRUCTOR - Northfield, MN (#Bloomington, MN) https://t.co/pLoQcbXShC #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.2982799,44.840798"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "Education", "Job", "Bloomington", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 21835489, "name": "TMJ-MSP Educ. Jobs", "screen_name": "tmj_msp_edu", "lang": "en", "location": "St. Paul, MN", "create_at": date("2009-02-24"), "description": "Follow this account for geo-targeted Education/Teaching job tweets in St. Paul, MN. Need help? Tweet us at @CareerArc!", "followers_count": 326, "friends_count": 237, "statues_count": 28 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomington, MN", "id": "080b8d8543aab399", "name": "Bloomington", "place_type": "city", "bounding_box": rectangle("-93.399443,44.78542 -93.203245,44.863519") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2706616, "cityName": "Bloomington" } }
+{ "create_at": datetime("2016-01-04T10:00:25.000Z"), "id": 684071902696964096, "text": "Thank you!!! �� https://t.co/n3LJL89ufg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 423809773, "name": "jooaannaॐ", "screen_name": "joooaaannna", "lang": "en", "location": "null", "create_at": date("2011-11-28"), "description": "21 ❃ traveling the world ❃ Hawaii || California", "followers_count": 6979, "friends_count": 459, "statues_count": 42608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Altos, CA", "id": "6a4364ea6f987c10", "name": "Los Altos", "place_type": "city", "bounding_box": rectangle("-122.129474,37.329932 -122.060782,37.406473") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 643280, "cityName": "Los Altos" } }
+{ "create_at": datetime("2016-01-04T10:00:25.000Z"), "id": 684071903686860804, "text": "44 days for me :-) https://t.co/LShOG6SbVM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4287190097, "name": "britt^2", "screen_name": "brittanyboyd_", "lang": "en", "location": "challenge 99", "create_at": date("2015-11-26"), "description": "dog enthusiast", "followers_count": 109, "friends_count": 121, "statues_count": 1396 }, "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-01-04T10:00:26.000Z"), "id": 684071904156725248, "text": "This is my Santa Anita racetrack look. I rarely take \"selfies\"..especially straight on, right… https://t.co/YfHRjjXvIk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.04473838,34.13897143"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27385442, "name": "heidi s. van tassell", "screen_name": "heidisabrina", "lang": "en", "location": "LA", "create_at": date("2009-03-28"), "description": "If at first you don't succeed, try, try, and try again. Then give up. There's no use being a damned fool about it. - W. C. Fields... writer/bass player for ehc", "followers_count": 242, "friends_count": 332, "statues_count": 1336 }, "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-01-04T10:00:26.000Z"), "id": 684071904316121089, "text": "@Harry_Styles hi Harry would you mind giving me a follow? It's my birthday and that's all I asked for, it would mean the world to me x10", "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": 1337677146, "name": "g loves h", "screen_name": "swiirlystyles", "lang": "en", "location": "Roswell, GA", "create_at": date("2013-04-08"), "description": "do whatever makes you happiest", "followers_count": 1094, "friends_count": 249, "statues_count": 34415 }, "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-04T10:00:26.000Z"), "id": 684071904550989824, "text": "And since you stalk my Twitter. \nThis goes out 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": 1258418714, "name": "IndiraMelisa", "screen_name": "indiraaaK", "lang": "en", "location": "Queens, NY", "create_at": date("2013-03-10"), "description": "add my snap! -indiramelisax3", "followers_count": 190, "friends_count": 174, "statues_count": 3862 }, "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-04T10:00:26.000Z"), "id": 684071904592949248, "text": "������ https://t.co/RfXLXaFUak", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "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": 2495, "friends_count": 624, "statues_count": 171241 }, "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-04T10:00:26.000Z"), "id": 684071904651563008, "text": "@Ms25Lighters aye, u https://t.co/UqZr06z2I3", "in_reply_to_status": 684071200415334400, "in_reply_to_user": 186976811, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 186976811 }}, "user": { "id": 496354569, "name": "Tweets4rmTheCounty", "screen_name": "El_estupendo", "lang": "en", "location": " Memphis, Egypt", "create_at": date("2012-02-18"), "description": "these niggas fu and these bitches too", "followers_count": 338, "friends_count": 295, "statues_count": 16108 }, "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-04T10:00:26.000Z"), "id": 684071904903323648, "text": "I miss my roommate. @Chrisssy_Jones come back to me ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 248371584 }}, "user": { "id": 426279676, "name": "Emily Nichols", "screen_name": "_emnichols", "lang": "en", "location": "null", "create_at": date("2011-12-01"), "description": "CLE | Kent State", "followers_count": 594, "friends_count": 479, "statues_count": 12368 }, "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-01-04T10:00:26.000Z"), "id": 684071905289093122, "text": "The one person I thought I can trust , can't even trust her . ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2416673504, "name": "| Lann • ☮ |", "screen_name": "queenlanii_", "lang": "en", "location": "null", "create_at": date("2014-03-28"), "description": "A Girl Who Stays In Her Shell...❣ #TattedAndPierced☺️", "followers_count": 398, "friends_count": 664, "statues_count": 11185 }, "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-04T10:00:26.000Z"), "id": 684071905306001410, "text": "i wonder what runs through your mind to think it's okay to say or do the things 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": 75621085, "name": "v a n e s s a", "screen_name": "vanessakasiuk", "lang": "en", "location": "oxford, michigan", "create_at": date("2009-09-19"), "description": "forget your sadness, anger, grudges, and hatred. let them pass like smoke caught in a breeze. do not let yourself indulge in such feelings. ♒️", "followers_count": 609, "friends_count": 867, "statues_count": 12731 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxford, MI", "id": "00c6aa3d045d7743", "name": "Oxford", "place_type": "city", "bounding_box": rectangle("-83.334197,42.795097 -83.215202,42.861512") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2662020, "cityName": "Oxford" } }
+{ "create_at": datetime("2016-01-04T10:00:26.000Z"), "id": 684071905368752128, "text": "We're #hiring! Click to apply: Pharmacy Technician - https://t.co/jRcr65sBRG #Healthcare #Orlando, FL #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3792365,28.5383355"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Healthcare", "Orlando", "Job", "Jobs", "CareerArc" }}, "user": { "id": 22036700, "name": "Orlando Health Jobs", "screen_name": "tmj_orl_health", "lang": "en", "location": "Orlando, FL", "create_at": date("2009-02-26"), "description": "Follow this account for geo-targeted Healthcare job tweets in Orlando, FL. Need help? Tweet us at @CareerArc!", "followers_count": 605, "friends_count": 303, "statues_count": 279 }, "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-04T10:00:26.000Z"), "id": 684071905372946432, "text": "Me https://t.co/pdb8dwQ0HL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3850612580, "name": "♏️add❣e", "screen_name": "madison_ssalas", "lang": "en", "location": "null", "create_at": date("2015-10-10"), "description": "cshs", "followers_count": 369, "friends_count": 232, "statues_count": 851 }, "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-04T10:00:26.000Z"), "id": 684071905469464576, "text": "@a_rab2011 boy i was in bed and asleep by 9 last night ! �� had to get up at 5am", "in_reply_to_status": 683868863763644416, "in_reply_to_user": 516046035, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 516046035 }}, "user": { "id": 538507971, "name": "Joshua Castro", "screen_name": "Joshhh_11", "lang": "en", "location": "Wharton", "create_at": date("2012-03-27"), "description": "Drive a Buick on 24's ☝️Snapchat jcas765", "followers_count": 471, "friends_count": 425, "statues_count": 5735 }, "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-04T10:00:26.000Z"), "id": 684071905645596672, "text": "Got some good vibes rn ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2308947924, "name": "theyscreamwineteer", "screen_name": "N_Wineteer3", "lang": "en", "location": "Euless Texas", "create_at": date("2014-01-24"), "description": "Varsity Football #3. Varsity Track. Rest In Paradise Lolo. Big Players make Big Plays on Big Stages. #theyscreamwineteer #watchoutfornumber3", "followers_count": 232, "friends_count": 323, "statues_count": 323 }, "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-04T10:00:26.000Z"), "id": 684071905708523520, "text": "I'll be finishing my last day of break watching Grey's Anatomy and crying ttyl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1109563164, "name": "Lupita", "screen_name": "Lupitaisafish", "lang": "en", "location": "null", "create_at": date("2013-01-21"), "description": "be happy", "followers_count": 531, "friends_count": 231, "statues_count": 16023 }, "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-04T10:00:26.000Z"), "id": 684071905796734976, "text": "I'm at NASA Ames T35 in Mountain View, CA https://t.co/nsnqpWuddh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.05554859,37.39528644"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 38123446, "name": "Игopь Кoндpaшoв", "screen_name": "I_Kond", "lang": "en", "location": "null", "create_at": date("2009-05-05"), "description": "null", "followers_count": 690, "friends_count": 467, "statues_count": 66752 }, "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-04T10:00:26.000Z"), "id": 684071905976926208, "text": "@webster and also thanks in advance for not judging my temporary window curtain", "in_reply_to_status": 684071800397942789, "in_reply_to_user": 22330197, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 7632132 }}, "user": { "id": 22330197, "name": "Albiniak", "screen_name": "malbiniak", "lang": "en", "location": "Portland, Ore / Kepler 452-b", "create_at": date("2009-02-28"), "description": "People tell me I remind them of Howard Hughes (the late years). Opinions are mine, bozo. #RCTID #LGRW", "followers_count": 1907, "friends_count": 356, "statues_count": 17043 }, "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-04T10:00:26.000Z"), "id": 684071906077708289, "text": "Lunch date! (@ Spinning Wheel Diner in Lebanon, NJ) https://t.co/0MRCtPPDDr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.796613,40.640364"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29638002, "name": "Stephaniie", "screen_name": "steph_palein", "lang": "en", "location": "New Jersey", "create_at": date("2009-04-07"), "description": "being a farmgirl is so contradictory for me.", "followers_count": 219, "friends_count": 223, "statues_count": 10938 }, "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": 34019, "countyName": "Hunterdon" } }
+{ "create_at": datetime("2016-01-04T10:00:26.000Z"), "id": 684071906106974208, "text": "Predictions for 2016: A Tale of Two Art Worlds | BLOUIN ARTINFO https://t.co/YDyEYqPmmj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2278128841, "name": "PiotrArt*", "screen_name": "Piotrartist", "lang": "en", "location": "In my studio.", "create_at": date("2014-01-05"), "description": "A post-historical artist. http://www.piotrart.com", "followers_count": 423, "friends_count": 705, "statues_count": 853 }, "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-04T10:00:26.000Z"), "id": 684071906148888576, "text": "If you use your fingers to apply makeup don't 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": 559850559, "name": "ivan ayub", "screen_name": "ivan_ayub", "lang": "en", "location": "null", "create_at": date("2012-04-21"), "description": "null", "followers_count": 417, "friends_count": 237, "statues_count": 4935 }, "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-04T10:00:26.000Z"), "id": 684071906572500992, "text": "When school and practice cancelled because of snow and ice https://t.co/vjVjmtGBrO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 321520906, "name": "Raekwon Mingledoff", "screen_name": "iBall_Raekwon", "lang": "en", "location": "PA/NJ✈✈✈Oregon", "create_at": date("2011-06-21"), "description": "Basketball is Life ! Success is the only option ! Education is Key.. Student Athelete @ Lane CC", "followers_count": 818, "friends_count": 609, "statues_count": 44513 }, "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-04T10:00:26.000Z"), "id": 684071906727755776, "text": "Can you recommend anyone for this #job? Shift Supervisor - https://t.co/cb4iN4gs7P #Schaumburg, IL #Hospitality #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.0834059,42.0333607"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Schaumburg", "Hospitality", "Hiring", "CareerArc" }}, "user": { "id": 22453816, "name": "Chicago Hospitality", "screen_name": "tmj_chi_hrta", "lang": "en", "location": "Chicago, IL", "create_at": date("2009-03-01"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Chicago, IL. Need help? Tweet us at @CareerArc!", "followers_count": 544, "friends_count": 299, "statues_count": 1475 }, "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-04T10:00:26.000Z"), "id": 684071906849472513, "text": "@cyrinafiallo did I just see you on a glasses commercial? If so awesome! If not you have a doppelgänger.", "in_reply_to_status": -1, "in_reply_to_user": 28029641, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28029641 }}, "user": { "id": 1373760151, "name": "Andrew S", "screen_name": "ashuman17", "lang": "en", "location": "Minneapolis, MN", "create_at": date("2013-04-22"), "description": "I'm from Minnesota. don't know what else to say about it but I enjoy it. Work in insurance. I know but it's a job. Bears, Cubs and stuff. #sixseasonsandamovie", "followers_count": 12, "friends_count": 58, "statues_count": 105 }, "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-04T10:00:26.000Z"), "id": 684071907038081024, "text": "Poe mans dreams x kendrick lamar\nYeah yeah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 55765327, "name": "Joshua alexander", "screen_name": "josh_fy", "lang": "en", "location": "Mesa, AZ", "create_at": date("2009-07-10"), "description": "#Findyou and stunt.\n|8/13/11|\nStay woke.\nMilan", "followers_count": 667, "friends_count": 415, "statues_count": 40531 }, "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-04T10:00:26.000Z"), "id": 684071907138748416, "text": "Crying https://t.co/7YnOJJz4oF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "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": 11333, "friends_count": 1008, "statues_count": 338859 }, "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-04T10:00:26.000Z"), "id": 684071907302440960, "text": "@OverweightAlex Happy Birthday Al!������ https://t.co/opufFgwD2r", "in_reply_to_status": -1, "in_reply_to_user": 598912552, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 598912552 }}, "user": { "id": 1676444966, "name": "®ylee Erwin", "screen_name": "RyleeErwin11", "lang": "en", "location": "Limestone, IL", "create_at": date("2013-08-16"), "description": "BMCHS K43 #FlyHighKae Instagram: rylee_erwin http://ask.fm/RyleeErwin", "followers_count": 581, "friends_count": 351, "statues_count": 3337 }, "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-01-04T10:00:26.000Z"), "id": 684071907344318464, "text": "#Monroe, LA #Hospitality #Job: Multi Unit Leader/Supervisor at SONIC Drive-In https://t.co/xnDxuI4rwA #SONIC #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.0722021,32.5748129"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Monroe", "Hospitality", "Job", "SONIC", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 132057683, "name": "TMJ-LAM HRTA Jobs", "screen_name": "tmj_LAM_HRTA", "lang": "en", "location": "Monroe, LA", "create_at": date("2010-04-11"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Monroe, LA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 293, "friends_count": 272, "statues_count": 128 }, "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-04T10:00:26.000Z"), "id": 684071907440726017, "text": "@_JossySays lol it's my damn autocorrect", "in_reply_to_status": 684071830273953792, "in_reply_to_user": 441157727, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 441157727 }}, "user": { "id": 46305981, "name": "Jack of DemBoy$", "screen_name": "ImDatNigga_Jack", "lang": "en", "location": "PVAMU", "create_at": date("2009-06-10"), "description": "I Don't Tweet No Lies Dem Boy$ IG:imdatnigga_jack #PVNation #HighLifePromos #LongLiveChopo Free @JoeSwaggerRight #LongLiveClyde", "followers_count": 5535, "friends_count": 3926, "statues_count": 318716 }, "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-04T10:00:26.000Z"), "id": 684071907461697536, "text": "Just cause ain't shit to do https://t.co/SPwPh5eInB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 447439316, "name": "J-®od", "screen_name": "Jerrincoontz", "lang": "en", "location": "Villa Rica, GA", "create_at": date("2011-12-26"), "description": "I Aint Afraid Of Dyin I'm Afraid Of Losin", "followers_count": 395, "friends_count": 289, "statues_count": 15858 }, "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-04T10:00:26.000Z"), "id": 684071907856089088, "text": "When you just now realizing she was a grown hoe & you didn't really give her the time of day...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 313779350, "name": "dillon moore ✨", "screen_name": "__dilluminati", "lang": "en", "location": "The Land", "create_at": date("2011-06-08"), "description": "17. Visionary. I can't lose.", "followers_count": 1112, "friends_count": 890, "statues_count": 40542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shaker Heights, OH", "id": "33d5fac88acb8fa8", "name": "Shaker Heights", "place_type": "city", "bounding_box": rectangle("-81.593675,41.457067 -81.50712,41.492867") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3971682, "cityName": "Shaker Heights" } }
+{ "create_at": datetime("2016-01-04T10:00:26.000Z"), "id": 684071907885510656, "text": "@_JAY_R| *Dolezal face*\n\nI don't understand the question.", "in_reply_to_status": 684071698950451201, "in_reply_to_user": 179621657, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 179621657 }}, "user": { "id": 224354592, "name": "Edgar Allan Hoe", "screen_name": "XIXMCMLXXXV", "lang": "en", "location": "Angel Grove", "create_at": date("2010-12-08"), "description": "Welcome...AND RAQUEL TAKE THAT FUCKING JACKET OFF!!", "followers_count": 2179, "friends_count": 613, "statues_count": 64344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Union City, GA", "id": "d1683f3c5c646d61", "name": "Union City", "place_type": "city", "bounding_box": rectangle("-84.597615,33.542538 -84.485509,33.633988") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1378324, "cityName": "Union City" } }
+{ "create_at": datetime("2016-01-04T10:00:26.000Z"), "id": 684071908036472832, "text": "They have returned! #parkingmongos https://t.co/n64LpPWA4x", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "parkingmongos" }}, "user": { "id": 15398296, "name": "Stephen DeBenedetto", "screen_name": "LevityNYC", "lang": "en", "location": "Hazlet, NJ", "create_at": date("2008-07-11"), "description": "Proud member of #MongoNation. Follow for tweets about sports, video games, and sports talk radio.", "followers_count": 555, "friends_count": 322, "statues_count": 59443 }, "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-04T10:00:27.000Z"), "id": 684071908397039616, "text": "If you're a #Nursing professional in #LosAngeles, CA, check out this #job: https://t.co/GBjEqKsMS1 #nursingjobs https://t.co/fv6kcd1MKK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.2452633,33.9239643"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "LosAngeles", "job", "nursingjobs" }}, "user": { "id": 3100019028, "name": "YourMLKCHJobs", "screen_name": "yourMLKCHjobs", "lang": "en", "location": "Los Angeles, CA ", "create_at": date("2015-03-20"), "description": "Now hiring qualified, committed and passionate individuals. Join the MLKCH Team!", "followers_count": 240, "friends_count": 539, "statues_count": 51 }, "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-04T10:00:27.000Z"), "id": 684071908493557760, "text": "@Dan_Welter @Zhawk44 * maybe", "in_reply_to_status": 684070312023953408, "in_reply_to_user": 94160104, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 94160104, 117279993 }}, "user": { "id": 208878148, "name": "Trip Manfro", "screen_name": "pfac51", "lang": "en", "location": "The 64157", "create_at": date("2010-10-27"), "description": "Life, an times of Manfro: #Hawkeyes, #Steelers, #Orioles, #Blackhawks, & #Royals. #CraftBeer #TRadio #TRadioMafia", "followers_count": 841, "friends_count": 1042, "statues_count": 296127 }, "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-04T10:00:27.000Z"), "id": 684071908745195520, "text": "@MayZellie @antonialee1978 @hardball White People?", "in_reply_to_status": 684067924961837057, "in_reply_to_user": 3177419908, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3177419908, 2196259189, 205864193 }}, "user": { "id": 21237679, "name": "WISE LATINA", "screen_name": "EusebiaAq", "lang": "en", "location": "America", "create_at": date("2009-02-18"), "description": "My mission is to end hate in America! You can can be part of the human solution or part of the injustice painful reality.GOD Bless America and our President!", "followers_count": 2141, "friends_count": 2062, "statues_count": 153465 }, "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-04T10:00:27.000Z"), "id": 684071908749348864, "text": "Some of the best memories have to go untold to 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": 825659390, "name": "Josh Shewmaker", "screen_name": "JoshShewmaker", "lang": "en", "location": "null", "create_at": date("2012-09-15"), "description": "Avila baseball", "followers_count": 702, "friends_count": 298, "statues_count": 2801 }, "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-04T10:00:27.000Z"), "id": 684071908900352000, "text": "Want to work in #Atlanta, GA? View our latest opening: https://t.co/VvcIHfNCWS #Nursing #LPN #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.352759,33.909187"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Atlanta", "Nursing", "LPN", "Job", "Jobs", "Hiring" }}, "user": { "id": 3161440380, "name": "Northside Careers", "screen_name": "NorthsideCareer", "lang": "en", "location": "null", "create_at": date("2015-04-17"), "description": "Dedicated to providing clinical excellence balanced with compassionate care. Learn how you can make a difference and continue growing your career.", "followers_count": 16, "friends_count": 1, "statues_count": 450 }, "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-01-04T10:00:27.000Z"), "id": 684071909122691072, "text": "WFHawaii™ https://t.co/xgSN2GIpTZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 277042460, "name": "Darren Wong", "screen_name": "darrendub", "lang": "en", "location": "Island of New Penzance", "create_at": date("2011-04-04"), "description": "Interactive Designer @Inkling", "followers_count": 306, "friends_count": 781, "statues_count": 1373 }, "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-01-04T10:00:27.000Z"), "id": 684071909156323328, "text": "@chef_fasano it kinda does. It's the culture. It's the \"hippie\" mindset a lot of Ph fans have.", "in_reply_to_status": 684071291293417472, "in_reply_to_user": 1061560159, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1061560159 }}, "user": { "id": 41732793, "name": "Darsh Vader", "screen_name": "CMDausch", "lang": "en", "location": "Philadelphia / Tuscaloosa ", "create_at": date("2009-05-21"), "description": "Get in the Groove & Let the Good Times Roll - 22 y/o Alumnus of the University of Alabama - #RollTide #Phish #Dead #CBDB #UM #tDB", "followers_count": 664, "friends_count": 1247, "statues_count": 26830 }, "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-04T10:00:27.000Z"), "id": 684071909282025472, "text": "The new Rumic Poll of the day!! Your favorite female crush in InuYasha!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 795413605, "name": "It's a Rumic World!", "screen_name": "TheRumicWorld", "lang": "es", "location": "Rumic World", "create_at": date("2012-08-31"), "description": "News and information About the work of Rumiko Takahashi / Noticias e Información sobre el Trabajo de Rumiko Takahashi", "followers_count": 1117, "friends_count": 1353, "statues_count": 1356 }, "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-04T10:00:27.000Z"), "id": 684071909328314369, "text": "Expecting the worst when I walk into my house later ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 301705930, "name": "Amanda∞Bowman", "screen_name": "manderrrrs", "lang": "en", "location": "null", "create_at": date("2011-05-19"), "description": "MU '16 Let go a little and just let life happen♈ Miguel Rivera 9/1/13", "followers_count": 645, "friends_count": 553, "statues_count": 26645 }, "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-04T10:00:27.000Z"), "id": 684071909366054916, "text": "Wind 2.0 mph NNW. Barometer 30.356 in, Rising. 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("-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": 287, "statues_count": 7254 }, "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-04T10:00:27.000Z"), "id": 684071909487685632, "text": "Midterm in bio tmr ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2161097285, "name": "vee", "screen_name": "ynaoivw", "lang": "en", "location": "null", "create_at": date("2013-10-30"), "description": "all bout hers ✨", "followers_count": 603, "friends_count": 500, "statues_count": 22961 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Methuen Town, MA", "id": "01597161672b6499", "name": "Methuen Town", "place_type": "city", "bounding_box": rectangle("-71.255938,42.6704 -71.115547,42.794273") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25009, "countyName": "Essex", "cityID": 2540710, "cityName": "Methuen Town" } }
+{ "create_at": datetime("2016-01-04T10:00:27.000Z"), "id": 684071909504466944, "text": "More #inking of a #piano man for an #asm promotion piece for their next show in January.… https://t.co/eUKGTUMdMq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.98112938,40.66760476"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "inking", "piano", "asm" }}, "user": { "id": 228543350, "name": "Diana Schoenbrun", "screen_name": "dianaschoenbrun", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2010-12-19"), "description": "Illustrator, 3-D artist, Author", "followers_count": 309, "friends_count": 399, "statues_count": 842 }, "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-04T10:00:27.000Z"), "id": 684071909940563968, "text": "Mom? https://t.co/AK78zTbkuF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3194747971, "name": "samantha", "screen_name": "Ptsamantha", "lang": "en", "location": "null", "create_at": date("2015-05-13"), "description": "null", "followers_count": 67, "friends_count": 65, "statues_count": 706 }, "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-04T10:00:27.000Z"), "id": 684071910401949697, "text": "@darkpassenger_x :'-) I love u my sweet peanut", "in_reply_to_status": 684070790938148864, "in_reply_to_user": 261046439, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 261046439 }}, "user": { "id": 541267649, "name": "h", "screen_name": "pukecarpet", "lang": "en", "location": "made u look lmao", "create_at": date("2012-03-30"), "description": "look at my location\n\n~team evian water~", "followers_count": 954, "friends_count": 192, "statues_count": 86033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wyandotte, MI", "id": "5d9b412793ef5656", "name": "Wyandotte", "place_type": "city", "bounding_box": rectangle("-83.181455,42.183685 -83.141012,42.235102") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2688900, "cityName": "Wyandotte" } }
+{ "create_at": datetime("2016-01-04T10:00:27.000Z"), "id": 684071910775259136, "text": "@fmanjoo sounds like a rom com", "in_reply_to_status": 684071626485334016, "in_reply_to_user": 2195241, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2195241 }}, "user": { "id": 16499261, "name": "Dan Monaghan", "screen_name": "danmonaghan", "lang": "en", "location": "Seattle", "create_at": date("2008-09-28"), "description": "Kinetic Poet * I love everybody. Including you stupid fuckers.", "followers_count": 1503, "friends_count": 2366, "statues_count": 69237 }, "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-04T10:00:27.000Z"), "id": 684071910812995584, "text": "General Manager - SONIC Drive-In: (#Florence, KY) https://t.co/lD10DgKSDe #Hospitality #SONIC #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.665645,38.969744"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Florence", "Hospitality", "SONIC", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 24173481, "name": "TMJ-CIN HRTA Jobs", "screen_name": "tmj_cin_hrta", "lang": "en", "location": "Cincinnati, OH", "create_at": date("2009-03-13"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Cincinnati, OH. Need help? Tweet us at @CareerArc!", "followers_count": 349, "friends_count": 291, "statues_count": 371 }, "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-01-04T10:00:27.000Z"), "id": 684071910842478593, "text": "2016 is going to be all about treasuring my friends and family ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 75886110, "name": "Αmanda", "screen_name": "caasIadnamA", "lang": "en", "location": "LI // Fredonia", "create_at": date("2009-09-20"), "description": "The greater your storm, the brighter the rainbow.", "followers_count": 331, "friends_count": 207, "statues_count": 34496 }, "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-04T10:00:27.000Z"), "id": 684071911366602752, "text": "Should I make this my Avi? https://t.co/6kcLb1AUWB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 392865979, "name": "TERIYAKI SMASH", "screen_name": "SpecialEdMont", "lang": "en", "location": "SpecialEd, Tx", "create_at": date("2011-10-17"), "description": "Archiemusictv@gmail.com #NeverSatisfied", "followers_count": 3879, "friends_count": 911, "statues_count": 79947 }, "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-04T10:00:27.000Z"), "id": 684071911421169665, "text": "We're #hiring! Read about our latest #job opening here: Medical Technologist/Clinical Laboratory... - https://t.co/F2Tw1TVVUd #Healthcare", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.0705556,29.9647222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Healthcare" }}, "user": { "id": 31610917, "name": "TM-LAN Health Jobs", "screen_name": "tmj_lan_health", "lang": "en", "location": "New Orleans, LA", "create_at": date("2009-04-15"), "description": "Follow this account for geo-targeted Healthcare job tweets in New Orleans, LA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 461, "friends_count": 307, "statues_count": 87 }, "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-04T10:00:27.000Z"), "id": 684071911807123457, "text": "You wonder why you feel like shit after a burger for lunch ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1588553503, "name": "$€£F M@D€ I©θN", "screen_name": "Self_Made_Icon", "lang": "en", "location": "407", "create_at": date("2013-07-12"), "description": "What's going on #Orlando. It's your co-host from @The407Blend make sure you go follow our podcast.", "followers_count": 266, "friends_count": 389, "statues_count": 2696 }, "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-04T10:00:27.000Z"), "id": 684071911912017921, "text": "@HeyUmpy @heavy_early @Raptors I'd say they played about 36 min vs Chicago", "in_reply_to_status": 684071673071452160, "in_reply_to_user": 61174226, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 61174226, 41221968, 73406718 }}, "user": { "id": 2180301955, "name": "Leo Rautins", "screen_name": "LeoRautins", "lang": "en", "location": "null", "create_at": date("2013-11-07"), "description": "@Raptors TV Analyst for @NBAonTSN @Sportsnet @NBATVCanada / Family, Friends, Dogs, #416 @CanBball @AlumniHoopsCan @Cuse @SMCS1852 forever!!", "followers_count": 15677, "friends_count": 633, "statues_count": 40634 }, "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-04T10:00:28.000Z"), "id": 684071912436314112, "text": "Joy! (at @Cinemark Movies 14 in Mishawaka, IN) https://t.co/WCXHTyCCeN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.19422436,41.69640438"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 374688790 }}, "user": { "id": 16976581, "name": "chrismskibbe", "screen_name": "chrismskibbe", "lang": "en", "location": "South Bend, IN", "create_at": date("2008-10-25"), "description": "I'm a videographer and editor. Also write, direct, and produce short films. Currently work at WNDU as videographer, editor, and director.", "followers_count": 312, "friends_count": 483, "statues_count": 6970 }, "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-01-04T10:00:28.000Z"), "id": 684071912541175808, "text": "@BrotherVet @sharon4marie @baalter @GatesRobin @sarobards @Kirk_Hinote @AURobNY @CajunJarhead @JAmy208 @ter2459 You can't fix stupid.", "in_reply_to_status": 684059594889363456, "in_reply_to_user": 384330450, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 384330450, 2632757957, 965259182, 438050333, 930566035, 2944909977, 2781324344, 25974930, 577337789, 1291118269 }}, "user": { "id": 221034430, "name": "Robert Mikaloff", "screen_name": "robertthedeuce", "lang": "en", "location": "De Soto, KS", "create_at": date("2010-11-29"), "description": "Conservative, once a career soldier, a veteran, OEF, not wed to GOP. The way my luck is running, if I was a politician I would be honest. Rodney Dangerfield", "followers_count": 931, "friends_count": 1205, "statues_count": 11421 }, "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-04T10:00:28.000Z"), "id": 684071912603955201, "text": "how does one acquire more \"twitter trolls\"? they can be kind of fun. like a gremlin...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25827743, "name": "steven spencer", "screen_name": "sspencer_smb", "lang": "en", "location": "nyc", "create_at": date("2009-03-22"), "description": "prop trader in nyc. focus is intra-day&swing. \n2014 will start to position trade... find me on rebelmouse http://www.rebelmouse.com/sspencer_smb", "followers_count": 13505, "friends_count": 380, "statues_count": 36471 }, "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-04T10:00:28.000Z"), "id": 684071912805371904, "text": "@KingLexington87 lol damn that's $15 ����", "in_reply_to_status": 684071447673880576, "in_reply_to_user": 33185177, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 33185177 }}, "user": { "id": 331884008, "name": "Gilly", "screen_name": "Yessmocha", "lang": "en", "location": " ", "create_at": date("2011-07-08"), "description": "Proud mother of a Beautiful little girl name Ryleigh,everthing i do i do it for her #INRYLEIGHWETRUST instagram/snapchat @Mocha_Lemon", "followers_count": 709, "friends_count": 734, "statues_count": 55449 }, "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-04T10:00:28.000Z"), "id": 684071912813756421, "text": "She give it too me when I want it \n\n ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2841665558, "name": "Sirachi papi", "screen_name": "kythe_guy", "lang": "en", "location": "Springfield, IL", "create_at": date("2014-10-05"), "description": "Iv always seen more with my eyes closed.", "followers_count": 173, "friends_count": 218, "statues_count": 3754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverton, IL", "id": "8f0c7d5ff0d261eb", "name": "Riverton", "place_type": "city", "bounding_box": rectangle("-89.558647,39.818284 -89.520842,39.863335") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17167, "countyName": "Sangamon", "cityID": 1764486, "cityName": "Riverton" } }
+{ "create_at": datetime("2016-01-04T10:00:28.000Z"), "id": 684071912910139392, "text": "#Sales #Job alert: Summer Student: Office | Vermeer Corporation | #Pella, Iowa https://t.co/6XrvvmgKiX #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.916405,41.4080327"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "Job", "Pella", "Jobs", "Hiring" }}, "user": { "id": 3321760873, "name": "Vermeer Jobs", "screen_name": "vermeerjobs", "lang": "en", "location": "Iowa & Across the Globe", "create_at": date("2015-08-20"), "description": "More than 3,000 team members designing, #manufacturing & supporting #industrial & #agricultural equipment to make a worldwide impact. Dig in! #job #jobs #hiring", "followers_count": 61, "friends_count": 74, "statues_count": 77 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pella, IA", "id": "810c9fe0da0bc3f5", "name": "Pella", "place_type": "city", "bounding_box": rectangle("-92.958155,41.374499 -92.890256,41.429928") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19125, "countyName": "Marion", "cityID": 1962040, "cityName": "Pella" } }
+{ "create_at": datetime("2016-01-04T10:00:28.000Z"), "id": 684071913052893184, "text": "#VerbalDisturbance at 4100-4199 Equity Row. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4288999,28.4447401"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VerbalDisturbance", "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": 274, "friends_count": 1, "statues_count": 67651 }, "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-01-04T10:00:28.000Z"), "id": 684071913073840128, "text": "Welp �� https://t.co/OLvc7L6vW6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2483076481, "name": "CayPitt✨", "screen_name": "shes_unek", "lang": "en", "location": "null", "create_at": date("2014-05-07"), "description": "Pretty, cool. Literally. Princess Caleigh. VIRGO ♍️ #foreverloveNesh #flyhighTeejay love you man.", "followers_count": 587, "friends_count": 436, "statues_count": 15826 }, "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-04T10:00:28.000Z"), "id": 684071913283567617, "text": "про весну... https://t.co/GylKi0MpEj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-123.01242649,37.69988626"), "retweet_count": 0, "lang": "uk", "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": 9975 }, "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-04T10:00:28.000Z"), "id": 684071913652539392, "text": "No school��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3300107724, "name": "Kylee", "screen_name": "Kylee59401793", "lang": "en", "location": "null", "create_at": date("2015-07-28"), "description": "FGDA Dancer|", "followers_count": 139, "friends_count": 490, "statues_count": 711 }, "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-04T10:00:28.000Z"), "id": 684071913723826176, "text": "#AirForceAmy https://t.co/EV6D6puW1B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "AirForceAmy" }}, "user": { "id": 2433752852, "name": "Nic Gonzales", "screen_name": "n_gonzales5733", "lang": "en", "location": "null", "create_at": date("2014-04-08"), "description": "18+ Only Please. I love Sports, Movies, Music and spending time with friends, and the Incredible people of Twitter.", "followers_count": 2244, "friends_count": 420, "statues_count": 37073 }, "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-04T10:00:28.000Z"), "id": 684071913866448897, "text": "I have no idea how I'm going to wake up tomorrow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1029551695, "name": "Lyndsey Achs", "screen_name": "lynds_a7", "lang": "en", "location": "Vandalia, OH", "create_at": date("2012-12-22"), "description": "Butler Hs snapchat: Lyndsey_achs", "followers_count": 634, "friends_count": 892, "statues_count": 3349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vandalia, OH", "id": "969a2f2b39d36110", "name": "Vandalia", "place_type": "city", "bounding_box": rectangle("-84.254898,39.834282 -84.162034,39.92978") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39113, "countyName": "Montgomery", "cityID": 3979492, "cityName": "Vandalia" } }
+{ "create_at": datetime("2016-01-04T10:00:28.000Z"), "id": 684071914059468800, "text": "Shower was 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": 181435294, "name": "O1/17", "screen_name": "petty_laBella", "lang": "en", "location": "DM[V)", "create_at": date("2010-08-21"), "description": "❤️", "followers_count": 1335, "friends_count": 900, "statues_count": 80312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Boston, VA", "id": "33f91bef2fc69db5", "name": "South Boston", "place_type": "city", "bounding_box": rectangle("-78.951141,36.693374 -78.872754,36.746024") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51083, "countyName": "Halifax", "cityID": 5173712, "cityName": "South Boston" } }
+{ "create_at": datetime("2016-01-04T10:00:28.000Z"), "id": 684071914176811008, "text": "duh bitch https://t.co/d204C2WXGf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1093565606, "name": "shady mom", "screen_name": "Monicaparas_", "lang": "en", "location": "null", "create_at": date("2013-01-15"), "description": "who's the little mouse now?", "followers_count": 350, "friends_count": 294, "statues_count": 15280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-04T10:00:28.000Z"), "id": 684071914281799681, "text": "��❄️⛅️ @ South Side Richmond https://t.co/xQZP94w3R6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.49049005,37.49428258"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1040888395, "name": "Josué Rivas", "screen_name": "EJozue", "lang": "en", "location": "Santa Rosa De Lima, El Salvador", "create_at": date("2012-12-27"), "description": "● Apocalipsis 22:13 ● Nadie Encuentra Su Camino Sin Haberse Perdido varias VECES⌚️ ● El Salvador - Sta. Rosa De Lima.", "followers_count": 355, "friends_count": 1128, "statues_count": 682 }, "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-04T10:00:28.000Z"), "id": 684071914294218752, "text": "Ima be in my hometown in 12 days!���� #sooooohappy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sooooohappy" }}, "user": { "id": 563357853, "name": "Des.", "screen_name": "DezzzyG", "lang": "en", "location": "Dallas, TX", "create_at": date("2012-04-25"), "description": "the names destiney, destiney guzman. Texas livin. God always got my back. thayvo❤ 8•27•15 ️", "followers_count": 929, "friends_count": 968, "statues_count": 5030 }, "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-04T10:00:28.000Z"), "id": 684071914395058176, "text": "New Year's Resolution - Work Smarter https://t.co/4zsMR4MZr2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.138923,26.119329"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 913680438, "name": "Stephen Ekstrom", "screen_name": "sparkresults", "lang": "en", "location": "New York, New York", "create_at": date("2012-10-29"), "description": "Marketing genius, tourism expert, talented in the kitchen & kinda funny.", "followers_count": 104, "friends_count": 29, "statues_count": 217 }, "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-04T10:00:28.000Z"), "id": 684071914642345986, "text": "I've felt so sick since New Year's Day like happy 2016", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2975527516, "name": "xxxxx", "screen_name": "artsoll", "lang": "en", "location": "north colorado", "create_at": date("2015-01-12"), "description": "19 and living. go look at my artwork :-) personal: @_baeker", "followers_count": 632, "friends_count": 1202, "statues_count": 3623 }, "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-01-04T10:00:28.000Z"), "id": 684071914718015488, "text": "#TrafficLight at W Irlo Bronson Memorial Hwy & E Orange Lake Blvd, Kissimmee, FL 34747. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.6126819,28.3470816"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TrafficLight", "orlpol", "ocso" }}, "user": { "id": 42131402, "name": "Police Calls 34747", "screen_name": "orlpol34747", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-23"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 26, "friends_count": 1, "statues_count": 474 }, "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": 12097, "countyName": "Osceola", "cityID": 1224581, "cityName": "Four Corners" } }
+{ "create_at": datetime("2016-01-04T10:00:28.000Z"), "id": 684071914894192640, "text": "Drowning in 3 ft. of Water - 1.0 https://t.co/mBbyKM9h6L", "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": 314113528, "name": "Lucas Aufenkamp", "screen_name": "lucasaufenkamp", "lang": "en", "location": "null", "create_at": date("2011-06-09"), "description": "Husband, dad, pastor, writer, apologist", "followers_count": 123, "friends_count": 116, "statues_count": 622 }, "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-04T10:00:28.000Z"), "id": 684071915057623040, "text": "Miss you too ��❤️ https://t.co/sqbm0JUgsh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2211490292, "name": "Asia.❤️Mae.", "screen_name": "AredAsia", "lang": "en", "location": "Washington ✈️ Texas", "create_at": date("2013-11-23"), "description": "Just be YOU. ❤️", "followers_count": 440, "friends_count": 363, "statues_count": 5516 }, "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-04T10:00:28.000Z"), "id": 684071915229700096, "text": "https://t.co/PTsKOc7E5N", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1700731190, "name": "mary", "screen_name": "_maryfernandez", "lang": "en", "location": "ATLANTA,GA", "create_at": date("2013-08-25"), "description": "null", "followers_count": 564, "friends_count": 206, "statues_count": 38317 }, "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-04T10:00:28.000Z"), "id": 684071915267440640, "text": "@OGKarrol lmfao backwoods are deadass for special occasions", "in_reply_to_status": 684071303909752832, "in_reply_to_user": 149008371, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 149008371 }}, "user": { "id": 190076085, "name": "honeyberrybri!", "screen_name": "selfiequeenbri", "lang": "en", "location": "RECKLESSTX 2 HOWARDU", "create_at": date("2010-09-12"), "description": "NEVERFOLDSAYLESSDOMORE", "followers_count": 3729, "friends_count": 985, "statues_count": 207096 }, "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-04T10:00:28.000Z"), "id": 684071915376361472, "text": "If she texts you and tell you \"it's ok\" I promise it's not ok��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2201305852, "name": ".", "screen_name": "EvansDemario", "lang": "en", "location": "null", "create_at": date("2013-11-29"), "description": "null", "followers_count": 195, "friends_count": 457, "statues_count": 2711 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-04T10:00:28.000Z"), "id": 684071915489751040, "text": "How did I survive without 18k a month!? Huh? https://t.co/oowBgvibE4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 61601739, "name": "ㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤ", "screen_name": "MiaMiaMia_", "lang": "en", "location": "ATL", "create_at": date("2009-07-30"), "description": "Atlanta — Artist⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀", "followers_count": 1738, "friends_count": 248, "statues_count": 33860 }, "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-04T10:00:28.000Z"), "id": 684071915506409472, "text": "@BanCollectivism @danrandolph10 yes you are you just won't admit it at least the protesters do", "in_reply_to_status": 684070997050327040, "in_reply_to_user": 429533485, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 429533485, 2505999475 }}, "user": { "id": 2563629540, "name": "Dave", "screen_name": "stanleycupcham1", "lang": "en", "location": "Utah, USA", "create_at": date("2014-06-12"), "description": "Trying to find a way to enjoy life more and work less.", "followers_count": 34, "friends_count": 79, "statues_count": 1958 }, "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-01-04T10:00:28.000Z"), "id": 684071915523338241, "text": "Might have to come in late tomorrow ☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 233425321, "name": "DrewHen", "screen_name": "drewINGTON", "lang": "en", "location": "in your panties", "create_at": date("2011-01-02"), "description": "Fighting on Arrival, Fighting for Survival http://mixcrate.com/drewington LISTEN TO MY MIXES!", "followers_count": 1297, "friends_count": 780, "statues_count": 183994 }, "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-04T10:00:28.000Z"), "id": 684071915602874368, "text": "I know #JohnnyFootball wants to go to #Dallas but I'm pretty sure he's not #DallasCowboysCheerleader material https://t.co/xWWweQXVRn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "JohnnyFootball", "Dallas", "DallasCowboysCheerleader" }}, "user": { "id": 22794926, "name": "audrey jablonski", "screen_name": "ajabs4258", "lang": "en", "location": "cleveland(west park), ohio", "create_at": date("2009-03-04"), "description": "Cavs Fanatic, Faith Based, Family! don't judge me & I won't judge you \nAnimal Advocate, ALL LIVES MATTER \n University of Miami", "followers_count": 567, "friends_count": 1980, "statues_count": 10501 }, "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-04T10:00:28.000Z"), "id": 684071915607179264, "text": "When life gives you Lemons (or Oranges), trade them with idiot kids for their Cherry and Pink", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 95928470, "name": "Doug Bies", "screen_name": "dougbies", "lang": "en", "location": "Long Island, New York", "create_at": date("2009-12-10"), "description": " Snob, Beer Snob, Chicago native, Candy fanatic, Proud Dad of fish (1), Hipster-friendly, Basic intolerant, Larry David idol; IG, Snapchat, & Untappd @dougbies", "followers_count": 7192, "friends_count": 5929, "statues_count": 28413 }, "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-01-04T10:00:28.000Z"), "id": 684071915636535296, "text": "the weeknd ft. labrinth - loser https://t.co/nbjCrJ0qVP #theweeknd #musicblog #music", "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": {{ "theweeknd", "musicblog", "music" }}, "user": { "id": 1059334374, "name": "char", "screen_name": "jubilantdreams", "lang": "en", "location": "orlando", "create_at": date("2013-01-03"), "description": "18. music and movie fanatic. sound & music tech. artist/tour management. business inquiries: charriverabennett@yahoo.com", "followers_count": 3375, "friends_count": 568, "statues_count": 46426 }, "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-04T10:00:28.000Z"), "id": 684071916051677184, "text": "@CStalling I hate you", "in_reply_to_status": 684071546680262656, "in_reply_to_user": 3245569350, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3245569350 }}, "user": { "id": 2596793167, "name": "Jendaya Christmas", "screen_name": "jayyclaus", "lang": "en", "location": "null", "create_at": date("2014-06-30"), "description": "null", "followers_count": 229, "friends_count": 213, "statues_count": 2114 }, "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-01-04T10:00:28.000Z"), "id": 684071916412469248, "text": "@Anmol_Pannu ooh yeah momma Pannu pouring that hot masala chai", "in_reply_to_status": 684044658918461440, "in_reply_to_user": 335075053, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 335075053 }}, "user": { "id": 438928328, "name": "Keegan Fouse", "screen_name": "KungFouse", "lang": "en", "location": "null", "create_at": date("2011-12-16"), "description": "hi I'm Jenna. I'm not here to party, I'm here to scissor", "followers_count": 805, "friends_count": 698, "statues_count": 17168 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mohrsville, PA", "id": "01580d5c6ee1c4f8", "name": "Mohrsville", "place_type": "city", "bounding_box": rectangle("-75.996324,40.467839 -75.973987,40.480378") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42011, "countyName": "Berks", "cityID": 4250280, "cityName": "Mohrsville" } }
+{ "create_at": datetime("2016-01-04T10:00:28.000Z"), "id": 684071916458643456, "text": "The Guys' 2015 NFL Picks Revisited https://t.co/Q2yosRHOGM", "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": 271167530, "name": "The Guys From", "screen_name": "guysfromblank", "lang": "en", "location": "Calgary, Alberta", "create_at": date("2011-03-23"), "description": "Home of sarcastic insight into film, tv, sports, news and gaming. Plus news on The Guys From {BLANK} comedy", "followers_count": 249, "friends_count": 19, "statues_count": 504 }, "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-04T10:00:29.000Z"), "id": 684071916546691072, "text": "I'm so lazy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2785029852, "name": "ashley anna", "screen_name": "asssleyyyyy", "lang": "en", "location": "null", "create_at": date("2014-09-01"), "description": "Emanuel", "followers_count": 347, "friends_count": 182, "statues_count": 15882 }, "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-01-04T10:00:29.000Z"), "id": 684071916638842881, "text": "�� https://t.co/rrQpzyYtcj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3469185194, "name": "babygirl", "screen_name": "irisnicoleee", "lang": "en", "location": "cali", "create_at": date("2015-09-06"), "description": "null", "followers_count": 642, "friends_count": 636, "statues_count": 13681 }, "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-04T10:00:29.000Z"), "id": 684071916756283393, "text": "This #Retail #job might be a great fit for you: Customer Service Representative - https://t.co/wAG9ejwrOY #ClintonTownship, MI #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.9439799,42.6257735"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "ClintonTownship", "Hiring" }}, "user": { "id": 59951643, "name": "TMJ-MI Retail Jobs", "screen_name": "tmj_mi_retail", "lang": "en", "location": "Michigan", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in Michigan Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 400, "friends_count": 304, "statues_count": 963 }, "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.539274 -82.855854,42.630481") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26091, "countyName": "Lenawee", "cityID": 2616480, "cityName": "Clinton" } }
+{ "create_at": datetime("2016-01-04T10:00:29.000Z"), "id": 684071916798357504, "text": "Yep https://t.co/YmpXi7RFax", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 468961328, "name": "King Nathan", "screen_name": "rodriguezdagawd", "lang": "en", "location": "Norfolk, VA", "create_at": date("2012-01-19"), "description": "@Ashley_Bxtchess | Miami ✈️Norfolk | #ImUglyButMyGirlAintHive | #Sneedssons", "followers_count": 5907, "friends_count": 696, "statues_count": 114761 }, "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-04T10:00:29.000Z"), "id": 684071917016498176, "text": "@KellyMadeira44 happy birthday Kelly!! Love you❤️❤️", "in_reply_to_status": -1, "in_reply_to_user": 561518171, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 561518171 }}, "user": { "id": 1058705592, "name": "zach stark", "screen_name": "starky7777", "lang": "en", "location": "null", "create_at": date("2013-01-03"), "description": "Rangers❤️⚪️Jets✈️", "followers_count": 265, "friends_count": 153, "statues_count": 6899 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montebello, NY", "id": "f438230c852518a1", "name": "Montebello", "place_type": "city", "bounding_box": rectangle("-74.142947,41.112235 -74.088961,41.151183") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36087, "countyName": "Rockland", "cityID": 3648090, "cityName": "Montebello" } }
+{ "create_at": datetime("2016-01-04T10:00:29.000Z"), "id": 684071917159059456, "text": "@JDCHS getting some great time with direct supervisors! Go Seniors. https://t.co/FeUWqj9EYF", "in_reply_to_status": -1, "in_reply_to_user": 61062845, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 61062845 }}, "user": { "id": 1360154623, "name": "Beth Clemenger", "screen_name": "bwclemenger", "lang": "en", "location": "null", "create_at": date("2013-04-17"), "description": "null", "followers_count": 2, "friends_count": 29, "statues_count": 7 }, "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-04T10:00:29.000Z"), "id": 684071917377159168, "text": "Working out with @korean_breeze later! Bout to see what those little guns are made of! ��✊������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2363333455 }}, "user": { "id": 1934979391, "name": "Joshua Rios", "screen_name": "josh02192443", "lang": "en", "location": "RGF Island", "create_at": date("2013-10-04"), "description": "Deez nutz before skinny sluts", "followers_count": 278, "friends_count": 323, "statues_count": 4903 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mason, MI", "id": "01c4b9cd35e9b6ba", "name": "Mason", "place_type": "city", "bounding_box": rectangle("-84.483582,42.517086 -84.422915,42.611176") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2652180, "cityName": "Mason" } }
+{ "create_at": datetime("2016-01-04T10:00:29.000Z"), "id": 684071918094385153, "text": "@ustedesmuyfeo the only negative though. Random shit https://t.co/aFAF0iIomP", "in_reply_to_status": 684071006781206528, "in_reply_to_user": 2725062476, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2725062476 }}, "user": { "id": 2725062476, "name": "NovembersVeryOwn", "screen_name": "ustedesmuyfeo", "lang": "en", "location": "yo momma panties", "create_at": date("2014-08-11"), "description": "Leader of the Dinosaurs // #FIU18 // #$QUAD ❌⭕️ // Leader of the Philippines", "followers_count": 816, "friends_count": 669, "statues_count": 4121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University Park, FL", "id": "883ee3f98abb082b", "name": "University Park", "place_type": "city", "bounding_box": rectangle("-80.385477,25.732128 -80.351653,25.761752") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1273287, "cityName": "University Park" } }
+{ "create_at": datetime("2016-01-04T10:00:29.000Z"), "id": 684071918475939840, "text": "Interested in a #Nursing #job near #Cheyenne, WY? This could be a great fit: https://t.co/79i7bKApqI #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.8202462,41.1399814"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "Cheyenne", "Hiring", "CareerArc" }}, "user": { "id": 49870147, "name": "Cheyenne Nursing", "screen_name": "tmj_wyc_nursing", "lang": "en", "location": "Cheyenne, WY", "create_at": date("2009-06-22"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Cheyenne, WY from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 332, "friends_count": 258, "statues_count": 44 }, "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-04T10:00:29.000Z"), "id": 684071918564052992, "text": "Is it mean that I want Aiko to attack June again?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 975125029, "name": "cinnamon", "screen_name": "emmycabb", "lang": "en", "location": "null", "create_at": date("2012-11-27"), "description": "until the oceans in me reflect the skies in you.", "followers_count": 441, "friends_count": 368, "statues_count": 22591 }, "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-04T10:00:29.000Z"), "id": 684071918778105857, "text": "Never leave your car anywhere , bad idea .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 260507878, "name": "jas", "screen_name": "_halffpint", "lang": "en", "location": "null", "create_at": date("2011-03-03"), "description": "null", "followers_count": 1038, "friends_count": 948, "statues_count": 39880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warrensville Heights, OH", "id": "166b716425d5bb87", "name": "Warrensville Heights", "place_type": "city", "bounding_box": rectangle("-81.555545,41.423881 -81.488156,41.457349") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3980990, "cityName": "Warrensville Heights" } }
+{ "create_at": datetime("2016-01-04T10:00:29.000Z"), "id": 684071919218495488, "text": "@leahisabelle_ how u get him on a schedule", "in_reply_to_status": 684071144677232641, "in_reply_to_user": 296417379, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 296417379 }}, "user": { "id": 348199307, "name": "Jurnee's Mommy♡", "screen_name": "Simply_joylene", "lang": "en", "location": "Longg Beaaach", "create_at": date("2011-08-03"), "description": "Jurnee Denise Warren ♡♡", "followers_count": 390, "friends_count": 343, "statues_count": 37691 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, MS", "id": "6e6ed692ac0b92c9", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-89.203925,30.331174 -89.124696,30.383762") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28047, "countyName": "Harrison", "cityID": 2841680, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-04T10:00:29.000Z"), "id": 684071919440773122, "text": "I just be in the house lol chilling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 322352701, "name": "GOTTI", "screen_name": "elgotti___", "lang": "en", "location": "null", "create_at": date("2011-06-22"), "description": "19|", "followers_count": 1847, "friends_count": 734, "statues_count": 75179 }, "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-04T10:00:29.000Z"), "id": 684071919709196290, "text": "This what got the child support topic going LMAO https://t.co/R5BOQ74RTC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 501171723, "name": ".", "screen_name": "sheCheatnOnU", "lang": "en", "location": "Baltimore, MD", "create_at": date("2012-02-23"), "description": "null", "followers_count": 1191, "friends_count": 176, "statues_count": 71999 }, "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-04T10:00:29.000Z"), "id": 684071919973478400, "text": "East Side Elementary Student Sophie Peterson. #faithoverfear @RaceForWandell @ALSTennessee https://t.co/2FxIzlieiz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "faithoverfear" }}, "user_mentions": {{ 1589974646, 593779417 }}, "user": { "id": 391281633, "name": "T.J. Brown", "screen_name": "TJBrown26", "lang": "en", "location": "Johnson City, TN", "create_at": date("2011-10-15"), "description": "There's a lot of beauty in ordinary things. HELP FIGHT ALS", "followers_count": 829, "friends_count": 784, "statues_count": 20583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elizabethton, TN", "id": "00bad99d6b05a213", "name": "Elizabethton", "place_type": "city", "bounding_box": rectangle("-82.319523,36.285879 -82.154114,36.368033") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47019, "countyName": "Carter", "cityID": 4723500, "cityName": "Elizabethton" } }
+{ "create_at": datetime("2016-01-04T10:00:29.000Z"), "id": 684071920065630209, "text": "Can you recommend anyone for this #job? Social Media Community Manager - https://t.co/3Ualr9JDrP #NewYork, NY #Marketing #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": {{ "job", "NewYork", "Marketing", "Hiring" }}, "user": { "id": 3004246198, "name": "Colgate Jobs - U.S.", "screen_name": "ColgateJobsUS", "lang": "en", "location": "United States", "create_at": date("2015-01-29"), "description": "null", "followers_count": 1142, "friends_count": 30, "statues_count": 113 }, "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-04T10:00:29.000Z"), "id": 684071920526995456, "text": "@mcmillan_glen lol", "in_reply_to_status": 684058735036137473, "in_reply_to_user": 3044792371, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3044792371 }}, "user": { "id": 1055403102, "name": "taylor reising", "screen_name": "taylor_reising", "lang": "en", "location": "null", "create_at": date("2013-01-02"), "description": "null", "followers_count": 520, "friends_count": 525, "statues_count": 3931 }, "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-01-04T10:00:29.000Z"), "id": 684071920619270144, "text": "If you're a #InvestmentBanking professional in #Merrimack, NH, check out this #job: https://t.co/MbXELDvaWh #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.4948355,42.8678651"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "InvestmentBanking", "Merrimack", "job", "Hiring", "CareerArc" }}, "user": { "id": 207352965, "name": "NH Inv. Banking", "screen_name": "tmj_NH_invbank", "lang": "en", "location": "New Hampshire", "create_at": date("2010-10-24"), "description": "Follow this account for geo-targeted Investment Banking job tweets in New Hampshire Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 239, "friends_count": 235, "statues_count": 13 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merrimack, NH", "id": "000a400a1aba18e8", "name": "Merrimack", "place_type": "city", "bounding_box": rectangle("-71.589882,42.790953 -71.453424,42.910922") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33011, "countyName": "Hillsborough", "cityID": 3321780, "cityName": "East Merrimack" } }
+{ "create_at": datetime("2016-01-04T10:00:29.000Z"), "id": 684071920631844864, "text": "Lol stuck at the airport.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 101365947, "name": "Justen", "screen_name": "J_Wattts", "lang": "en", "location": "Portland/Vegas", "create_at": date("2010-01-02"), "description": "Nap connoisseur, dabbling in Public Health. Syrian/Italian.", "followers_count": 217, "friends_count": 110, "statues_count": 19745 }, "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-04T10:00:29.000Z"), "id": 684071920715829249, "text": "@_samuel55 @FBIsLife okay best one I've witnessed", "in_reply_to_status": 684069993349267456, "in_reply_to_user": 2319497262, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2319497262, 2292180146 }}, "user": { "id": 416017736, "name": "Bailey A. Crane", "screen_name": "b_crane14", "lang": "en", "location": "Stanley, NC", "create_at": date("2011-11-18"), "description": "R.I.P JOSH miss you. Hope is never lost when you put your faith in God. Things may not be working out the way you planned but God has a plan far greater for you", "followers_count": 1579, "friends_count": 1379, "statues_count": 19638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stanley, NC", "id": "00bc58b24f870344", "name": "Stanley", "place_type": "city", "bounding_box": rectangle("-81.133373,35.321436 -81.075745,35.417608") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37071, "countyName": "Gaston", "cityID": 3764500, "cityName": "Stanley" } }
+{ "create_at": datetime("2016-01-04T10:00:30.000Z"), "id": 684071920791228416, "text": "@pattyguilmette Hey Lopez! Did you see this?", "in_reply_to_status": 684071600422023168, "in_reply_to_user": 25283105, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25283105 }}, "user": { "id": 25283105, "name": "Patty Guilmette", "screen_name": "pattyguilmette", "lang": "en", "location": "Massachusetts", "create_at": date("2009-03-19"), "description": "I'm a female author/poet/love ambassador. I write from the heart so my poems are easy to understand & straight to the point. Go purple for Epilepsy.", "followers_count": 138, "friends_count": 184, "statues_count": 1396 }, "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("2016-01-04T10:00:30.000Z"), "id": 684071920803921920, "text": "@yungsweetpea_ that shit went out the window when allyn called the fucking cops on me and got in my face, lmao.", "in_reply_to_status": 684071695896965120, "in_reply_to_user": 607785504, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 607785504 }}, "user": { "id": 607785504, "name": "Madison", "screen_name": "yungsweetpea_", "lang": "en", "location": "440", "create_at": date("2012-06-13"), "description": "You all wanna be seen, I just wanna be heard.", "followers_count": 341, "friends_count": 440, "statues_count": 15414 }, "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-01-04T10:00:30.000Z"), "id": 684071921005248513, "text": "Come to our first Youth Group/Bible Study of the Year today at 8 PM! Don't miss out on all the exciting new plans we have for our church!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4528872202, "name": "Youth On Fire PA", "screen_name": "MaranathaYM", "lang": "en", "location": "Perth Amboy, NJ", "create_at": date("2015-12-18"), "description": "Just some youth in love with Jesus . follow us on IG: @Yof_PA", "followers_count": 18, "friends_count": 32, "statues_count": 8 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Perth Amboy, NJ", "id": "6813a7d84bf141d5", "name": "Perth Amboy", "place_type": "city", "bounding_box": rectangle("-74.292392,40.499466 -74.250015,40.543459") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3458200, "cityName": "Perth Amboy" } }
+{ "create_at": datetime("2016-01-04T10:00:30.000Z"), "id": 684071921441488896, "text": "Hanging with Michael Dixon thesmokerszone so he can tell me about his trip to the Keys & I can… https://t.co/keDCQy5SK3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.9962082,40.147274"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2688097029, "name": "CigarMick", "screen_name": "CigarMick", "lang": "en", "location": "null", "create_at": date("2014-07-08"), "description": "null", "followers_count": 73, "friends_count": 93, "statues_count": 289 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Feasterville, PA", "id": "011e6a8e05912812", "name": "Feasterville", "place_type": "city", "bounding_box": rectangle("-75.00239,40.139757 -74.970898,40.163116") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42017, "countyName": "Bucks", "cityID": 4225496, "cityName": "Feasterville" } }
+{ "create_at": datetime("2016-01-04T10:00:30.000Z"), "id": 684071921462325252, "text": "In EXACTLY 5 months I will be walking down the stage in my cap and gown!��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3273398228, "name": "Destiny", "screen_name": "DestinySoto16", "lang": "en", "location": "Spurs Nation", "create_at": date("2015-07-09"), "description": "❤️: San Antonio Spurs & Sneakers", "followers_count": 166, "friends_count": 411, "statues_count": 2977 }, "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-04T10:00:30.000Z"), "id": 684071921810460672, "text": "Rapier and Cloak, now with cloaks! #stagecombat #actortraining @macenfp #rapier #swordplay @… https://t.co/1ndN7RNwzj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.8644714,41.9774704"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "stagecombat", "actortraining", "rapier", "swordplay" }}, "user_mentions": {{ 374323443 }}, "user": { "id": 1045513634, "name": "Craig Lawrence", "screen_name": "fightguyphoto", "lang": "en", "location": "Fairfax, Virginia", "create_at": date("2012-12-29"), "description": "Event, Portrait and artistic photographer in Northern Virginia.", "followers_count": 259, "friends_count": 317, "statues_count": 1246 }, "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-01-04T10:00:30.000Z"), "id": 684071921852530692, "text": "Goodmorning or evening 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": 3149858057, "name": "fr3⃣akybãbyy", "screen_name": "lahfreakyy", "lang": "en", "location": "somewhere buying honeybuns", "create_at": date("2015-04-08"), "description": "#RestDaddy#RestNatedawg\n12GanG9⃣0⃣'sBaBy.....DontGetKill(DGK).Niggas Hating Bitchs Love Me...ōVō.....", "followers_count": 585, "friends_count": 946, "statues_count": 1961 }, "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-04T10:00:30.000Z"), "id": 684071922175356928, "text": "Momma said I can get my belly pierced ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2460072898, "name": "IKE", "screen_name": "ikiyah_walker", "lang": "en", "location": "null", "create_at": date("2014-04-04"), "description": "7/16/15", "followers_count": 211, "friends_count": 173, "statues_count": 5142 }, "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-04T10:00:30.000Z"), "id": 684071922246791168, "text": "CC is this you?¿ https://t.co/Q2NYPtLLvu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 729462884, "name": "ⓐⓝⓢ", "screen_name": "ansleetuck", "lang": "en", "location": "912", "create_at": date("2012-07-31"), "description": "blessed by the grace of God✝", "followers_count": 948, "friends_count": 542, "statues_count": 7013 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hazlehurst, GA", "id": "493a49cec998862a", "name": "Hazlehurst", "place_type": "city", "bounding_box": rectangle("-82.62451,31.839955 -82.569544,31.883851") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13161, "countyName": "Jeff Davis", "cityID": 1337564, "cityName": "Hazlehurst" } }
+{ "create_at": datetime("2016-01-04T10:00:30.000Z"), "id": 684071922422824960, "text": "If you're a #Hospitality professional in #Houston, TX, check out this #job: https://t.co/5PEUe5X04y #IHOP #htx #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.1964494,29.7714176"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "Houston", "job", "IHOP", "htx", "Hiring" }}, "user": { "id": 216428496, "name": "IHOP_ACG", "screen_name": "IHOP_ACGTexas", "lang": "en", "location": "Richardson, TX", "create_at": date("2010-11-16"), "description": "ACG Texas is proud to be Texas' largest #IHOP Franchisee. #Hospitality #jobs in 66 restaurants in Texas. Join our Team!", "followers_count": 307, "friends_count": 619, "statues_count": 713 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-04T10:00:30.000Z"), "id": 684071922422951936, "text": "Greetings in the name of his H.I.MEmperor Haile Selassie I the first give the youth the teachings of H.I.M https://t.co/OPILMBeEGj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1195313984, "name": "roots....12 tribes", "screen_name": "rasking43", "lang": "en", "location": "null", "create_at": date("2013-02-18"), "description": "null", "followers_count": 196, "friends_count": 223, "statues_count": 2944 }, "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-04T10:00:30.000Z"), "id": 684071922494246913, "text": "@__MIGOOO no!", "in_reply_to_status": 684071812964036608, "in_reply_to_user": 64891503, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 64891503 }}, "user": { "id": 2269974621, "name": "❤️", "screen_name": "LaToriaaaaa", "lang": "en", "location": "null", "create_at": date("2014-01-08"), "description": "null", "followers_count": 1413, "friends_count": 1061, "statues_count": 21796 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Camp Springs, MD", "id": "4b656a23df9bee4a", "name": "Camp Springs", "place_type": "city", "bounding_box": rectangle("-76.945472,38.768037 -76.890929,38.835708") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2412600, "cityName": "Camp Springs" } }
+{ "create_at": datetime("2016-01-04T10:00:30.000Z"), "id": 684071922875920385, "text": "Hello WWE. I'm totally in. I'm out of shape & have no moves, but these masks? Cmon @TripleH #wwe #luchador #RAW https://t.co/6XbtokzHDZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wwe", "luchador", "RAW" }}, "user_mentions": {{ 355708717 }}, "user": { "id": 93704869, "name": "Matty Rock", "screen_name": "mattyrock2k", "lang": "en", "location": "NYC", "create_at": date("2009-11-30"), "description": "I love music, travel, cigars, playing guitar, laughing - What I do: Promotions / Productions / Photography / Marketing, Health - http://rock.dz10.com", "followers_count": 6452, "friends_count": 2320, "statues_count": 999 }, "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-04T10:00:30.000Z"), "id": 684071923052056576, "text": "Ready for you @BachelorABC \n�������� https://t.co/UtcxTCWAt2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 346878074 }}, "user": { "id": 1063603531, "name": "Olivia Eiken", "screen_name": "oliviaeiken91", "lang": "en", "location": "The Nearest Chipotle", "create_at": date("2013-01-05"), "description": "•Naperville North Aquatics•", "followers_count": 804, "friends_count": 399, "statues_count": 3463 }, "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-01-04T10:00:30.000Z"), "id": 684071923064659969, "text": "I'm at Apple Store, Short Hills in Short Hills, NJ https://t.co/TI0mY6Uqco", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.365168,40.740174"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 161125519, "name": "Carlos Suarez", "screen_name": "cafesuga", "lang": "es", "location": "San José - Costa Rica", "create_at": date("2010-06-29"), "description": "Bellas Artes-UCR. Productor y postproductor audiovisual.", "followers_count": 225, "friends_count": 292, "statues_count": 1773 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Millburn, NJ", "id": "1935ab8020a2553b", "name": "Millburn", "place_type": "city", "bounding_box": rectangle("-74.372452,40.713308 -74.284841,40.76105") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3467320, "cityName": "Short Hills" } }
+{ "create_at": datetime("2016-01-04T10:00:30.000Z"), "id": 684071923274260480, "text": "Despised in China\nAnna May Wong\nThe American actress who died 1000x\nhttps://t.co/bVDEwYoU9T\n#MondayBlogs\nhttps://t.co/TLdKDGCT1N", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MondayBlogs" }}, "user": { "id": 24728152, "name": "Lloyd Lofthouse", "screen_name": "lflwriter", "lang": "en", "location": "Northern California", "create_at": date("2009-03-16"), "description": "Thank You ReTweeters!\n\nI'm taking a sabbatical to write and will not be appearing regularly on Twitter. \n\n#AmWriting but not that much on Twitter.", "followers_count": 15287, "friends_count": 15993, "statues_count": 134412 }, "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-04T10:00:30.000Z"), "id": 684071923479896064, "text": "What a night! Thank you @CastilleJoshua for doing #Shrek with me! @54Below, I will miss you! https://t.co/j3N5QqhXQr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Shrek" }}, "user_mentions": {{ 2813614514, 455315694 }}, "user": { "id": 104658279, "name": "Sandra Mae Frank", "screen_name": "sandy21mae", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-01-13"), "description": "Broadway Debut this Fall as Wendla Bergman in Deaf West's Spring Awakening ❤️ http://www.springawakeningthemusical.com", "followers_count": 3432, "friends_count": 407, "statues_count": 524 }, "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-04T10:00:30.000Z"), "id": 684071923588857856, "text": "@_xerica ����������������������", "in_reply_to_status": 684071758127853569, "in_reply_to_user": 246070591, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 246070591 }}, "user": { "id": 433377317, "name": "AlmightyJae", "screen_name": "xjazaline_", "lang": "en", "location": "null", "create_at": date("2011-12-10"), "description": "CLE | 19 BlackRican.", "followers_count": 1033, "friends_count": 366, "statues_count": 23361 }, "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-04T10:00:30.000Z"), "id": 684071923769192450, "text": "Argosy Casino Hotel & Spa #CustomerService #Job: Cage Cashier - #Part-time (#Riverside, MO) https://t.co/X0kX3xAbf7 #cashier #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.6243649,39.162493"), "retweet_count": 0, "lang": "it", "is_retweet": false, "hashtags": {{ "CustomerService", "Job", "Part", "Riverside", "cashier", "Jobs", "Hiring" }}, "user": { "id": 1725772980, "name": "Argosy Casino KC", "screen_name": "ArgosyKC_Jobs", "lang": "en", "location": "null", "create_at": date("2013-09-03"), "description": "A Penn National Gaming property, Argosy Casino Hotel & Spa offers gaming excitement, luxury hotel rooms, fine dining and a rejuvenating spa.", "followers_count": 182, "friends_count": 137, "statues_count": 78 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, MO", "id": "f8e9521355ca84c2", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-94.659895,39.154401 -94.601419,39.189295") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29165, "countyName": "Platte", "cityID": 2962156, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-01-04T10:00:30.000Z"), "id": 684071923844820994, "text": "Stuck in 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": 2729111111, "name": ".", "screen_name": "selfmadeleqacy", "lang": "en", "location": "null", "create_at": date("2014-07-30"), "description": "null", "followers_count": 48, "friends_count": 52, "statues_count": 4203 }, "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-04T10:00:30.000Z"), "id": 684071924243279873, "text": "You know it's January when your social media and your #myfitnesspal app switch places", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "myfitnesspal" }}, "user": { "id": 397444370, "name": "amygdala", "screen_name": "amy_benenson9", "lang": "en", "location": "here and now", "create_at": date("2011-10-24"), "description": "sorry i smile so much. '15 [] UAlbany Equestrian Captain [] EMT❤️FQVAS", "followers_count": 423, "friends_count": 666, "statues_count": 19714 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elwood, NY", "id": "2d951a00b6c3c6fb", "name": "Elwood", "place_type": "city", "bounding_box": rectangle("-73.370637,40.827194 -73.308946,40.864857") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3624405, "cityName": "Elwood" } }
+{ "create_at": datetime("2016-01-04T10:00:30.000Z"), "id": 684071924322971653, "text": "@lysspojo ��", "in_reply_to_status": 684066927979933696, "in_reply_to_user": 2267102127, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2267102127 }}, "user": { "id": 386770352, "name": "Mary Kate O'Rourke", "screen_name": "MaryKatharine_O", "lang": "en", "location": "null", "create_at": date("2011-10-07"), "description": "Get on your feet... Get up and make it happen!", "followers_count": 727, "friends_count": 723, "statues_count": 14548 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shark River Hills, NJ", "id": "45f85361dc8cd523", "name": "Shark River Hills", "place_type": "city", "bounding_box": rectangle("-74.058114,40.184771 -74.038336,40.206282") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3466840, "cityName": "Shark River Hills" } }
+{ "create_at": datetime("2016-01-04T10:00:30.000Z"), "id": 684071924851326976, "text": "We're #hiring! Click to apply: Registered Nurse - RN - Wound Care - https://t.co/1lgtLlGF9g #Nursing #Allentown, PA #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.4683784,40.6083521"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Nursing", "Allentown", "Job", "Jobs", "CareerArc" }}, "user": { "id": 57569071, "name": "TMJ-PA Health Jobs", "screen_name": "tmj_pa_health", "lang": "en", "location": "Pennsylvania", "create_at": date("2009-07-16"), "description": "Follow this account for geo-targeted Healthcare job tweets in Pennsylvania Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 498, "friends_count": 305, "statues_count": 1956 }, "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-04T10:00:31.000Z"), "id": 684071924926955520, "text": "@XpRapist link your umg", "in_reply_to_status": -1, "in_reply_to_user": 3033830187, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user_mentions": {{ 3033830187 }}, "user": { "id": 3314611386, "name": "Recuff", "screen_name": "Recuff1", "lang": "en", "location": "Piqua, OH", "create_at": date("2015-08-13"), "description": "|BO3 God| @Detricakes is my E-GIRL", "followers_count": 227, "friends_count": 152, "statues_count": 2302 }, "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-04T10:00:31.000Z"), "id": 684071924994015232, "text": "Oh FFS https://t.co/rc68SLX9pI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 52209611, "name": "Tiger Poacher.", "screen_name": "CPERRY_5", "lang": "en", "location": "North Georgia", "create_at": date("2009-06-29"), "description": "Run the damn ball, Lane. Basketball staff writer @RollBamaRoll", "followers_count": 333, "friends_count": 275, "statues_count": 22275 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ringgold, GA", "id": "2e5b7aeec8569317", "name": "Ringgold", "place_type": "city", "bounding_box": rectangle("-85.195737,34.885052 -85.087955,34.940708") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13047, "countyName": "Catoosa", "cityID": 1365324, "cityName": "Ringgold" } }
+{ "create_at": datetime("2016-01-04T10:00:31.000Z"), "id": 684071925035839488, "text": "rip https://t.co/ZsQatzVfBc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1482817771, "name": "™", "screen_name": "thomas07m", "lang": "en", "location": "null", "create_at": date("2013-06-04"), "description": "Never give up - John Cena\n⚾", "followers_count": 229, "friends_count": 68, "statues_count": 10292 }, "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-04T10:00:31.000Z"), "id": 684071925149110272, "text": "Hope y'all had an awesome Christmas and New Year's!!! Who's ready for some #WINTERJAM!?! Listen… https://t.co/TVYMejLPCZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.5063629,27.9363594"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WINTERJAM" }}, "user": { "id": 33599071, "name": "Spirit FM 90.5", "screen_name": "ItsMySpiritFM", "lang": "en", "location": "Tampa, FL, USA", "create_at": date("2009-04-20"), "description": "Tampa Bay's Hit Christian Music! (WBVM) We love Jesus, dessert, puppies, popping bubble wrap and playing your absolute favorite songs!", "followers_count": 3709, "friends_count": 2998, "statues_count": 6955 }, "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-04T10:00:31.000Z"), "id": 684071925212053504, "text": "So there is only 2 girls in my English class & I'm one of them #LifeAtCMA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LifeAtCMA" }}, "user": { "id": 3178828434, "name": "M a g a l i ✨", "screen_name": "_princessgali", "lang": "en", "location": "Vallejo, CA", "create_at": date("2015-04-28"), "description": "19 ♏️ CalMaritime 2018 ⚓️", "followers_count": 156, "friends_count": 176, "statues_count": 1071 }, "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-04T10:00:31.000Z"), "id": 684071925220425728, "text": "Want to work in #WisconsinRapids, WI? View our latest opening: https://t.co/xfU0vETRWR #Physician #Veterans #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.8173465,44.3835763"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WisconsinRapids", "Physician", "Veterans", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 73026398, "name": "WI Physician Jobs", "screen_name": "WI_physician", "lang": "en", "location": "Wisconsin", "create_at": date("2009-09-09"), "description": "Follow this account for geo-targeted Physician job tweets in Wisconsin Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 147, "friends_count": 137, "statues_count": 110 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wisconsin Rapids, WI", "id": "25f4e3714f48e177", "name": "Wisconsin Rapids", "place_type": "city", "bounding_box": rectangle("-89.872931,44.3367 -89.744889,44.423607") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55141, "countyName": "Wood", "cityID": 5588200, "cityName": "Wisconsin Rapids" } }
+{ "create_at": datetime("2016-01-04T10:00:31.000Z"), "id": 684071925535125504, "text": "Jerzey Love - That (Radio Edit) https://t.co/VJhgca9K7O #JerzeyLove #Jers #Paterson #Love #Repost #Hulkshare #Free #Win #Retweet #RT #Enjoy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "JerzeyLove", "Jers", "Paterson", "Love", "Repost", "Hulkshare", "Free", "Win", "Retweet", "RT", "Enjoy" }}, "user": { "id": 622274344, "name": "Forever Lhove", "screen_name": "ForeverLhove", "lang": "en", "location": "United States of America", "create_at": date("2012-06-29"), "description": "JERZEY LOVE @thatloveis4ever Promotions...", "followers_count": 284, "friends_count": 1323, "statues_count": 1953 }, "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-04T10:00:31.000Z"), "id": 684071925715439616, "text": "@nedinaxo just sleeping like a baby ��", "in_reply_to_status": 684071006747652100, "in_reply_to_user": 355006170, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 355006170 }}, "user": { "id": 252417889, "name": "Allison Reardon", "screen_name": "AllieReardon", "lang": "en", "location": "860", "create_at": date("2011-02-14"), "description": "null", "followers_count": 721, "friends_count": 491, "statues_count": 18952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Torrington, CT", "id": "01d25a2412697203", "name": "Torrington", "place_type": "city", "bounding_box": rectangle("-73.168971,41.76345 -73.058981,41.885906") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9005, "countyName": "Litchfield", "cityID": 976500, "cityName": "Torrington" } }
+{ "create_at": datetime("2016-01-04T10:00:31.000Z"), "id": 684071925732216832, "text": "https://t.co/LAXQJeFbEo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1615481928, "name": "Branan Sipps", "screen_name": "SippsBranan", "lang": "en", "location": "Cary, Ga", "create_at": date("2013-07-23"), "description": "yeah", "followers_count": 114, "friends_count": 96, "statues_count": 1699 }, "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-01-04T10:00:31.000Z"), "id": 684071925832876033, "text": "Dylan O'Brien in new girl ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 306379692, "name": "❁kierstyn", "screen_name": "kierstyn96", "lang": "en", "location": "Smyrna, DE", "create_at": date("2011-05-27"), "description": "❁ 19 | @jesse27garza ♡", "followers_count": 1071, "friends_count": 674, "statues_count": 14955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Smyrna, DE", "id": "321da8a00987ba9f", "name": "Smyrna", "place_type": "city", "bounding_box": rectangle("-75.631177,39.255897 -75.587461,39.311468") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10001, "countyName": "Kent", "cityID": 1067310, "cityName": "Smyrna" } }
+{ "create_at": datetime("2016-01-04T10:00:31.000Z"), "id": 684071926122319872, "text": "@MistaKJ Hello KJ! I'm watching the Black Dahlia epi of #GhostHunters & u got some AMAZING responses! Anything else happen not shown?����", "in_reply_to_status": -1, "in_reply_to_user": 124848346, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GhostHunters" }}, "user_mentions": {{ 124848346 }}, "user": { "id": 2327894479, "name": "Jamie Allen", "screen_name": "JamieAllen121", "lang": "en", "location": "Illinois, USA", "create_at": date("2014-02-04"), "description": "Proud Parent, GAC Follower, Nick Groff Tourist(Ashmore Estates Event), Chad Lindberg fam member #ChadFam 1st Zak Bagans Tweet 09.12.15 https://t.co/QmEgruziwf", "followers_count": 479, "friends_count": 616, "statues_count": 10188 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tuscola, IL", "id": "f990e39ffab8ae79", "name": "Tuscola", "place_type": "city", "bounding_box": rectangle("-88.29429,39.780555 -88.25781,39.820545") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17041, "countyName": "Douglas", "cityID": 1776407, "cityName": "Tuscola" } }
+{ "create_at": datetime("2016-01-04T10:00:31.000Z"), "id": 684071926529044480, "text": "Travis Scott // A-Team ��\n\nLet it grow 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": 634951731, "name": "Rhee ®", "screen_name": "aurelia_parkss", "lang": "en", "location": "cincinnati", "create_at": date("2012-07-13"), "description": "if it aint about basketball dont tweet me |PHS'16| #teamhede 2•25•15❤️", "followers_count": 966, "friends_count": 887, "statues_count": 55889 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sharonville, OH", "id": "f4f98739e07e926f", "name": "Sharonville", "place_type": "city", "bounding_box": rectangle("-84.449769,39.254798 -84.352567,39.304105") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3971892, "cityName": "Sharonville" } }
+{ "create_at": datetime("2016-01-04T10:00:31.000Z"), "id": 684071926592069632, "text": "@suzettepetillo @kharyp @Tennessean @bimmerella If this militia were a shade of mocha or darker, they would have been dead already.", "in_reply_to_status": 684036247623344128, "in_reply_to_user": 2307296066, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2307296066, 53503202, 16639736, 41984111 }}, "user": { "id": 244675157, "name": "BernieBot", "screen_name": "evolved_deb", "lang": "en", "location": "Liberal in Mississippi", "create_at": date("2011-01-29"), "description": "Liberal going through reinvention, to survive in this economy.", "followers_count": 1457, "friends_count": 3406, "statues_count": 1703 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pascagoula, MS", "id": "b5d9160030d685ba", "name": "Pascagoula", "place_type": "city", "bounding_box": rectangle("-88.607587,30.3343 -88.500496,30.393797") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28059, "countyName": "Jackson", "cityID": 2855360, "cityName": "Pascagoula" } }
+{ "create_at": datetime("2016-01-04T10:00:31.000Z"), "id": 684071926596255746, "text": "Happy New Year Lions! We're working hard, before you return, to develop our new menu full of… https://t.co/voEjMArhWH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.41771092,33.96955528"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 171679972, "name": "LMUHospitality", "screen_name": "LMUHospitality", "lang": "en", "location": "Los Angeles, Ca", "create_at": date("2010-07-27"), "description": "LMU Hospitality by Sodexo has been a proud partner of Loyola Marymount University for over 30 years. Go Lions!", "followers_count": 330, "friends_count": 283, "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-01-04T10:00:31.000Z"), "id": 684071927061868544, "text": "I'm pissed this whole semester is going to suck", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1470755810, "name": "happy bday chris ,!", "screen_name": "chrisobling", "lang": "en", "location": "null", "create_at": date("2013-05-30"), "description": "alright dad", "followers_count": 107, "friends_count": 75, "statues_count": 19417 }, "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-01-04T10:00:31.000Z"), "id": 684071927099441152, "text": ".@adidasFballUS Unveils adizero Chromaflage 'Stars & Stripes' Cleats for the 2016 #ArmyBowl. #cleathead #teamadidas https://t.co/rfsAuWXukn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ArmyBowl", "cleathead", "teamadidas" }}, "user_mentions": {{ 2249043324 }}, "user": { "id": 16476964, "name": "Paul Murphy", "screen_name": "PabloMurphy", "lang": "en", "location": "A Global Entity", "create_at": date("2008-09-26"), "description": "Sports Fanatic, Sneakerologist, Lifestyle Space Dweller, Tech Junkie, Nomad, PR for @adidasUS. (Opinions are my own IP)", "followers_count": 1238, "friends_count": 1656, "statues_count": 8641 }, "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-04T10:00:31.000Z"), "id": 684071927263158273, "text": "@floridalottery #TriviaDayGiveaway 145 million", "in_reply_to_status": 684071808803450880, "in_reply_to_user": 36780362, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TriviaDayGiveaway" }}, "user_mentions": {{ 36780362 }}, "user": { "id": 94224252, "name": "paul radtke", "screen_name": "dasryt", "lang": "en", "location": "null", "create_at": date("2009-12-02"), "description": "null", "followers_count": 20, "friends_count": 213, "statues_count": 191 }, "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-04T10:00:31.000Z"), "id": 684071927678382080, "text": "Synthien vs Jotune - Choose Your Side! https://t.co/oxT1IczHwQ #NewCardGame #ChooseYourSide", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.6477771,40.7422154"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NewCardGame", "ChooseYourSide" }}, "user": { "id": 2675192989, "name": "BattleforSularia", "screen_name": "SulariaBCG", "lang": "en", "location": "Lincoln, Ne", "create_at": date("2014-07-23"), "description": "null", "followers_count": 148, "friends_count": 241, "statues_count": 405 }, "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-04T10:00:31.000Z"), "id": 684071927774887938, "text": "osomatsu VAs: todomatsu is *****, *****, *************\nME: YES HE ISD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "et", "is_retweet": false, "user": { "id": 2578825753, "name": "julien", "screen_name": "beelegs", "lang": "en", "location": "Belleview, FL", "create_at": date("2014-06-20"), "description": "*honk* i aint mean to", "followers_count": 153, "friends_count": 89, "statues_count": 8031 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Belleview, FL", "id": "b42e3aa9d6cd5b84", "name": "Belleview", "place_type": "city", "bounding_box": rectangle("-82.086703,29.032874 -82.0241,29.086579") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12083, "countyName": "Marion", "cityID": 1205375, "cityName": "Belleview" } }
+{ "create_at": datetime("2016-01-04T10:00:31.000Z"), "id": 684071927816654848, "text": "This #Transportation #job might be a great fit for you: Delivery Drivers - https://t.co/2xzwO0o452 #Driver #Hiring https://t.co/4CH1wrhujy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.769923,32.802955"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Transportation", "job", "Driver", "Hiring" }}, "user": { "id": 2557323745, "name": "Labatt Food Service", "screen_name": "LabattFoodJobs", "lang": "en", "location": "null", "create_at": date("2014-06-09"), "description": "Labatt Food Service is a privately owned food distribution company. Follow this page for information about our open #job opportunities.", "followers_count": 155, "friends_count": 317, "statues_count": 101 }, "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-04T10:00:31.000Z"), "id": 684071927946809349, "text": "Gucci Mane \nWasted", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1623105589, "name": "Just Like Candy", "screen_name": "AIRMAXRMLVOL2", "lang": "en", "location": "OHIGHO", "create_at": date("2013-07-26"), "description": "The Candy Shop", "followers_count": 484, "friends_count": 947, "statues_count": 44255 }, "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-04T10:00:31.000Z"), "id": 684071928085278722, "text": "กรี๊ดดดดดดในนิทรรศการมีโชว์แท่นที่ Han Solo ตอนโดนแช่ carbonite ใน The Empire Strikes Back ของจริงด้วย https://t.co/BtHQrng4td", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 39714634, "name": "แอชต้น คุชเชอร์", "screen_name": "tkjr007", "lang": "en", "location": "Chicago U.S.A", "create_at": date("2009-05-13"), "description": "The Americanization of Tony ซีซั่น 2", "followers_count": 835, "friends_count": 717, "statues_count": 32306 }, "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-04T10:00:31.000Z"), "id": 684071928512950272, "text": "I'm the new chef Boyardee https://t.co/DDhh8WyhQ5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 603169211, "name": "Taylee", "screen_name": "_tayleeelders", "lang": "en", "location": "|dtx|", "create_at": date("2012-06-08"), "description": "baby girl was living life for the feeling", "followers_count": 642, "friends_count": 445, "statues_count": 9528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gainesville, TX", "id": "342e8c0818874630", "name": "Gainesville", "place_type": "city", "bounding_box": rectangle("-97.174239,33.599665 -97.086339,33.669068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48097, "countyName": "Cooke", "cityID": 4827984, "cityName": "Gainesville" } }
+{ "create_at": datetime("2016-01-04T10:00:31.000Z"), "id": 684071928621998080, "text": "Popsicle https://t.co/yvKC8NwNna", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3220647008, "name": "alligator hunter", "screen_name": "senortatertot", "lang": "en", "location": "Tejas", "create_at": date("2015-05-19"), "description": "southern advocate", "followers_count": 411, "friends_count": 395, "statues_count": 3751 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rendon, TX", "id": "00386da25bfa7106", "name": "Rendon", "place_type": "city", "bounding_box": rectangle("-97.30462,32.551138 -97.190905,32.610434") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4861568, "cityName": "Rendon" } }
+{ "create_at": datetime("2016-01-04T10:00:31.000Z"), "id": 684071928705843200, "text": "Isn't it annoying when you make plans and they don't go through has you planned them a week ago?!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 163348049, "name": "Ana Laura", "screen_name": "alrxox", "lang": "en", "location": "Miami, FL", "create_at": date("2010-07-05"), "description": "null", "followers_count": 276, "friends_count": 252, "statues_count": 4835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glenvar Heights, FL", "id": "f546db03397f6e09", "name": "Glenvar Heights", "place_type": "city", "bounding_box": rectangle("-80.334426,25.690519 -80.292923,25.733912") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1226100, "cityName": "Glenvar Heights" } }
+{ "create_at": datetime("2016-01-04T10:00:31.000Z"), "id": 684071928806641664, "text": "I got a 3 hour layaway in Denver ☹️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 96095544, "name": "Dario Resendiz", "screen_name": "DarioResendiz", "lang": "en", "location": "i play call of duty", "create_at": date("2009-12-11"), "description": "IG/SC: DarioResendiz", "followers_count": 1733, "friends_count": 110, "statues_count": 34723 }, "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-04T10:00:31.000Z"), "id": 684071928986898432, "text": "I'm going to do some research on where the often used term, \"amen,\" originated.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 418051510, "name": "RodWayne", "screen_name": "RodWayneSaidIt", "lang": "en", "location": "Arlington Texas", "create_at": date("2011-11-21"), "description": "/iHeart Media/Talk-Radio/Radio-Personality/Motivator/Lifer/Occasional Asshole/...Dedicated to being me! Focused on bettering me! Now lets get it!", "followers_count": 527, "friends_count": 75, "statues_count": 15703 }, "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-04T10:00:32.000Z"), "id": 684071929163198466, "text": "Thanks for following us, @PinChasers @FlaStrawberries @YborCityFT @tonieysmith @LibraryTravel @Circuit_Show. https://t.co/2K9jKeKje1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 44019580, 3871704783, 2798068603, 4643133222, 4314710772, 3300498423 }}, "user": { "id": 37913210, "name": "Holiday Inn Tampa", "screen_name": "TampaHI", "lang": "en", "location": "Tampa, FL", "create_at": date("2009-05-05"), "description": "Tampa Holiday Inn. Come experience the change and excitement! #GetReinspired", "followers_count": 3429, "friends_count": 1643, "statues_count": 1747 }, "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-04T10:00:32.000Z"), "id": 684071929301499904, "text": "The problem with humans is that we're always trying to figure everything out. Looking for \"Why\" in everything removes the beauty of \"Is\". ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.7094333,40.810971"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 170009308, "name": "John Egger", "screen_name": "johnegger42", "lang": "en", "location": "Hickman, Nebraska", "create_at": date("2010-07-23"), "description": "Live life to the fullest. Act goofy if you are. Don't worry about things you can't control and do the best you can with everything else.", "followers_count": 3992, "friends_count": 4302, "statues_count": 40451 }, "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-04T10:00:32.000Z"), "id": 684071929687314432, "text": "I would love to show you my #listing at 11301 COYLE Street #Detroit #MI https://t.co/n1WEWVWKww #realestate https://t.co/cmWofS4GQ4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.193142,42.371279"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "Detroit", "MI", "realestate" }}, "user": { "id": 110314605, "name": "Reginald Perryman", "screen_name": "ReggieisRE", "lang": "en", "location": "Royal Oak, MI", "create_at": date("2010-01-31"), "description": "Real Estate Broker, Past President of Detroit Association of Realtors, HUD, Bank of America, VA, and Fannie Mae Listing Broker experience.", "followers_count": 141, "friends_count": 160, "statues_count": 585 }, "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-04T10:00:32.000Z"), "id": 684071929708285952, "text": "Bc you can sense my sarcasm seeping through my text messages", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 219601864, "name": "IG: pretty.jazzy", "screen_name": "livelikejazzz", "lang": "en", "location": "Oak Cliff.Atlanta.London.Cali", "create_at": date("2010-11-25"), "description": "I'm a southern belle I love the woman that is me IG @jazzminestyles Spelman College Alumna", "followers_count": 823, "friends_count": 638, "statues_count": 59401 }, "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-04T10:00:32.000Z"), "id": 684071929880428544, "text": "No even back at school yet and we're planning our happy hour trip 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": 212094140, "name": "Nicholas Pineda", "screen_name": "nicpineda12", "lang": "en", "location": "Conway, SC", "create_at": date("2010-11-04"), "description": "Gentleman of the Sigma Phi Epsilon fraternity: SC Epsilon, Resort Tourism Management and Marketing Major |The only limit is the one you set yourself| MD✈️SC", "followers_count": 387, "friends_count": 746, "statues_count": 4576 }, "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-01-04T10:00:32.000Z"), "id": 684071929884446720, "text": "������������ https://t.co/vbPpEP6X59", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1549027560, "name": "aprill", "screen_name": "__aprill__", "lang": "en", "location": "Upland, CA", "create_at": date("2013-06-26"), "description": "appreciate all the little blessings", "followers_count": 845, "friends_count": 578, "statues_count": 23091 }, "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-04T10:00:32.000Z"), "id": 684071930044006405, "text": "Sgo de volta #MendigosNTagsNoSDV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "hashtags": {{ "MendigosNTagsNoSDV" }}, "user": { "id": 3199079783, "name": "Bea Miller ♥", "screen_name": "_devonnememory", "lang": "pt", "location": "Albuquerque, NM", "create_at": date("2015-04-23"), "description": "Nome : Leandro :) \nI'M Lovatic & Beasts. @ddlovato my. Queen @BeaMiller My Doll", "followers_count": 21811, "friends_count": 18306, "statues_count": 74488 }, "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-04T10:00:32.000Z"), "id": 684071930110988288, "text": "Rare. https://t.co/0mFcThEo4v", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 309383758, "name": "Georgia", "screen_name": "GeorgiaSimoneL", "lang": "en", "location": "LA | MN | GA ", "create_at": date("2011-06-01"), "description": "Georgia from Georgia", "followers_count": 170, "friends_count": 298, "statues_count": 827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:32.000Z"), "id": 684071930748518400, "text": "Quentin - \"dad, I need to take a break from thinking right now because my brain is out of thinkers.\" @jsavege", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24380081 }}, "user": { "id": 15198534, "name": "Jordan Rapp ★", "screen_name": "rappstar", "lang": "en", "location": "Nomad", "create_at": date("2008-06-22"), "description": "Racing’s important to men who do it well. When you’re racing, it... it’s life. Anything that happens before or after... is just waiting.", "followers_count": 18009, "friends_count": 517, "statues_count": 12396 }, "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-04T10:00:32.000Z"), "id": 684071930790543360, "text": "I keep forgetting how easy it is for me to not eat", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 175422486, "name": "alina▪️", "screen_name": "alinadawnxo", "lang": "en", "location": "Catonsville, MD", "create_at": date("2010-08-06"), "description": "~may 23rd 2014~ •say whatcha need to say•", "followers_count": 893, "friends_count": 566, "statues_count": 27532 }, "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-01-04T10:00:32.000Z"), "id": 684071930937229312, "text": "@RyLoDJ @jamesegbert awesome - thanks for posting it! Here you go, @ChaiseLoper!", "in_reply_to_status": 683923388943867904, "in_reply_to_user": 1284767029, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1284767029, 138133234, 2497066200 }}, "user": { "id": 297367648, "name": "Kathy Dom", "screen_name": "live_music_fan", "lang": "en", "location": "Minneapolis, Minnesota", "create_at": date("2011-05-12"), "description": "Living my life again thanks to music & the artists that create it-what a fantastic ride!", "followers_count": 301, "friends_count": 474, "statues_count": 2535 }, "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-04T10:00:32.000Z"), "id": 684071931256025088, "text": "Want to work in #Mansfield, OH? View our latest opening: https://t.co/25IMLiCn6z #Sales #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.5154471,40.75839"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Mansfield", "Sales", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22058982, "name": "TMJ-OH-US Sales Jobs", "screen_name": "tmj_oh_sales", "lang": "en", "location": "Ohio", "create_at": date("2009-02-26"), "description": "Follow this account for geo-targeted Sales job tweets in Ohio Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 416, "friends_count": 301, "statues_count": 375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mansfield, OH", "id": "f136163002bd51f6", "name": "Mansfield", "place_type": "city", "bounding_box": rectangle("-82.572489,40.681009 -82.378653,40.815657") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39139, "countyName": "Richland", "cityID": 3947138, "cityName": "Mansfield" } }
+{ "create_at": datetime("2016-01-04T10:00:32.000Z"), "id": 684071931449049088, "text": "Working it! (@ Infinite Dimensions in New York, NY) https://t.co/iXJHboxR0F https://t.co/K7w6SPT8wf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9764975,40.75515421"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14819061, "name": "Jeff Wallace", "screen_name": "rocketman528", "lang": "en", "location": "Washington, D.C.", "create_at": date("2008-05-18"), "description": "CTO, athlete, author, chef: Saving world 1 project @ a time:-) Causes Célèbres - ESA, NASA, Smithsonian and White House. Extra in @KatyPerry's Hot n Cold!", "followers_count": 8556, "friends_count": 9375, "statues_count": 89112 }, "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-04T10:00:32.000Z"), "id": 684071932090650624, "text": "See our latest #Circleville, OH #job and click to apply: Quality Section Manager - https://t.co/rPmSgynxWQ #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.9460133,39.600618"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Circleville", "job", "Hiring" }}, "user": { "id": 28447324, "name": "Cameron Craig Group", "screen_name": "cameroncraig", "lang": "en", "location": "USA", "create_at": date("2009-04-02"), "description": "Thousands of jobs, one click away. Find a job here today.", "followers_count": 25912, "friends_count": 25801, "statues_count": 28679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Circleville, OH", "id": "bc04497c33fee903", "name": "Circleville", "place_type": "city", "bounding_box": rectangle("-82.961991,39.574744 -82.903716,39.633876") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39129, "countyName": "Pickaway", "cityID": 3915070, "cityName": "Circleville" } }
+{ "create_at": datetime("2016-01-04T10:00:32.000Z"), "id": 684071932145332224, "text": "I be on GO fuck planning 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": 837187825, "name": "21$ummersforever", "screen_name": "CIDTRIPS", "lang": "en", "location": "Follow your heart & feel free.", "create_at": date("2012-09-20"), "description": "Old enough to know better\nyoung enough to not give a fuck .", "followers_count": 367, "friends_count": 398, "statues_count": 10504 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edgewood, FL", "id": "b8ede6f302482b6c", "name": "Edgewood", "place_type": "city", "bounding_box": rectangle("-81.393182,28.475202 -81.36171,28.500213") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1219900, "cityName": "Edgewood" } }
+{ "create_at": datetime("2016-01-04T10:00:32.000Z"), "id": 684071932480876544, "text": "@_ree215 ����", "in_reply_to_status": 681857878123245569, "in_reply_to_user": 447408760, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 447408760 }}, "user": { "id": 447408760, "name": "Stretch➰", "screen_name": "_ree215", "lang": "en", "location": "chasing money ", "create_at": date("2011-12-26"), "description": "null", "followers_count": 1404, "friends_count": 1433, "statues_count": 35586 }, "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-04T10:00:32.000Z"), "id": 684071932795330562, "text": "was it too much to ask to start the year off happy ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2795289005, "name": "ELI", "screen_name": "HauteAddict", "lang": "en", "location": "null", "create_at": date("2014-09-30"), "description": "null", "followers_count": 956, "friends_count": 688, "statues_count": 27118 }, "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-04T10:00:32.000Z"), "id": 684071932984045570, "text": "BI-LO: Meat Associate (#EASLEY, SC) https://t.co/EfxOb0A1KK #Retail #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.5992629,34.817461"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "EASLEY", "Retail", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 28708336, "name": "TMJ-GRN Retail Jobs", "screen_name": "tmj_grn_retail", "lang": "en", "location": "Greenville, SC", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Greenville, SC. Need help? Tweet us at @CareerArc!", "followers_count": 474, "friends_count": 311, "statues_count": 566 }, "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-01-04T10:00:33.000Z"), "id": 684071933374251008, "text": "We're #hiring! Click to apply: Specialist, Digital Creative Production - https://t.co/VZhmqbGqmv #Retail #holidayjobs #BocaRaton, FL #Job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.0830984,26.3586885"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Retail", "holidayjobs", "BocaRaton", "Job" }}, "user": { "id": 2798260664, "name": "Boston Proper Jobs", "screen_name": "BostonProperJob", "lang": "en", "location": "null", "create_at": date("2014-09-08"), "description": "Boston Proper provides fashions for women who are fearlessly feminine, enviably chic and who possess the poise and confidence to “wear it like no one else“.", "followers_count": 30, "friends_count": 0, "statues_count": 460 }, "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-04T10:00:33.000Z"), "id": 684071933407670272, "text": "HUMAN RESOURCES ASSISTANT - Celebrity Staff: (#KansasCity, MO) https://t.co/PmD5zFDrAR #HR #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.5785667,39.0997265"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "KansasCity", "HR", "Job", "Jobs", "Hiring" }}, "user": { "id": 2366002093, "name": "Celebrity Staff Jobs", "screen_name": "celebstaffjobs", "lang": "en", "location": "Omaha\\Lincoln\\KC\\Des Moines", "create_at": date("2014-02-28"), "description": "Follow @celebstaffjobs for administrative, management, and legal #jobs. \r\nFollow our company @celebritystaff.", "followers_count": 61, "friends_count": 1, "statues_count": 143 }, "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-04T10:00:33.000Z"), "id": 684071934112301056, "text": "I finally. I have a plan", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 448264444, "name": "Nurah Holloway", "screen_name": "nunubaby_94", "lang": "en", "location": "null", "create_at": date("2011-12-27"), "description": "csusm collegelife. just having fun @nuraholloway", "followers_count": 197, "friends_count": 211, "statues_count": 2445 }, "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-04T10:00:33.000Z"), "id": 684071934393368577, "text": "Movie date tomorrow w 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": 1592656904, "name": "dama ❥", "screen_name": "damabepoppin", "lang": "en", "location": "null", "create_at": date("2013-07-13"), "description": "♏️", "followers_count": 596, "friends_count": 436, "statues_count": 25576 }, "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-01-04T10:00:33.000Z"), "id": 684071934640828416, "text": "@AustinMahone I am dying right now in school", "in_reply_to_status": 684071402639605761, "in_reply_to_user": 196795202, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 196795202 }}, "user": { "id": 1025480996, "name": "SxB✨Sydney", "screen_name": "imSydnxy", "lang": "en", "location": " San Antonio✨ Burbank HS", "create_at": date("2012-12-20"), "description": "16//concerts//Taylor Caniff is my sunshine♥//", "followers_count": 238, "friends_count": 198, "statues_count": 12354 }, "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-04T10:00:33.000Z"), "id": 684071934707892225, "text": "Can you recommend anyone for this #Healthcare #job? https://t.co/Yw9qjJh9GV #pharmacy #Yreka, CA #Veterans #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.6344708,41.7354186"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "pharmacy", "Yreka", "Veterans", "Hiring" }}, "user": { "id": 2786536903, "name": "Raley's Jobs", "screen_name": "raleysjobs", "lang": "en", "location": "null", "create_at": date("2014-09-02"), "description": "Feed your passion at Raley’s; join our team! Job opportunities you can grow your career with. Get news, recipes and more @raleys.", "followers_count": 74, "friends_count": 1, "statues_count": 1541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yreka, CA", "id": "e057ac551ce5df02", "name": "Yreka", "place_type": "city", "bounding_box": rectangle("-122.670691,41.681649 -122.612592,41.762438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6093, "countyName": "Siskiyou", "cityID": 686944, "cityName": "Yreka" } }
+{ "create_at": datetime("2016-01-04T10:00:33.000Z"), "id": 684071934967988225, "text": "Want to work in #Valdosta, GA? View our latest opening: https://t.co/BWUVcngp7F #Nursing #RN #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.2863,30.8625"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Valdosta", "Nursing", "RN", "Job", "Jobs", "Hiring" }}, "user": { "id": 2842711225, "name": "SGMC Careers", "screen_name": "SGMCCareers", "lang": "en", "location": "Valdosta, Georgia", "create_at": date("2014-10-06"), "description": "SGMC offers employees a warm and welcoming environment in a rapidly growing, dynamic organization. Follow us to learn more about our openings!", "followers_count": 64, "friends_count": 33, "statues_count": 357 }, "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-01-04T10:00:33.000Z"), "id": 684071935030984704, "text": "@Elfrinc yea it keeps me entertained at least", "in_reply_to_status": 684069668974407680, "in_reply_to_user": 53947197, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 53947197 }}, "user": { "id": 3406207523, "name": "francescaaaaa", "screen_name": "lightsfrancesca", "lang": "en", "location": "Manhattan, NY", "create_at": date("2015-08-06"), "description": "nawwwwwwwwwwwwwwwwwwwwwww", "followers_count": 143, "friends_count": 160, "statues_count": 4643 }, "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-04T10:00:33.000Z"), "id": 684071935123193856, "text": "Try again", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 33979026, "name": "50ShadesofAye❤️✨", "screen_name": "_aye_LOVE", "lang": "en", "location": "null", "create_at": date("2009-04-21"), "description": "GOD 1st Pretty fly mommy of one Lipstick Junkie", "followers_count": 1624, "friends_count": 1294, "statues_count": 98827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Griffin, GA", "id": "a846e2ce51e1233f", "name": "Griffin", "place_type": "city", "bounding_box": rectangle("-84.317652,33.202291 -84.228626,33.295471") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13255, "countyName": "Spalding", "cityID": 1335324, "cityName": "Griffin" } }
+{ "create_at": datetime("2016-01-04T10:00:33.000Z"), "id": 684071935550984192, "text": "bc u never wake up https://t.co/BYsVSJZGBD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 581350841, "name": "mace◡̈", "screen_name": "macymichelle12", "lang": "en", "location": "null", "create_at": date("2012-05-15"), "description": "Jesus is better // @ethanpeck12", "followers_count": 823, "friends_count": 461, "statues_count": 31203 }, "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-01-04T10:00:33.000Z"), "id": 684071935731318784, "text": "See a virtual tour of my listing on 11700 HALLER Street #Livonia #MI https://t.co/JQ5cRcWuNI #realestate https://t.co/lirVWf0bO5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.329842,42.370009"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Livonia", "MI", "realestate" }}, "user": { "id": 110314605, "name": "Reginald Perryman", "screen_name": "ReggieisRE", "lang": "en", "location": "Royal Oak, MI", "create_at": date("2010-01-31"), "description": "Real Estate Broker, Past President of Detroit Association of Realtors, HUD, Bank of America, VA, and Fannie Mae Listing Broker experience.", "followers_count": 141, "friends_count": 160, "statues_count": 586 }, "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-01-04T10:00:33.000Z"), "id": 684071935773257728, "text": "@ikaymoney bc we could relate ��", "in_reply_to_status": 684065082217648129, "in_reply_to_user": 431110901, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 431110901 }}, "user": { "id": 271709632, "name": "kat", "screen_name": "TheOfficialKath", "lang": "en", "location": "tejas ", "create_at": date("2011-03-24"), "description": "originality is so overrated.", "followers_count": 836, "friends_count": 411, "statues_count": 31343 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Elm, TX", "id": "bd331d141f66eead", "name": "Little Elm", "place_type": "city", "bounding_box": rectangle("-96.983818,33.140885 -96.881861,33.199847") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4843012, "cityName": "Little Elm" } }
+{ "create_at": datetime("2016-01-04T10:00:33.000Z"), "id": 684071936020869122, "text": "I wish the streets didn't close down", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 534471593, "name": "sludge lord", "screen_name": "AllHailSkatan", "lang": "en", "location": "null", "create_at": date("2012-03-23"), "description": "see you at the bottom", "followers_count": 302, "friends_count": 98, "statues_count": 5009 }, "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.32554") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26161, "countyName": "Washtenaw", "cityID": 2689140, "cityName": "Ypsilanti" } }
+{ "create_at": datetime("2016-01-04T10:00:33.000Z"), "id": 684071936092160000, "text": "@Mesmeratize I dont hit people off anymore lol.", "in_reply_to_status": 684070669752090624, "in_reply_to_user": 1909003196, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1909003196 }}, "user": { "id": 2532900750, "name": "Mitch", "screen_name": "YaBoiMitch_", "lang": "en", "location": "my moms basement", "create_at": date("2014-05-29"), "description": "I make Call of Duty Videos / Streamer and content creator for @TheOnlyeVilClan", "followers_count": 577, "friends_count": 315, "statues_count": 46272 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Milwaukee, WI", "id": "b1f4904c914e185d", "name": "South Milwaukee", "place_type": "city", "bounding_box": rectangle("-87.882122,42.89315 -87.839324,42.930342") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5575125, "cityName": "South Milwaukee" } }
+{ "create_at": datetime("2016-01-04T10:00:33.000Z"), "id": 684071936180269056, "text": "@KRandaaaa @ me", "in_reply_to_status": 684070767500357633, "in_reply_to_user": 300542134, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 300542134 }}, "user": { "id": 2791079059, "name": "kp", "screen_name": "KimmieHotzelt", "lang": "en", "location": "Murrells Inlet, SC", "create_at": date("2014-09-04"), "description": "basic average girl, tiger enthusiast #STJ~~ ️instagram: kimmie_hotzelt #JVO", "followers_count": 527, "friends_count": 369, "statues_count": 3392 }, "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-04T10:00:33.000Z"), "id": 684071936607961088, "text": "@PlayHearthstone sunwalker 2 op", "in_reply_to_status": 684071825769279489, "in_reply_to_user": 1209608880, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1209608880 }}, "user": { "id": 1682153106, "name": "keen", "screen_name": "keenanissocool", "lang": "en", "location": "check out my boys soundcloud⤵️", "create_at": date("2013-08-18"), "description": "16. Blessed.", "followers_count": 627, "friends_count": 460, "statues_count": 11801 }, "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-04T10:00:33.000Z"), "id": 684071936624742400, "text": "Can you recommend anyone for this #Sales #job? https://t.co/ijr2vw6R1r #Philadelphia, PA #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.163789,39.952335"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "job", "Philadelphia", "Hiring" }}, "user": { "id": 4499586313, "name": "New Signature Jobs", "screen_name": "NewSigCareers", "lang": "en", "location": "null", "create_at": date("2015-12-08"), "description": "New Signature is hiring qualified professionals with a background in #information #technology, #computer #science, #web #development or #graphic #design.", "followers_count": 11, "friends_count": 83, "statues_count": 22 }, "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-04T10:00:33.000Z"), "id": 684071936637448193, "text": "So yesterday my mom asks me, \"Did @H_Ta_The tweet anything today?\" Um, yea mom. She did.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 425605576 }}, "user": { "id": 894021554, "name": "Kelli O'Brian", "screen_name": "kellm0", "lang": "en", "location": "null", "create_at": date("2012-10-20"), "description": "null", "followers_count": 58, "friends_count": 659, "statues_count": 1200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:33.000Z"), "id": 684071936662491139, "text": "���������� https://t.co/kR4sblJVQ9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 441157727, "name": "the8ThLetter", "screen_name": "_JossySays", "lang": "en", "location": "null", "create_at": date("2011-12-19"), "description": "#TAMUC #LongLiveClyde", "followers_count": 1650, "friends_count": 1263, "statues_count": 90097 }, "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-04T10:00:33.000Z"), "id": 684071936670982144, "text": "This dude really think he is like my boss �� like bro stopppppp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 198647699, "name": "Aunt Koko", "screen_name": "LittleKrittt", "lang": "en", "location": "null", "create_at": date("2010-10-04"), "description": "Drink Henny, Dance Naked & Laugh EhhyDay.❤️", "followers_count": 378, "friends_count": 637, "statues_count": 17890 }, "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-04T10:00:33.000Z"), "id": 684071936708628480, "text": "@Z1043 Please play #History by @onedirection", "in_reply_to_status": 684067836436717572, "in_reply_to_user": 87365529, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "History" }}, "user_mentions": {{ 87365529, 209708391 }}, "user": { "id": 1135599187, "name": "HARRY'S FOOL", "screen_name": "Foolish4Harry", "lang": "en", "location": "stumbling through the dark ", "create_at": date("2013-01-30"), "description": "This is not the end", "followers_count": 4837, "friends_count": 2483, "statues_count": 299823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:33.000Z"), "id": 684071936729726976, "text": "@WholeFoods I �� grocery stores! #Icouldspendhourswondering #WeirdIKnow", "in_reply_to_status": -1, "in_reply_to_user": 15131310, "favorite_count": 0, "coordinate": point("-82.735902,28.017051"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Icouldspendhourswondering", "WeirdIKnow" }}, "user_mentions": {{ 15131310 }}, "user": { "id": 816078337, "name": "Lisa DiDario", "screen_name": "Lisa_D_Johnson", "lang": "en", "location": "null", "create_at": date("2012-09-10"), "description": "null", "followers_count": 402, "friends_count": 228, "statues_count": 3957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whole Foods Market", "id": "07d9dab477885000", "name": "Whole Foods Market", "place_type": "poi", "bounding_box": rectangle("-82.73590209999999,28.017050899999997 -82.735902,28.017051") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1212875, "cityName": "Clearwater" } }
+{ "create_at": datetime("2016-01-04T10:00:33.000Z"), "id": 684071936779915264, "text": "This #Clerical #job might be a great fit for you: HR Manager - https://t.co/GMhSqzSmBB #OklahomaCity, OK #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.5164276,35.4675602"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Clerical", "job", "OklahomaCity", "Hiring" }}, "user": { "id": 23006807, "name": "TMJ-OKC Cleric. Jobs", "screen_name": "tmj_okc_cler", "lang": "en", "location": "Oklahoma City, OK", "create_at": date("2009-03-05"), "description": "Follow this account for geo-targeted Office/Clerical job tweets in Oklahoma City, OK. Need help? Tweet us at @CareerArc!", "followers_count": 251, "friends_count": 199, "statues_count": 82 }, "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-04T10:00:33.000Z"), "id": 684071936784228353, "text": "@sierramendezzz ur pale ��", "in_reply_to_status": -1, "in_reply_to_user": 3906455838, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3906455838 }}, "user": { "id": 2227985699, "name": "kyle ©", "screen_name": "TheOneN0nly", "lang": "en", "location": "614", "create_at": date("2013-12-15"), "description": "RIP DBEATY Columbus, OH born and raised ~ Won't stop until I reach an 〽️ ~ OTG", "followers_count": 380, "friends_count": 598, "statues_count": 15776 }, "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-04T10:00:33.000Z"), "id": 684071936884740096, "text": "@Ramsey_Scott_ @Son_of_Darren @Boegel12 @Robinsonjaron_ guilty", "in_reply_to_status": 684064361237745664, "in_reply_to_user": 2717897740, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2717897740, 382091413, 332004118, 2197951092 }}, "user": { "id": 343754827, "name": "€aleb Hicks", "screen_name": "calebhicks30", "lang": "en", "location": "St. Louis- Jeffco- Murray ⚾️", "create_at": date("2011-07-27"), "description": "Don't worry bout' nothing pray about everything. -Luke Bryan", "followers_count": 344, "friends_count": 298, "statues_count": 1206 }, "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-01-04T10:00:33.000Z"), "id": 684071936910032897, "text": "it'll all make sense 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": 59942228, "name": "17thAve / M.T", "screen_name": "TerryPresume", "lang": "en", "location": "#17thAve ", "create_at": date("2009-07-24"), "description": "PunkRap | South Flawda | 17th Ave | ⭕❌ http://soundcloud.com/terry-presume #17thAve", "followers_count": 2184, "friends_count": 3089, "statues_count": 16574 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Golden Gate, FL", "id": "0bc912edeeadc71a", "name": "Golden Gate", "place_type": "city", "bounding_box": rectangle("-81.719513,26.168823 -81.686846,26.19952") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12021, "countyName": "Collier", "cityID": 1226300, "cityName": "Golden Gate" } }
+{ "create_at": datetime("2016-01-04T10:00:33.000Z"), "id": 684071937245593601, "text": "ion do no pillow talking that ain't 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": 494302042, "name": "shawniee", "screen_name": "_ShawnieShawn", "lang": "en", "location": "null", "create_at": date("2012-02-16"), "description": "null", "followers_count": 1406, "friends_count": 484, "statues_count": 28204 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lealman, FL", "id": "0267e86ec4e84f86", "name": "Lealman", "place_type": "city", "bounding_box": rectangle("-82.728631,27.808081 -82.657667,27.835836") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1239775, "cityName": "Lealman" } }
+{ "create_at": datetime("2016-01-04T10:00:34.000Z"), "id": 684071937505652736, "text": "@Randizzle13 Check out these videos from Eleutheros Books!\nhttps://t.co/EU8CwGpoYo &\nhttps://t.co/BkAtcIxITG https://t.co/mcl2rs9vbG", "in_reply_to_status": -1, "in_reply_to_user": 104660897, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 104660897 }}, "user": { "id": 197017206, "name": "John Rataczak", "screen_name": "johnrataczak", "lang": "en", "location": "Toledo, OH", "create_at": date("2010-09-30"), "description": "John Rataczak, Ph.D. Teacher, pastor, now writing Christian books. http://www.eleutherosbooks.com", "followers_count": 12710, "friends_count": 14897, "statues_count": 20279 }, "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-04T10:00:34.000Z"), "id": 684071937614548992, "text": "If you're a #Education professional in #Denver, CO, check out this #job: https://t.co/kwaCuecVmx #Hiring https://t.co/CGl4yhngLv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.9393531,39.6723811"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Education", "Denver", "job", "Hiring" }}, "user": { "id": 4036025892, "name": "MHCD Careers", "screen_name": "MHCD_Careers", "lang": "en", "location": "Denver, CO", "create_at": date("2015-10-27"), "description": "We believe that people can and do recover from mental illness and should have that chance. We hire people dedicated to working toward that goal. Join us!", "followers_count": 10, "friends_count": 2, "statues_count": 48 }, "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-04T10:00:34.000Z"), "id": 684071937706975232, "text": "All natural I guess https://t.co/w0Wn6iO2DI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3249902922, "name": "Li' Shawt ❣", "screen_name": "miraclelashayy", "lang": "en", "location": "@financer_ ", "create_at": date("2015-06-19"), "description": "SC: adoresmiracle || fineeassk ❤️ @prettyasanna long_liveexx", "followers_count": 667, "friends_count": 473, "statues_count": 26709 }, "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-04T10:00:34.000Z"), "id": 684071937778315264, "text": "@BackToMichael_ I don't like my family, don't and won't have friends and I guess the person I'm with and our kid.", "in_reply_to_status": 684071750091448320, "in_reply_to_user": 52322329, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 52322329 }}, "user": { "id": 61546390, "name": "️", "screen_name": "httpscottie", "lang": "en", "location": "Michigan", "create_at": date("2009-07-30"), "description": "...gfg", "followers_count": 7138, "friends_count": 392, "statues_count": 233606 }, "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-04T10:00:34.000Z"), "id": 684071938000433152, "text": "#AbundanceNOW isn't just book it’s MOVEMENT. We're convicted to bring new #Abundance definition to become status quo https://t.co/zrMbV7spgi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AbundanceNOW", "Abundance" }}, "user": { "id": 2864263071, "name": "Motivating Masses", "screen_name": "MotivatingMass", "lang": "en", "location": "San Diego, CA ", "create_at": date("2014-11-06"), "description": "Leaders serving leaders, inspiring success while achieving results for #speakers #writers #entrepreneurs #coaches", "followers_count": 1153, "friends_count": 326, "statues_count": 1308 }, "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-04T10:00:34.000Z"), "id": 684071938109607936, "text": "I see some of y'all decided to carry your horrifically uneducated comments into 2016.....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20288046, "name": "كمال كندريكس", "screen_name": "KamalKendricks", "lang": "en", "location": "Philadelphia || DC || الرباط", "create_at": date("2009-02-06"), "description": "20. Aquarius. Atlantean. Unleashing innovation through self-renovation. #Lightwerker20", "followers_count": 441, "friends_count": 618, "statues_count": 12704 }, "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-04T10:00:34.000Z"), "id": 684071938130448385, "text": "\"You're the best girlfriend in the entire Milky Way\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2861658680, "name": "Chase Levan", "screen_name": "chase_levan", "lang": "en", "location": "null", "create_at": date("2014-10-17"), "description": "null", "followers_count": 160, "friends_count": 189, "statues_count": 179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Castle Rock, CO", "id": "0fd0097acd635907", "name": "Castle Rock", "place_type": "city", "bounding_box": rectangle("-104.92104,39.322269 -104.773048,39.451319") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8035, "countyName": "Douglas", "cityID": 812415, "cityName": "Castle Rock" } }
+{ "create_at": datetime("2016-01-04T10:00:34.000Z"), "id": 684071938294067200, "text": "Want to work in #WinterPark, FL? View our latest opening: https://t.co/epQ6pW3uVr #Transportation #sales #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3392352,28.5999998"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WinterPark", "Transportation", "sales", "Job", "Jobs", "Hiring" }}, "user": { "id": 1425504374, "name": "CORT Careers", "screen_name": "CORTCareers", "lang": "en", "location": "Nationwide", "create_at": date("2013-05-13"), "description": "Join the CORT team where we help make a house a home, an office a great place to work, and an event a memorable celebration. @CORTFurniture", "followers_count": 666, "friends_count": 492, "statues_count": 1192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winter Park, FL", "id": "c87bc3adb59d807f", "name": "Winter Park", "place_type": "city", "bounding_box": rectangle("-81.375805,28.567893 -81.307654,28.632478") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1278300, "cityName": "Winter Park" } }
+{ "create_at": datetime("2016-01-04T10:00:34.000Z"), "id": 684071938516385793, "text": "Late Nights & Early Mornings - Nipsey Hussle", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 484121648, "name": "joey", "screen_name": "jayohhvee", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-02-05"), "description": "USA #JUSTBECOOL™", "followers_count": 515, "friends_count": 519, "statues_count": 16456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:34.000Z"), "id": 684071938533097472, "text": "I'm actually so sick I can barely see straight rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 212798842, "name": "tinkerbell", "screen_name": "_tatie00", "lang": "en", "location": "suckramento, CA", "create_at": date("2010-11-06"), "description": "dutch☕️", "followers_count": 720, "friends_count": 522, "statues_count": 33355 }, "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-04T10:00:34.000Z"), "id": 684071938558312448, "text": "Went to sleep mad! & woke up with a headache�� & this bs still continues ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2515942598, "name": "steph", "screen_name": "__steephx", "lang": "en", "location": "Tepatitlán de Morelos, Jalisco", "create_at": date("2014-05-22"), "description": "Senior / soccer #21 & #17⚽️ / Pretty Ricki❤️❤️", "followers_count": 183, "friends_count": 304, "statues_count": 687 }, "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-04T10:00:34.000Z"), "id": 684071938688430081, "text": "Found this little #beauty in #louisville on #frankfortavenue #flower #yellow @ Louisville,… https://t.co/Zjhiag8M5R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.7495,38.2289"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "beauty", "louisville", "frankfortavenue", "flower", "yellow" }}, "user": { "id": 206978747, "name": "Josh Tyson", "screen_name": "drayton797", "lang": "en", "location": "louisville, KY", "create_at": date("2010-10-23"), "description": "Photographer of JDTphotography, coffee snob, Christ follower and somewhat of a nice guy, I guess.", "followers_count": 225, "friends_count": 332, "statues_count": 6443 }, "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-01-04T10:00:34.000Z"), "id": 684071938856218625, "text": "@musiclover854 I got 41", "in_reply_to_status": 684070404772605953, "in_reply_to_user": 547210984, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 547210984 }}, "user": { "id": 2300509987, "name": "ca$h money", "screen_name": "keelyeld", "lang": "en", "location": "Grants Pass", "create_at": date("2014-01-19"), "description": "Birkenstocks aren't cool", "followers_count": 229, "friends_count": 276, "statues_count": 7904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grants Pass, OR", "id": "00a8b2b85821e087", "name": "Grants Pass", "place_type": "city", "bounding_box": rectangle("-123.360581,42.386741 -123.283133,42.471891") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41033, "countyName": "Josephine", "cityID": 4130550, "cityName": "Grants Pass" } }
+{ "create_at": datetime("2016-01-04T10:00:34.000Z"), "id": 684071938998730752, "text": "Can you recommend anyone for this #job? Jewelry Merchandiser - https://t.co/3pbaWu6Y2V #Missoula, MT #Retail #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.9939982,46.872146"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Missoula", "Retail", "Hiring", "CareerArc" }}, "user": { "id": 59967634, "name": "TMJ-MT Retail Jobs", "screen_name": "tmj_mt_retail", "lang": "en", "location": "Montana", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in Montana Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 353, "friends_count": 300, "statues_count": 197 }, "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-01-04T10:00:34.000Z"), "id": 684071939061596160, "text": "We're #hiring! Read about our latest #job opening here: RN II-Dialysis-Baptist Medical Center-FT36 7p - https://t.co/LEqlv9CoUW #Nursing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.492653,29.4336006"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Nursing" }}, "user": { "id": 259944587, "name": "BaptistHealthCareers", "screen_name": "BHSCareers", "lang": "en", "location": "San Antonio, TX", "create_at": date("2011-03-02"), "description": "Since 1903, Baptist Health System has been involved in the community, caring for the health and spiritual needs of people in San Antonio and beyond. Join us!", "followers_count": 258, "friends_count": 160, "statues_count": 2089 }, "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-04T10:00:34.000Z"), "id": 684071939200004097, "text": "This picture is so legendary.. https://t.co/otsoh2zrup", "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": 875, "friends_count": 781, "statues_count": 6510 }, "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-04T10:00:34.000Z"), "id": 684071939481055232, "text": "@XXXTyRoderick @enio15vv #nice dick!", "in_reply_to_status": 683544468780363780, "in_reply_to_user": 316809295, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "nice" }}, "user_mentions": {{ 316809295, 1465733964 }}, "user": { "id": 46353503, "name": "jerry smith", "screen_name": "highland07", "lang": "en", "location": "null", "create_at": date("2009-06-11"), "description": "null", "followers_count": 775, "friends_count": 69, "statues_count": 4034 }, "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-04T10:00:34.000Z"), "id": 684071939745443841, "text": "#LRT ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "LRT" }}, "user": { "id": 357080471, "name": "Mike Denzel™", "screen_name": "___BeLikeMike", "lang": "en", "location": "Philly ", "create_at": date("2011-08-17"), "description": "UMD '15 | ']['emple U '17 | You get more of what you think of | Hacked at 6 million.", "followers_count": 654, "friends_count": 562, "statues_count": 23049 }, "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-04T10:00:34.000Z"), "id": 684071940152266753, "text": "@GeT_THUMPD_23 only cause I kno how it be bruh.", "in_reply_to_status": 684071688141574144, "in_reply_to_user": 419029392, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 419029392 }}, "user": { "id": 2172200117, "name": "Niesh Flemings", "screen_name": "nieshflemings", "lang": "en", "location": "null", "create_at": date("2013-11-07"), "description": "null", "followers_count": 437, "friends_count": 184, "statues_count": 20980 }, "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-04T10:00:34.000Z"), "id": 684071940709945348, "text": "Niggas probably at home like I coulda made that \n\nMaybe you could , but you can't find that", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3291850148, "name": "Go'Brazy'Jay", "screen_name": "LilJaySoIcy", "lang": "en", "location": "null", "create_at": date("2015-07-24"), "description": "Forever Wealthy | Houston", "followers_count": 291, "friends_count": 96, "statues_count": 5716 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-04T10:00:34.000Z"), "id": 684071940714291203, "text": "Thanks. �� stuffed tomatoes for breakfast & chill tomorrow or nah? https://t.co/lO4pRSZEGK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 135271298, "name": "◹◭Nadler◮◸", "screen_name": "Biptar", "lang": "en", "location": "Tartarus", "create_at": date("2010-04-20"), "description": "Not of this World.", "followers_count": 284, "friends_count": 219, "statues_count": 19995 }, "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-01-04T10:00:34.000Z"), "id": 684071940785614848, "text": "There's another dimension on the tip of my finger.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 45281558, "name": "SpaceMan", "screen_name": "The_Jovian", "lang": "en", "location": "Europa, Jupiter", "create_at": date("2009-06-06"), "description": "Matt Lopes- Cosmic Debris", "followers_count": 518, "friends_count": 300, "statues_count": 7799 }, "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-04T10:00:34.000Z"), "id": 684071940923985920, "text": "Study hall, is going to suck", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2497363537, "name": "Keaton", "screen_name": "keatonparks58", "lang": "en", "location": "Delaware, OH", "create_at": date("2014-05-15"), "description": "All American Rejection", "followers_count": 118, "friends_count": 267, "statues_count": 220 }, "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-04T10:00:34.000Z"), "id": 684071940986802176, "text": "@FortuneMagazine The problem with people like this if u go against their policies they will find a way 2 pass their agenda! #WeThePeople", "in_reply_to_status": 684069474455130113, "in_reply_to_user": 25053299, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WeThePeople" }}, "user_mentions": {{ 25053299 }}, "user": { "id": 39921150, "name": "Tonya G. Dallas", "screen_name": "tonyagdallas", "lang": "en", "location": "California ", "create_at": date("2009-05-13"), "description": "Physician Assistant (near future) in Cardiovascular Surgery, American Heart Association Instructor, Health Educator, Marine, 2017 Secretary of Veterans Affairs", "followers_count": 704, "friends_count": 1403, "statues_count": 15335 }, "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-01-04T10:00:34.000Z"), "id": 684071941251072000, "text": "Can you recommend anyone for this #job? Restaurant Team Member (Crew) - https://t.co/iJb9WfejRY #SONIC #CaveCity, AR #Hospitality #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-91.5488375,35.946177"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "SONIC", "CaveCity", "Hospitality", "Hiring" }}, "user": { "id": 88003429, "name": "TMJ-AR HRTA Jobs", "screen_name": "tmj_ar_hrta", "lang": "en", "location": "Arkansas", "create_at": date("2009-11-06"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Arkansas Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 329, "friends_count": 281, "statues_count": 2056 }, "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": 5135, "countyName": "Sharp", "cityID": 512280, "cityName": "Cave City" } }
+{ "create_at": datetime("2016-01-04T10:00:35.000Z"), "id": 684071941716586496, "text": "Soft ass �� https://t.co/ICIUHQ9sO9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2443796036, "name": "☁️9⃣", "screen_name": "__GodsGift_", "lang": "en", "location": "Astro ⛽️ ", "create_at": date("2014-04-14"), "description": "He Died For Me So I Play For Him ✊ {R.I.P Pop R.I.P. Carolyn} •|1|2|3|4|5|6|7|8|• ⛅️ Glo'd {Every Rose Needs Rain} #BlinnJuniorCollege", "followers_count": 604, "friends_count": 354, "statues_count": 15957 }, "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-01-04T10:00:35.000Z"), "id": 684071941800640512, "text": "@whyNOt_saluteME this is why I do not like your ass now.", "in_reply_to_status": 684071846229225473, "in_reply_to_user": 312251833, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 312251833 }}, "user": { "id": 66003042, "name": "kyHENDRIX", "screen_name": "kyithediamond", "lang": "en", "location": "Raleigh, NC", "create_at": date("2009-08-15"), "description": "#FUTUREHIVE • A FYI ASS NX LEVEL MODEL ✨", "followers_count": 2339, "friends_count": 1959, "statues_count": 79311 }, "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-04T10:00:35.000Z"), "id": 684071942177964032, "text": "Everything related to #DJWHO music/events/releases will now be posted on my fan page. Make sure you follow here > https://t.co/AU9D8lFV3H", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DJWHO" }}, "user": { "id": 32711307, "name": "DJ WHO | Mr. WHO", "screen_name": "DJWhO1", "lang": "en", "location": "San Diego", "create_at": date("2009-04-17"), "description": "Real Is Classic and Classic Never Goes Out Of Style© | DJ/Producer | 1/2 @WhoPaulodaRosa | New single on iTunes - I'm Falling Up http://hyperurl.co/c5dohv", "followers_count": 2031, "friends_count": 617, "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-01-04T10:00:35.000Z"), "id": 684071942324891648, "text": "Horrory\n\n�� rak • Noc żywych trupów https://t.co/U5Cri9yOPa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pl", "is_retweet": false, "user": { "id": 716586110, "name": "Znaki Zodiaku Jako", "screen_name": "ZnakiZodiaku", "lang": "pl", "location": "null", "create_at": date("2012-07-25"), "description": "✨ Jeśli masz hejtować, bo jakiś Tweet ci się nie podoba to po prostu nie czytaj ✨ Tweety są znalezione, tłumaczone lub wymyślane", "followers_count": 3185, "friends_count": 2955, "statues_count": 3050 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:35.000Z"), "id": 684071942345879552, "text": "@illbeatyourkids who can't be ugly?", "in_reply_to_status": 684071626724499456, "in_reply_to_user": 232655219, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 232655219 }}, "user": { "id": 153998008, "name": "Lioness", "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": 1557, "friends_count": 1342, "statues_count": 176470 }, "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-04T10:00:35.000Z"), "id": 684071942387793920, "text": "I WANT A PUPPY :(((((", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2470535146, "name": "ashbash", "screen_name": "ashleylandin23", "lang": "en", "location": "Miami, FL", "create_at": date("2014-04-08"), "description": "null", "followers_count": 328, "friends_count": 425, "statues_count": 3965 }, "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-04T10:00:35.000Z"), "id": 684071942765203457, "text": "@_OhThatsLexii wya", "in_reply_to_status": -1, "in_reply_to_user": 286416404, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 286416404 }}, "user": { "id": 430875079, "name": "6000", "screen_name": "C__LVW", "lang": "en", "location": "null", "create_at": date("2011-12-07"), "description": "Bickin Back Bein Bool ! DTX", "followers_count": 1031, "friends_count": 742, "statues_count": 26064 }, "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-01-04T10:00:35.000Z"), "id": 684071942995881984, "text": "Anyone trying to lax? ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 227896844, "name": "Alexander Supertramp", "screen_name": "maxdixon12", "lang": "en", "location": "Peoria Illinois", "create_at": date("2010-12-17"), "description": "WQ #Explore", "followers_count": 451, "friends_count": 361, "statues_count": 14649 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellevue, IL", "id": "d044c44277ad698c", "name": "Bellevue", "place_type": "city", "bounding_box": rectangle("-89.718179,40.671419 -89.641798,40.711002") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17143, "countyName": "Peoria", "cityID": 1704871, "cityName": "Bellevue" } }
+{ "create_at": datetime("2016-01-04T10:00:35.000Z"), "id": 684071943004381184, "text": "@saturnprince_ dammmmmmmt will", "in_reply_to_status": 684071828227239936, "in_reply_to_user": 428000639, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user_mentions": {{ 428000639 }}, "user": { "id": 1000591519, "name": "cait!!!!!!!!!!!", "screen_name": "___blackhole", "lang": "en", "location": "WVU", "create_at": date("2012-12-09"), "description": "my own larger than life sized best friend", "followers_count": 463, "friends_count": 241, "statues_count": 106818 }, "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-04T10:00:35.000Z"), "id": 684071943054712832, "text": "@LittleGeminii luuuuucky ass", "in_reply_to_status": 684071543404539904, "in_reply_to_user": 3681146121, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3681146121 }}, "user": { "id": 628607600, "name": "sam", "screen_name": "fforestgump", "lang": "en", "location": "Chesapeake, VA", "create_at": date("2012-07-06"), "description": "oh really?", "followers_count": 785, "friends_count": 601, "statues_count": 23047 }, "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-04T10:00:35.000Z"), "id": 684071943151173633, "text": "@PargaOrlando Gracias por la trillita del 2015 y q continue dando batazos con la misma integridad en el 2016 Saludos Desde Daytona Beach FL", "in_reply_to_status": -1, "in_reply_to_user": 529557059, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 529557059 }}, "user": { "id": 339955328, "name": "Hector L Fernandez", "screen_name": "928_porky", "lang": "en", "location": "Daytona Beach, FL", "create_at": date("2011-07-21"), "description": "Hardcore NY Yankees & San Antonio Spurs Fan Love Boxing and NY Giants NFL Football & Politics", "followers_count": 1515, "friends_count": 2045, "statues_count": 44772 }, "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-01-04T10:00:35.000Z"), "id": 684071943188828160, "text": "Voicemail for them tings", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3256542661, "name": "Ratpakk", "screen_name": "RatpakkLA", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-06-26"), "description": "https://instagram.com/p/8MH8tvChH4/ ratpakkla@gmail.com", "followers_count": 60, "friends_count": 126, "statues_count": 45 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:35.000Z"), "id": 684071943251857408, "text": "⛅️❄️�� @ South Side Richmond https://t.co/UKzDOggC7v", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.49049005,37.49428258"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1040888395, "name": "Josué Rivas", "screen_name": "EJozue", "lang": "en", "location": "Santa Rosa De Lima, El Salvador", "create_at": date("2012-12-27"), "description": "● Apocalipsis 22:13 ● Nadie Encuentra Su Camino Sin Haberse Perdido varias VECES⌚️ ● El Salvador - Sta. Rosa De Lima.", "followers_count": 355, "friends_count": 1128, "statues_count": 683 }, "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-04T10:00:35.000Z"), "id": 684071943423823872, "text": "Dang they already have the Valentine's Day stuff out at Kroger.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20627121, "name": "Sara Kowatch", "screen_name": "diornotwar", "lang": "en", "location": "cincinnati", "create_at": date("2009-02-11"), "description": "graphic designer & creative thinker extraordinaire. young professional ¯\\_(ツ)_/¯. bacon aficionado. dog lover. auburn fan. WAR EAGLE. rip JDM [12.24.12]", "followers_count": 934, "friends_count": 2074, "statues_count": 25702 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sixteen Mile Stand, OH", "id": "0160fe6c59b9f2a9", "name": "Sixteen Mile Stand", "place_type": "city", "bounding_box": rectangle("-84.336852,39.263808 -84.31265,39.286729") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3972620, "cityName": "Sixteen Mile Stand" } }
+{ "create_at": datetime("2016-01-04T10:00:35.000Z"), "id": 684071943457251328, "text": "Stanford Health Care: Performance Improvement Consultant (34230) (#PaloAlto, CA) https://t.co/VuoGwLLZG0 #BusinessMgmt #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.1430195,37.4418834"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PaloAlto", "BusinessMgmt", "Job", "Jobs", "Hiring" }}, "user": { "id": 24214354, "name": "TMJ-SFO Mgmt. Jobs", "screen_name": "tmj_sfo_mgmt", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-03-13"), "description": "Follow this account for geo-targeted Business/Mgmt. job tweets in San Francisco, CA. Need help? Tweet us at @CareerArc!", "followers_count": 443, "friends_count": 284, "statues_count": 669 }, "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-01-04T10:00:35.000Z"), "id": 684071943515971585, "text": "@alexis8pizza I'm so jealous ��", "in_reply_to_status": 684066832148348928, "in_reply_to_user": 2567881632, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2567881632 }}, "user": { "id": 1315805905, "name": "Shelbs", "screen_name": "Shelby_Arbeiter", "lang": "en", "location": "CA ", "create_at": date("2013-03-29"), "description": "live a little", "followers_count": 1151, "friends_count": 739, "statues_count": 11244 }, "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-01-04T10:00:35.000Z"), "id": 684071943771795456, "text": "I'm at the orthodontist and I might fall asleep while they're working on my teeth.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2982878623, "name": "Kassidy", "screen_name": "kassidy_ferrell", "lang": "en", "location": "null", "create_at": date("2015-01-14"), "description": "hhs freshman | I'm cooler in real life", "followers_count": 277, "friends_count": 248, "statues_count": 2104 }, "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-04T10:00:35.000Z"), "id": 684071943809568768, "text": "https://t.co/L5FFu79tab", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1959047250, "name": "Iron Patriot Media", "screen_name": "Ipatriotmedia", "lang": "en", "location": "Quinlan, TX", "create_at": date("2013-10-13"), "description": "Publishing official conservative political news and commentary from Texas and around the country. We will be publishing a news website in the near future.", "followers_count": 769, "friends_count": 2286, "statues_count": 14023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Quinlan, TX", "id": "ca0086eee6e5a7aa", "name": "Quinlan", "place_type": "city", "bounding_box": rectangle("-96.154053,32.888325 -96.097296,32.920033") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48231, "countyName": "Hunt", "cityID": 4860140, "cityName": "Quinlan" } }
+{ "create_at": datetime("2016-01-04T10:00:35.000Z"), "id": 684071943943749633, "text": "Covelli Enterprises: Panera Bread Shift Leaders/ Hourly Associates New... (#Chillicothe, OH) https://t.co/ia71oH7ZrU #Hospitality #parttime", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.5930338,39.2039915"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Chillicothe", "Hospitality", "parttime" }}, "user": { "id": 2352066656, "name": "Covelli Enterprises", "screen_name": "CovelliJobs", "lang": "en", "location": "null", "create_at": date("2014-02-19"), "description": "At Covelli Enterprises, we pride ourselves on our unrivaled atmosphere, outstanding customer service, and talented people. Apply to our open #jobs here!", "followers_count": 61, "friends_count": 39, "statues_count": 281 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsboro, OH", "id": "b411bb59c0ada8f6", "name": "Hillsboro", "place_type": "city", "bounding_box": rectangle("-83.641444,39.176052 -83.589688,39.239334") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39071, "countyName": "Highland", "cityID": 3935560, "cityName": "Hillsboro" } }
+{ "create_at": datetime("2016-01-04T10:00:35.000Z"), "id": 684071943998435328, "text": "I'd like to take this opportunity to apologize to @iamdiddy for sleeping on #MMM that joint is ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MMM" }}, "user_mentions": {{ 18220175 }}, "user": { "id": 27523638, "name": "DJ RaQuest", "screen_name": "djraquest", "lang": "en", "location": "iPhone: 35.235878,-80.918423", "create_at": date("2009-03-29"), "description": "Top 40/Video/EDM/Hip Hop DJ | Power 98 DJ Competition Finalist | For Booking 9809392269 or djraquest@gmail.com | Sock Connoisseur | Funny Pic Poster |", "followers_count": 1439, "friends_count": 533, "statues_count": 81391 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cornelius, NC", "id": "6ac2e48ec4892d20", "name": "Cornelius", "place_type": "city", "bounding_box": rectangle("-80.946496,35.445762 -80.842478,35.507042") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3714700, "cityName": "Cornelius" } }
+{ "create_at": datetime("2016-01-04T10:00:35.000Z"), "id": 684071944317091840, "text": "Can you recommend anyone for this #Automotive #job? https://t.co/9jefzhcFBe #mechanic #NewUlm, MN #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.461114,44.312225"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Automotive", "job", "mechanic", "NewUlm", "Hiring" }}, "user": { "id": 2963298858, "name": "Royal Tire Jobs", "screen_name": "RoyalTireJobs", "lang": "en", "location": "St. Cloud, MN", "create_at": date("2015-01-07"), "description": "Since 1948, Royal Tire has been providing customers with quality transportation care, products, and services.", "followers_count": 70, "friends_count": 279, "statues_count": 31 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Ulm, MN", "id": "40aef2dae5f2e9cd", "name": "New Ulm", "place_type": "city", "bounding_box": rectangle("-94.507217,44.280315 -94.414288,44.344475") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27015, "countyName": "Brown", "cityID": 2746042, "cityName": "New Ulm" } }
+{ "create_at": datetime("2016-01-04T10:00:35.000Z"), "id": 684071944896032769, "text": "@x2twisted @TheSavageMafia https://t.co/xjUhP6kSGE illmac first round had me dying", "in_reply_to_status": 684071556788649985, "in_reply_to_user": 1648316113, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1648316113, 2639352359 }}, "user": { "id": 1859387370, "name": "Armbar Jesus", "screen_name": "OneTruePhoenix", "lang": "en", "location": "Chicago Heights, IL", "create_at": date("2013-09-12"), "description": "Submissions and elbows 2 the jaw that is all...ima member of PURE, NAH, BDG and KAPOW...but forever a member of The Robbie Scotts gym", "followers_count": 755, "friends_count": 1675, "statues_count": 6308 }, "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-04T10:00:35.000Z"), "id": 684071944975589377, "text": "I have a problem with spending money when I get it so I never bring my wallet in the store", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 289492733, "name": "Queen Victoria", "screen_name": "QueeennV_", "lang": "en", "location": "Louisiana", "create_at": date("2011-04-28"), "description": "#LSU ✨", "followers_count": 791, "friends_count": 527, "statues_count": 62502 }, "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-04T10:00:35.000Z"), "id": 684071945248178177, "text": "Registered Nurse - FT - Kindred Lewiston - Lewiston, ID. - Kindred Hospital: (#Lewiston, ID) https://t.co/aD5pshJ0CN #Nursing #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.0011889,46.4004089"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Lewiston", "Nursing", "Job", "Jobs" }}, "user": { "id": 22531792, "name": "TMJ-ID Nursing Jobs", "screen_name": "tmj_id_nursing", "lang": "en", "location": "Idaho", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Idaho Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 335, "friends_count": 294, "statues_count": 56 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lewiston, ID", "id": "96d6787fed95377c", "name": "Lewiston", "place_type": "city", "bounding_box": rectangle("-117.046429,46.361653 -116.912409,46.436879") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16069, "countyName": "Nez Perce", "cityID": 1646540, "cityName": "Lewiston" } }
+{ "create_at": datetime("2016-01-04T10:00:35.000Z"), "id": 684071945269161984, "text": "AIDS N : Beatings, torture and HIV: Inside the 'hellish' Kyrgyzstan jail where Brit welder Michael McFeat ...:… https://t.co/7w3VVvDCSc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.2732666,25.77508716"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 186478529, "name": "Stigmabase | ES", "screen_name": "PairsonnalitesE", "lang": "es", "location": "Las Américas | España ", "create_at": date("2010-09-03"), "description": "Keeping up-to-date on social exclusion worldwide | Español | contra la exclusión social | Marginación y exclusión | Violencia | Mayores | HIV LGBT | Not-for-PR", "followers_count": 2070, "friends_count": 1001, "statues_count": 565644 }, "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-04T10:00:35.000Z"), "id": 684071945642496000, "text": "@esparza21_b smdh DM right quick family ��", "in_reply_to_status": 684071614367961089, "in_reply_to_user": 2795582330, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2795582330 }}, "user": { "id": 3258044241, "name": "Big Boss Daddy.", "screen_name": "zayrobbins62", "lang": "en", "location": "null", "create_at": date("2015-05-15"), "description": "Dobson football #52", "followers_count": 364, "friends_count": 285, "statues_count": 3771 }, "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-04T10:00:35.000Z"), "id": 684071945789243392, "text": "I hate every time Rachel leaves but I'm so blessed to have these people in my life �� https://t.co/oSVPc6a7m5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3399677741, "name": "Mackenzie", "screen_name": "xokenziecole", "lang": "en", "location": "null", "create_at": date("2015-08-01"), "description": "19/ UINDY/ Special Education Major/ Dancer", "followers_count": 175, "friends_count": 294, "statues_count": 1831 }, "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-01-04T10:00:35.000Z"), "id": 684071945818796032, "text": "oh so you're a star wars fan all of the sudden? name 5 of their songs.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 56440306, "name": "Michael Wolfe", "screen_name": "ItsTheWolfe", "lang": "en", "location": "Charlotte, NC", "create_at": date("2009-07-13"), "description": "Slightly nerdy and terribly awkward. Pop Punk. Music is life. Electrical Engineer. Literal Genius. #EndTheStigma", "followers_count": 59, "friends_count": 77, "statues_count": 2744 }, "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-04T10:00:35.000Z"), "id": 684071945831223296, "text": "Sooooo @coachella, tomorrow?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 688583 }}, "user": { "id": 27551085, "name": "Daniel Dudley", "screen_name": "DDisBORED", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-03-29"), "description": "@RadioDisney Weekend Host. Former Pittsburgh SportsTalk Host. Pop Culture Expert. Music Lover. #WVU Grad. Scored 15 on LeBron James. WELLSBURG vs. EVERYBODY", "followers_count": 3481, "friends_count": 445, "statues_count": 70239 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:35.000Z"), "id": 684071945873285120, "text": "@BTBreezy271 @TDoucet24 that album has my 3 favorite songs.. #frenchtownkillas", "in_reply_to_status": 684041804128190464, "in_reply_to_user": 2426090547, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "frenchtownkillas" }}, "user_mentions": {{ 2426090547, 486158324 }}, "user": { "id": 633392541, "name": "gca", "screen_name": "gavynandre", "lang": "en", "location": "null", "create_at": date("2012-07-11"), "description": "The Money Family♻️ Brielle❤️ CPS Baseball Check out my highlight: http://www.hudl.com/athlete/5631882/highlights/289238375", "followers_count": 489, "friends_count": 294, "statues_count": 9218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Central, LA", "id": "01df2ed232dfaaf3", "name": "Central", "place_type": "city", "bounding_box": rectangle("-91.101943,30.486062 -90.967394,30.598818") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2213960, "cityName": "Central" } }
+{ "create_at": datetime("2016-01-04T10:00:36.000Z"), "id": 684071946229661696, "text": "softball practice 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": 2648758264, "name": "Jas", "screen_name": "jasminerhene", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2014-06-27"), "description": "he belongs with me ❤\nbhs softball #8 \n I dont have time for you .", "followers_count": 203, "friends_count": 394, "statues_count": 4945 }, "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-04T10:00:36.000Z"), "id": 684071946426957824, "text": "Canadá, seu lindo!!! @ Moraine Lake https://t.co/F2uIHJc1hQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-116.18555556,51.3225"), "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 2472223542, "name": "Blog PerdidosNoMundo", "screen_name": "perdidosatmundo", "lang": "en", "location": "null", "create_at": date("2014-05-01"), "description": "http://www.perdidosnomundo.com/ - Largamos o conforto da casa e a estabilidade do trabalho pra, de mochila nas costas, nos perdermos nesse mundão.", "followers_count": 89, "friends_count": 632, "statues_count": 461 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Improvement District No. 9 Banff, Alberta", "id": "2a71c43fd7a708ca", "name": "Improvement District No. 9 Banff", "place_type": "city", "bounding_box": rectangle("-117.320398,50.705371 -115.170174,52.270998") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-04T10:00:36.000Z"), "id": 684071946632445952, "text": "Seems to be good��: https://t.co/enCT5b0lCN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 363574980, "name": "Rie.K", "screen_name": "kitachanx", "lang": "en", "location": "Boston, MA ⇄ Brooklyn, NY", "create_at": date("2011-08-28"), "description": "Nobody's born a butterfly. Rie fu", "followers_count": 82, "friends_count": 91, "statues_count": 1288 }, "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-04T10:00:36.000Z"), "id": 684071946661851137, "text": "@UltraFlix @mollywood can you come up with phone magnetic by wed", "in_reply_to_status": 684071563864469504, "in_reply_to_user": 2253558433, "favorite_count": 0, "coordinate": point("-90.78054055,30.32473226"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2253558433, 6476742 }}, "user": { "id": 3219673158, "name": "Jason ayers", "screen_name": "jsayers3975", "lang": "en", "location": "null", "create_at": date("2015-05-18"), "description": "null", "followers_count": 73, "friends_count": 277, "statues_count": 9462 }, "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": 22063, "countyName": "Livingston", "cityID": 2227435, "cityName": "French Settlement" } }
+{ "create_at": datetime("2016-01-04T10:00:36.000Z"), "id": 684071946703642624, "text": "@HRingstaff CHS teacher and my bride, presenting to Mayfield Elementary on peer pressure teaching strategies for PD https://t.co/lDEPWOtpDw", "in_reply_to_status": -1, "in_reply_to_user": 404100518, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 404100518 }}, "user": { "id": 327918869, "name": "Martin Ringstaff", "screen_name": "MRingstaff", "lang": "en", "location": "Cleveland, TN", "create_at": date("2011-07-02"), "description": "Superintendent of Schools, Cleveland City Schools, Cleveland, TN\r\nParrothead, Va. Tech Hokies, and Red Sox fan.", "followers_count": 3676, "friends_count": 1735, "statues_count": 7487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland, TN", "id": "6ad5fc2baffd0f57", "name": "Cleveland", "place_type": "city", "bounding_box": rectangle("-84.945705,35.120064 -84.786995,35.266608") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47011, "countyName": "Bradley", "cityID": 4715400, "cityName": "Cleveland" } }
+{ "create_at": datetime("2016-01-04T10:00:36.000Z"), "id": 684071946724585473, "text": "im so out 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": 363951088, "name": "lex", "screen_name": "alexis_solario", "lang": "en", "location": ":))", "create_at": date("2011-08-28"), "description": "null", "followers_count": 758, "friends_count": 553, "statues_count": 17082 }, "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-04T10:00:36.000Z"), "id": 684071946753986560, "text": "Can I be in SoCal instead? https://t.co/tQ3Ux4rjSt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3197019139, "name": "cami✨☁️", "screen_name": "camila_craig", "lang": "en", "location": "California, USA", "create_at": date("2015-05-15"), "description": "alt • naïve // Cal 16'", "followers_count": 320, "friends_count": 273, "statues_count": 930 }, "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-04T10:00:36.000Z"), "id": 684071946766684160, "text": "Call me Fag!! @AppleJackz34", "in_reply_to_status": 684055158771331072, "in_reply_to_user": 343092133, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 343092133 }}, "user": { "id": 48069039, "name": "ErickaDenay", "screen_name": "ErickaDenay", "lang": "en", "location": "Norf philly!! ", "create_at": date("2009-06-17"), "description": "pretty whittle gal from the norf side of Philly... happy and blessed ❤️❣❣", "followers_count": 257, "friends_count": 92, "statues_count": 21688 }, "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-04T10:00:36.000Z"), "id": 684071946934456321, "text": "No more Mr. Nice guy (Ace hood Voice)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.8081284,35.4604798"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1891840926, "name": "Diontre'", "screen_name": "TreCrutcher", "lang": "en", "location": "null", "create_at": date("2013-09-21"), "description": "Rise and Grind.I'm Motivated everyday to work hard and fulfill my destiny.MCHS-#HeartOfATiger#10-Jr.#JS#GloryGoesUp:Snap:t_crutch https://t.co/AQNKRSBsMN", "followers_count": 573, "friends_count": 808, "statues_count": 1797 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lewisburg, TN", "id": "ce0d989d411b1369", "name": "Lewisburg", "place_type": "city", "bounding_box": rectangle("-86.828629,35.420871 -86.750358,35.478205") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47117, "countyName": "Marshall", "cityID": 4741860, "cityName": "Lewisburg" } }
+{ "create_at": datetime("2016-01-04T10:00:36.000Z"), "id": 684071946955427840, "text": "Attention������ #djampm #middaymix 12PM��92.3FM @power92chicago @radiomaha tunein!❤#bts #bucketboyz… https://t.co/rLdbCoXvT8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.50917718,41.59781463"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "djampm", "middaymix", "bts", "bucketboyz" }}, "user_mentions": {{ 31212891, 198985471 }}, "user": { "id": 97866323, "name": "dj ampm", "screen_name": "DJAMPMchicago", "lang": "en", "location": "Chicago", "create_at": date("2009-12-19"), "description": "The Hottest Chick In The Game. 92.3FM MixShow DJ @Power92Chicago Mon-Sat 12pm / Fri-Sat 12am #chicagobullspower92guestdj", "followers_count": 1987, "friends_count": 775, "statues_count": 10074 }, "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-04T10:00:36.000Z"), "id": 684071947009830912, "text": "Day 2 of waking up early and I'm already dead", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1566550658, "name": "nia", "screen_name": "niafann", "lang": "en", "location": "Westwood, Los Angeles", "create_at": date("2013-07-03"), "description": "vibin. #pray4nia2016", "followers_count": 563, "friends_count": 327, "statues_count": 5084 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-04T10:00:36.000Z"), "id": 684071947102072832, "text": "Anybody else cry when they read Hunter Henry's letter? No? Someone in my home was so distraught, she went back to the bed. #baddream #wps", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "baddream", "wps" }}, "user": { "id": 288570387, "name": "Julie Cloe", "screen_name": "hogfanswife", "lang": "en", "location": "WPS ", "create_at": date("2011-04-26"), "description": "null", "followers_count": 324, "friends_count": 857, "statues_count": 5694 }, "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-04T10:00:36.000Z"), "id": 684071947290853376, "text": "She say that I'ma dog, well if I'm a dog then Bitch you tick/fleas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30321322, "name": "Big Homie", "screen_name": "OG_MastersKush", "lang": "en", "location": "Where eva you might see me, Tx", "create_at": date("2009-04-10"), "description": "Son, Brother, Friend, Hustla", "followers_count": 1435, "friends_count": 993, "statues_count": 180144 }, "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-04T10:00:36.000Z"), "id": 684071947894964227, "text": "Best rap album ever in my opinion #WhitePeopleOpinionsDontMatter https://t.co/So2r0ZjZjT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WhitePeopleOpinionsDontMatter" }}, "user": { "id": 42025851, "name": "TYUS TRUMBETTI", "screen_name": "ttrum4", "lang": "en", "location": "Burlington, n e w jers", "create_at": date("2009-05-23"), "description": "Caitlin Carey 1.26.13❤️❤️", "followers_count": 912, "friends_count": 555, "statues_count": 34296 }, "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-01-04T10:00:36.000Z"), "id": 684071948066803712, "text": "Every day is a struggle", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 52907461, "name": "Lunari", "screen_name": "LunariH", "lang": "en", "location": "Wheat Ridge, CO", "create_at": date("2009-07-01"), "description": "I tweet about League, shitty customers, and busses. get on my level.", "followers_count": 579, "friends_count": 398, "statues_count": 34856 }, "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-01-04T10:00:36.000Z"), "id": 684071948163395585, "text": "@sotomorrow they're soooooo cute!", "in_reply_to_status": 684056372086976512, "in_reply_to_user": 18429928, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18429928 }}, "user": { "id": 33116898, "name": "Brooke Newberry", "screen_name": "be_newberry", "lang": "en", "location": "La Crosse, WI", "create_at": date("2009-04-18"), "description": "I am the Beyonce of the baby world, shaky egg queen, and coffee addict.", "followers_count": 525, "friends_count": 581, "statues_count": 4588 }, "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-01-04T10:00:36.000Z"), "id": 684071948305981441, "text": "3 i think https://t.co/q1efl0wydV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 239213725, "name": "madeline", "screen_name": "state_chumps", "lang": "en", "location": "VA/SC", "create_at": date("2011-01-16"), "description": "i tweet a lot of pictures of my eye", "followers_count": 2712, "friends_count": 981, "statues_count": 136604 }, "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-01-04T10:00:36.000Z"), "id": 684071948389756928, "text": "Llegamos a McAllen un frío y lluvia de la fregada y el día que nos vamos sale el sol ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 274810861, "name": "Ximena Machuca", "screen_name": "ximenamachucaa", "lang": "es", "location": "null", "create_at": date("2011-03-30"), "description": "null", "followers_count": 358, "friends_count": 212, "statues_count": 3750 }, "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-04T10:00:36.000Z"), "id": 684071948448448512, "text": "@OpenCarryTexas @cjgrisham @GRAAmerica through various neighborhoods in my youth. I've seen to many people pull out weapons and escalate", "in_reply_to_status": 684071655111434240, "in_reply_to_user": 1306314474, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1571621442, 3274879178, 1190825605 }}, "user": { "id": 1306314474, "name": "lexluthor415", "screen_name": "lexluthor415", "lang": "en", "location": "null", "create_at": date("2013-03-26"), "description": "null", "followers_count": 151, "friends_count": 615, "statues_count": 1407 }, "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-04T10:00:36.000Z"), "id": 684071948482027520, "text": "�� https://t.co/spZNczFF22", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 235702743, "name": "Patricia G✺nzalez", "screen_name": "OseaPatti", "lang": "en", "location": "null", "create_at": date("2011-01-08"), "description": "*Luna de miel, Rosa Pastel, Clichés y Tonterías*", "followers_count": 306, "friends_count": 471, "statues_count": 19135 }, "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-04T10:00:36.000Z"), "id": 684071948612075520, "text": "Today is the day! Come out and let's get this place rocking! #thenewscanwait @ J&J's PIZZA ON… https://t.co/rBVvVoOGjH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.1333084,33.2158318"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thenewscanwait" }}, "user": { "id": 4357325775, "name": "Drown This Kingdom", "screen_name": "DrownThisKingdo", "lang": "en", "location": "Dallas, TX", "create_at": date("2015-12-02"), "description": "Musicians from Dallas Tx just trying to make great music", "followers_count": 10, "friends_count": 40, "statues_count": 12 }, "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-04T10:00:36.000Z"), "id": 684071948649824256, "text": "Yeah and we don't hear that b4 she opened her mouth they had one of Slick Willey as a fornicator. https://t.co/DPsaOKGjLD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 521761797, "name": "Marie Shelton", "screen_name": "SheltonMarie", "lang": "en", "location": "DFW Tx", "create_at": date("2012-03-11"), "description": "Love God, family, country & outdoors. Grandkids mean world to us. Native Texan. Cherish Freedom & Troops protecting it. #Trump2016", "followers_count": 2198, "friends_count": 1684, "statues_count": 51747 }, "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-04T10:00:36.000Z"), "id": 684071948683472896, "text": "Very sick. ��\n\nLaying in bed reading all day. Tanking on whether to start Making 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": 605583497, "name": "T.J. Jurkiewicz", "screen_name": "RandalGravesNL", "lang": "en", "location": "Profitsville, NJ/MD", "create_at": date("2012-06-11"), "description": "Full time poker pro AKA Randal_Graves, 2012 Rutgers NB alumni, chicken wing connoisseur, and long suffering fan of #Rutgers #Jets #Mets #Magic #Devils", "followers_count": 891, "friends_count": 1395, "statues_count": 7551 }, "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-04T10:00:36.000Z"), "id": 684071948775641088, "text": "AIDS N : Beatings, torture and HIV: Inside the 'hellish' Kyrgyzstan jail where Brit welder Michael McFeat ... https://t.co/Ef2OusHw63", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.2732666,25.77508716"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 191092262, "name": "Stigmabase | NORDIC", "screen_name": "pairsonnalitesN", "lang": "en", "location": "Scot | Inuit | Skandinavien", "create_at": date("2010-09-15"), "description": "Keeping up-to-date on social exclusion worldwide | NORDIC | Scotland | Norsk | Svenska | Русский || Not-for-PROFIT", "followers_count": 1520, "friends_count": 1474, "statues_count": 358296 }, "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-04T10:00:36.000Z"), "id": 684071948901466112, "text": "See our latest #Rochester, MI #job and click to apply: Registered Nurse (RN) - Float... - https://t.co/rVMG8JzS06 https://t.co/5rhfYVceSt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.1492553,42.6807334"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Rochester", "job" }}, "user": { "id": 1076984880, "name": "Crittenton Jobs", "screen_name": "CrittentonJobs", "lang": "en", "location": "Rochester, Michigan", "create_at": date("2013-01-10"), "description": "Working for Crittenton Hospital Medical Center is more than just a job. It’s a career. Follow us for current opportunities.", "followers_count": 228, "friends_count": 249, "statues_count": 3317 }, "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-01-04T10:00:36.000Z"), "id": 684071949035802624, "text": "I'm at White Sands National Monument - @whitesands_nps in Alamogordo, NM https://t.co/JsBWeZUtIo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-106.1727023,32.7824565"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 292512828 }}, "user": { "id": 16917799, "name": "luiz", "screen_name": "luizotavio", "lang": "en", "location": "Texas", "create_at": date("2008-10-22"), "description": "null", "followers_count": 276, "friends_count": 140, "statues_count": 59511 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Mexico, USA", "id": "71d65c0e6d94efab", "name": "New Mexico", "place_type": "admin", "bounding_box": rectangle("-109.050173,31.332176 -103.002065,37.000294") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35035, "countyName": "Otero" } }
+{ "create_at": datetime("2016-01-04T10:00:36.000Z"), "id": 684071949044158464, "text": "niggas pissy & bitches too!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 605117281, "name": "alaya", "screen_name": "alayadivine", "lang": "en", "location": "null", "create_at": date("2012-06-10"), "description": "null", "followers_count": 2057, "friends_count": 857, "statues_count": 51631 }, "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-04T10:00:36.000Z"), "id": 684071949056659456, "text": "����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3114613922, "name": "sabine", "screen_name": "sabine_rosales", "lang": "en", "location": "ATX", "create_at": date("2015-03-28"), "description": "null", "followers_count": 246, "friends_count": 183, "statues_count": 6942 }, "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-04T10:00:36.000Z"), "id": 684071949081825280, "text": "Interested in a #job near #Wilmington, DE? This could be a great fit: https://t.co/7i822Fiy75 #cfgjobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.5917092,39.7472496"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Wilmington", "cfgjobs", "Hiring" }}, "user": { "id": 379141823, "name": "CitizensBankITJobs", "screen_name": "RBSCFG_Tech", "lang": "en", "location": "Rhode Island", "create_at": date("2011-09-24"), "description": "We are hiring! Go to http://CFGCareers.com for more information about working at Citizens Bank.", "followers_count": 112, "friends_count": 50, "statues_count": 273 }, "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-04T10:00:36.000Z"), "id": 684071949106982913, "text": "Rain all week? But where the fuck you at ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1258649953, "name": "⚽️KiingRegime⚽️", "screen_name": "Irving_Radilla", "lang": "en", "location": "Tustin, CA", "create_at": date("2013-03-10"), "description": "⚽  8 More Than 92 With Me @jessicaaveee", "followers_count": 585, "friends_count": 454, "statues_count": 18506 }, "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-04T10:00:36.000Z"), "id": 684071949165834241, "text": "bitches treat PINK like polo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 227853866, "name": "G Ro ‼️", "screen_name": "therealromyo", "lang": "en", "location": "Indianapolis, IN", "create_at": date("2010-12-17"), "description": "Cappin", "followers_count": 1081, "friends_count": 506, "statues_count": 32087 }, "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-01-04T10:00:36.000Z"), "id": 684071949207638016, "text": "Bright future squad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2978044416, "name": "Elkapone", "screen_name": "elkaroderick", "lang": "en", "location": "hoplaaaaaand", "create_at": date("2015-01-12"), "description": "Abzolutely", "followers_count": 259, "friends_count": 136, "statues_count": 685 }, "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-04T10:00:36.000Z"), "id": 684071949346144256, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 242964115, "name": "CJ Echols", "screen_name": "CJ_ECHOLS", "lang": "en", "location": "- MARIO 03/07/97 - 03/06/14", "create_at": date("2011-01-25"), "description": "Ain't stressin.", "followers_count": 605, "friends_count": 284, "statues_count": 8815 }, "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-04T10:00:36.000Z"), "id": 684071949618786308, "text": "If mad max wins kill me https://t.co/JFsn29pnHo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 45302879, "name": "George K.", "screen_name": "georgekrz", "lang": "en", "location": "Queens, NY ", "create_at": date("2009-06-07"), "description": "There's a lot of things in life worth living for, isn't there?", "followers_count": 111, "friends_count": 480, "statues_count": 3144 }, "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-04T10:00:36.000Z"), "id": 684071949975289856, "text": "@omfgkeema lol ⚰", "in_reply_to_status": 684071861441937408, "in_reply_to_user": 1170188048, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1170188048 }}, "user": { "id": 2657701926, "name": "TAELEFLARE", "screen_name": "TAELEFLARE", "lang": "en", "location": "Atlanta, GA", "create_at": date("2014-07-18"), "description": "Creative director I am the vision ++", "followers_count": 497, "friends_count": 296, "statues_count": 9476 }, "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-01-04T10:00:37.000Z"), "id": 684071950625423360, "text": "Jordan clarkson is underrated. Get that kid in the starting line up and get some peices ariund him and the #Lakers will start moving forward", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Lakers" }}, "user": { "id": 52559920, "name": "EvanAbshire", "screen_name": "EvanAbshire", "lang": "en", "location": "null", "create_at": date("2009-06-30"), "description": "null", "followers_count": 308, "friends_count": 475, "statues_count": 5672 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Menomonee Falls, WI", "id": "e37629210522d6a6", "name": "Menomonee Falls", "place_type": "city", "bounding_box": rectangle("-88.185675,43.104401 -88.063351,43.192126") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55133, "countyName": "Waukesha", "cityID": 5551000, "cityName": "Menomonee Falls" } }
+{ "create_at": datetime("2016-01-05T10:00:00.000Z"), "id": 684434182844502016, "text": "i'll take that's bullshit for $100 https://t.co/9NP7IPoTQt", "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": "rené", "screen_name": "501rene", "lang": "en", "location": "gtc", "create_at": date("2012-11-01"), "description": "and who are these clowns?", "followers_count": 520, "friends_count": 395, "statues_count": 40957 }, "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-05T10:00:00.000Z"), "id": 684434182857084928, "text": "@JebBush Biz ldrs say bigst brake on grwth's lack o'demand. Y, cause growin income gap! Lowr tax & fewr regs is BS. #FeelTheBern #morningjoe", "in_reply_to_status": -1, "in_reply_to_user": 113047940, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FeelTheBern", "morningjoe" }}, "user_mentions": {{ 113047940 }}, "user": { "id": 629689423, "name": "DaveG", "screen_name": "SongBird1154", "lang": "en", "location": "null", "create_at": date("2012-07-07"), "description": "Gimme Shelter ....... from the nonsense!", "followers_count": 9, "friends_count": 14, "statues_count": 529 }, "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-05T10:00:00.000Z"), "id": 684434183351959553, "text": "This #BusinessMgmt #job might be a great fit for you: Supervisor, Registration/Admitting, Financial... - https://t.co/OmzKUM6Rd3 #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.272747,37.8715926"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BusinessMgmt", "job", "Hiring" }}, "user": { "id": 24214354, "name": "TMJ-SFO Mgmt. Jobs", "screen_name": "tmj_sfo_mgmt", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-03-13"), "description": "Follow this account for geo-targeted Business/Mgmt. job tweets in San Francisco, CA. Need help? Tweet us at @CareerArc!", "followers_count": 439, "friends_count": 284, "statues_count": 677 }, "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-05T10:00:00.000Z"), "id": 684434183465205761, "text": "I wish I could make rent payments with my credit card like the last few places I lived at. Would be able to get explorer $25K/year ezpz.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 12155422, "name": "Kris Ziel", "screen_name": "kziel", "lang": "en", "location": "Mountain View, CA", "create_at": date("2008-01-12"), "description": "United • Aviation • 1K // Macau • Singapore • Vegas // Operations Management at Leeds School of Business // Views are mine, not my employer's", "followers_count": 603, "friends_count": 117, "statues_count": 25279 }, "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-01-05T10:00:00.000Z"), "id": 684434183607988224, "text": "@shannonrosexoxo @glaser97 @natiebear00 @courtneyxo199 @JaclynSerfaty then there's court...", "in_reply_to_status": 684434050870853632, "in_reply_to_user": 377033574, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 377033574, 1475202186, 376513078, 52101727, 375705703 }}, "user": { "id": 381817622, "name": "S", "screen_name": "SumSum_love", "lang": "en", "location": "null", "create_at": date("2011-09-28"), "description": "Just a smalltown teen with big city dreams", "followers_count": 361, "friends_count": 412, "statues_count": 14833 }, "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-01-05T10:00:00.000Z"), "id": 684434183733788676, "text": "@_funfun61 check out the contact page on our website: https://t.co/Ss8N5QkieC", "in_reply_to_status": 684431901747200002, "in_reply_to_user": 1634189096, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1634189096 }}, "user": { "id": 21889970, "name": "Masquerade Atlanta", "screen_name": "masquerade_atl", "lang": "en", "location": "695 North Ave NE, ATL GA 30308", "create_at": date("2009-02-25"), "description": "The Masquerade is a concert venue in Atlanta, GA that houses 4 stages - Purgatory, Hell, Heaven and The Masquerade Music Park", "followers_count": 18476, "friends_count": 545, "statues_count": 10757 }, "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-05T10:00:00.000Z"), "id": 684434183771406336, "text": "Remember when I went to all our sister restaurants and made these? Where should I go next? ������ #WhenGoodEggsCollide https://t.co/kZyQEJoSxT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WhenGoodEggsCollide" }}, "user": { "id": 262315486, "name": "Tucker'sOnionBurgers", "screen_name": "TucksOKC", "lang": "en", "location": "Uptown23rd/ClassenCurve/Quail", "create_at": date("2011-03-07"), "description": "Proudly serving delicious onion burgers made from premium local ingredients!", "followers_count": 8091, "friends_count": 4163, "statues_count": 4943 }, "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-05T10:00:00.000Z"), "id": 684434183901560832, "text": "@Chantalamarieee @Ally_Gerczak @laneygilliam camel toe Chantal", "in_reply_to_status": 684427787344691204, "in_reply_to_user": 263851333, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 263851333, 269619202, 258457090 }}, "user": { "id": 263851844, "name": "Maddy LeRoy", "screen_name": "Madysen_LeRoy", "lang": "en", "location": "Green Bay/UWLAX", "create_at": date("2011-03-10"), "description": "in my city im a young god.", "followers_count": 1000, "friends_count": 496, "statues_count": 6032 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellevue, WI", "id": "017b2d0871df7b7c", "name": "Bellevue", "place_type": "city", "bounding_box": rectangle("-88.017371,44.438884 -87.917359,44.489067") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55009, "countyName": "Brown", "cityID": 5506350, "cityName": "Bellevue" } }
+{ "create_at": datetime("2016-01-05T10:00:00.000Z"), "id": 684434183901605888, "text": "Ripley SW Limestone Co. Temp: 45.1°F Wind:6.9mph Pressure: 1011.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": 47103 }, "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-05T10:00:00.000Z"), "id": 684434184207646720, "text": "Note to a self: If they don't give you credits for being like them, separate yourself and highlight your differences. Shine.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3257116784, "name": "Ashura B. Lovelady", "screen_name": "ABLovelady", "lang": "en", "location": "in the mud.", "create_at": date("2015-06-26"), "description": "Blood Type A-Positive", "followers_count": 142, "friends_count": 553, "statues_count": 1030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:00.000Z"), "id": 684434184392163328, "text": "@ProfessMoravec Kewl. I'm hosting a meeting at Wheaton 4/6-8, but I'll be going to OAH 4/9-10. Who forgot to check her calendar?! Doh.", "in_reply_to_status": 684433371716567040, "in_reply_to_user": 369529173, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 369529173 }}, "user": { "id": 14873669, "name": "Kathryn Tomasek", "screen_name": "KathrynTomasek", "lang": "en", "location": "SE Massachusetts/RI", "create_at": date("2008-05-22"), "description": "Digital Humanities and Digital History at Wheaton College in Massachusetts and at http://encodinghfrs.org. Always learning.", "followers_count": 1551, "friends_count": 1502, "statues_count": 6657 }, "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-05T10:00:00.000Z"), "id": 684434184648146944, "text": "Where my #PracticalMagic fans at? #GaryHallet #NekoAstume https://t.co/W6jcyR5T5P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PracticalMagic", "GaryHallet", "NekoAstume" }}, "user": { "id": 242818071, "name": "Becca James", "screen_name": "WreckaFlames", "lang": "en", "location": "Chicago, IL ", "create_at": date("2011-01-25"), "description": "Assistant Editor @TheAVclub and Founder of @popstache #Hedgehog owner, #Menswear enthusiast, #Rush devotee.", "followers_count": 1098, "friends_count": 666, "statues_count": 7168 }, "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-05T10:00:00.000Z"), "id": 684434185541386240, "text": "Level scoops of your potatoes please", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 612143773, "name": "Jake Zink", "screen_name": "Jake_Zink", "lang": "en", "location": "Papillion, Nebraska", "create_at": date("2012-06-18"), "description": "Broadcaster, procrastinator, and varsity swimmer. PLHS class of 2016.", "followers_count": 228, "friends_count": 249, "statues_count": 938 }, "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-01-05T10:00:00.000Z"), "id": 684434185730170880, "text": "See our latest #Montoursville, PA #job and click to apply: Part-Time ISL&G CSA - https://t.co/wC4dc1Eeid #CustomerService #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.9205199,41.2542459"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Montoursville", "job", "CustomerService", "Hiring" }}, "user": { "id": 72046929, "name": "PA Cust. Srv. Jobs", "screen_name": "tmj_PA_cstsrv", "lang": "en", "location": "Pennsylvania", "create_at": date("2009-09-06"), "description": "Follow this account for geo-targeted Customer Service job tweets in Pennsylvania Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 378, "friends_count": 281, "statues_count": 424 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montoursville, PA", "id": "229ac897a74add15", "name": "Montoursville", "place_type": "city", "bounding_box": rectangle("-76.941018,41.23027 -76.898714,41.268554") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42081, "countyName": "Lycoming", "cityID": 4250720, "cityName": "Montoursville" } }
+{ "create_at": datetime("2016-01-05T10:00:00.000Z"), "id": 684434185893883904, "text": "@XAFCATM753 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": 684430636921282560, "in_reply_to_user": 2812022449, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2812022449 }}, "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": 4426, "friends_count": 4838, "statues_count": 49850 }, "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-05T10:00:00.000Z"), "id": 684434185914839040, "text": "But they sure can mess it up https://t.co/Wh5MMNLtg1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25377285, "name": "Blue Dilly Gal", "screen_name": "bluedillygal", "lang": "en", "location": "America", "create_at": date("2009-03-19"), "description": "grad student/ RhetComp #ClimateChangeIsReal #PrivateProfitPublicRisk #cop21 #RWNJs are funny/scary #UniteBlue #HumanRights #LoveWins #Sustainability", "followers_count": 1248, "friends_count": 2181, "statues_count": 23414 }, "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-05T10:00:00.000Z"), "id": 684434185960820736, "text": "Too bad I'll probably never go to coachella ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1647166100, "name": "Austin Bell", "screen_name": "XcBell14", "lang": "en", "location": "null", "create_at": date("2013-08-05"), "description": "CSUN // Film Student", "followers_count": 208, "friends_count": 209, "statues_count": 1223 }, "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-05T10:00:00.000Z"), "id": 684434186040651776, "text": "We wanted the world baby", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 345773306, "name": "Blessed", "screen_name": "MasonAllMighty_", "lang": "en", "location": "#KEEPWORKIN #GODWILL ", "create_at": date("2011-07-30"), "description": "I Can Do All Things Through God Who Strengthens Me...", "followers_count": 3462, "friends_count": 1576, "statues_count": 117330 }, "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-05T10:00:00.000Z"), "id": 684434186174738436, "text": "@ApolloNightLA thanks for the love forreal I honestly appreciate it . #thatshitkraey #party #breakinit #theydontwantit #tdwi #mocity #la", "in_reply_to_status": 684433765456711680, "in_reply_to_user": 540031780, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thatshitkraey", "party", "breakinit", "theydontwantit", "tdwi", "mocity", "la" }}, "user_mentions": {{ 540031780 }}, "user": { "id": 284264806, "name": "Kraey", "screen_name": "ThatShitKraey", "lang": "en", "location": "Houston/Mo City Texas", "create_at": date("2011-04-18"), "description": "#HoustonArtist #BankMusic #BreakinIt http://youtube.com/watch?v=DqRa3mOwEHw & MY NEW SINGLE on YOUTUBE Party prod. by @Ismbeats http://youtu.be/p1gBawaferc", "followers_count": 11786, "friends_count": 2175, "statues_count": 11222 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:00.000Z"), "id": 684434186388680704, "text": "@ErwinMedina19 u know she is I'm going to slap u", "in_reply_to_status": 684434080037879808, "in_reply_to_user": 3307284013, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3307284013 }}, "user": { "id": 3307706400, "name": "perla", "screen_name": "_pperks", "lang": "en", "location": "null", "create_at": date("2015-08-06"), "description": "rip daddy", "followers_count": 1223, "friends_count": 1084, "statues_count": 10094 }, "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-05T10:00:00.000Z"), "id": 684434186497703936, "text": "@KISS1023ALBANY Please play #History by @onedirection", "in_reply_to_status": 684431686298255361, "in_reply_to_user": 19632788, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "History" }}, "user_mentions": {{ 19632788, 209708391 }}, "user": { "id": 1135599187, "name": "HARRY'S FOOL", "screen_name": "Foolish4Harry", "lang": "en", "location": "stumbling through the dark ", "create_at": date("2013-01-30"), "description": "This is not the end", "followers_count": 4846, "friends_count": 2482, "statues_count": 300093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:00.000Z"), "id": 684434186539823104, "text": "love when ugly people call me ugly like how about ya look in the mirror", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2254942120, "name": "lex", "screen_name": "alexis___002", "lang": "en", "location": "null", "create_at": date("2013-12-31"), "description": "|fly high steven ❤️| south '17 |", "followers_count": 383, "friends_count": 179, "statues_count": 27707 }, "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-01-05T10:00:00.000Z"), "id": 684434186602704896, "text": "@kbischer yup it took me a bunch of tries... But finally got it. Still will be easier to try every day ��", "in_reply_to_status": 684433952556367873, "in_reply_to_user": 252733434, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 252733434 }}, "user": { "id": 596487575, "name": "Rana", "screen_name": "heartsandroses9", "lang": "en", "location": "NYC", "create_at": date("2012-06-01"), "description": "Come here for the Darren Criss, Kris Allen, NKOTB, Klaine and Glee chatter. You've been warned.", "followers_count": 863, "friends_count": 1532, "statues_count": 38381 }, "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-05T10:00:00.000Z"), "id": 684434186627756032, "text": "Join the BJ's Restaurants, Inc. team! See our latest #Hospitality #job opening here: https://t.co/efaRYhHfUf #Huntsville, AL #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.6748454,34.7153754"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Huntsville", "Hiring" }}, "user": { "id": 2377573225, "name": "BJ's Restaurants", "screen_name": "WorkAtBJs", "lang": "en", "location": "null", "create_at": date("2014-03-07"), "description": "null", "followers_count": 265, "friends_count": 0, "statues_count": 15290 }, "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-05T10:00:00.000Z"), "id": 684434186778849284, "text": "Love u 2 dad https://t.co/5Z3nvnAqdC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 350639873, "name": "Jamie Giguere", "screen_name": "jamoeeee", "lang": "en", "location": "Lincoln RI", "create_at": date("2011-08-07"), "description": "Love and football are the two greatest things in the world. - T&G Hairdressing Academy- snap@jamoeeee | insta@jamoeeee | #wolfpack", "followers_count": 620, "friends_count": 602, "statues_count": 36917 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cranston, RI", "id": "4015afc69a8a8506", "name": "Cranston", "place_type": "city", "bounding_box": rectangle("-71.550463,41.730422 -71.377473,41.807454") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4419180, "cityName": "Cranston" } }
+{ "create_at": datetime("2016-01-05T10:00:00.000Z"), "id": 684434186795618304, "text": "@GlomarResponder do they count as what Obama called a\"advanced\" nation? Because that stuff doesn't happen there", "in_reply_to_status": 684433351281917952, "in_reply_to_user": 506958753, "favorite_count": 0, "coordinate": point("-74.1455431,42.6737925"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 506958753 }}, "user": { "id": 2405067108, "name": "Thomas Burroughs", "screen_name": "BurroughsTb", "lang": "en", "location": "hilltowns upstate NY", "create_at": date("2014-03-22"), "description": "upstate NY via LI USMC 89-93 2/4 G Co wpns 0331 Lcpl for life hope I'm sane by the time I'm done", "followers_count": 404, "friends_count": 448, "statues_count": 1940 }, "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": 36001, "countyName": "Albany" } }
+{ "create_at": datetime("2016-01-05T10:00:01.000Z"), "id": 684434187500261377, "text": "01/05@13:00 - Temp 29.5F, WC 25.3F. Wind 3.9mph NNW, Gust 7.0mph. Bar 30.633in, Falling slowly. Rain 0.01in. Hum 36%. UV 2.0. SolarRad 471.", "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": 46050 }, "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-05T10:00:01.000Z"), "id": 684434187542106112, "text": "(2 of 2): they will lose a portion of their fan base. I'm the opposite. I'm transparent & proud of it. My views are always my own. #EndRant", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "EndRant" }}, "user": { "id": 34371972, "name": "Watts", "screen_name": "MrDeeJayWatts", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2009-04-22"), "description": "On-Air Personality @965AMPRadio | Music Curator | Live DJ | My views; not those of CBS | #TempleMade | Book: wattsbookings@gmail.com | Snapchat: MrDeeJayWatts", "followers_count": 1311, "friends_count": 992, "statues_count": 18934 }, "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-05T10:00:01.000Z"), "id": 684434188012007424, "text": "They forced it with the Novacane", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 598707633, "name": "Kyarahevette M.", "screen_name": "Kyaarah_", "lang": "en", "location": "Boston ", "create_at": date("2012-06-03"), "description": "So numb to it now, I feel nothing at all.", "followers_count": 1115, "friends_count": 1384, "statues_count": 10788 }, "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-05T10:00:01.000Z"), "id": 684434188070600704, "text": "Have the worst Service with AT&T slow internet as well at least 30 spam emails a day and the give excuses to why verses fixing Ugh #att", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "att" }}, "user": { "id": 52816701, "name": "Stephen Salyers", "screen_name": "Stephen_Salyers", "lang": "en", "location": "ÜT: 34.088656,-118.371366", "create_at": date("2009-07-01"), "description": "Celebrity Makeup Artist & Celebrity Groomer", "followers_count": 178, "friends_count": 183, "statues_count": 6846 }, "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-05T10:00:01.000Z"), "id": 684434188263669760, "text": "Welcome to Harambe �� #Harambe #AnimalKingdom @ Harambe Market https://t.co/Hg0M3hN5kV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.59143142,28.35895421"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Harambe", "AnimalKingdom" }}, "user": { "id": 401119180, "name": "Trey Madara", "screen_name": "TreyMadara", "lang": "en", "location": "Douglassville, PA", "create_at": date("2011-10-29"), "description": "Just a kid with a camera and a dream. Soccer lover. Movie buff. Disney nerd.", "followers_count": 356, "friends_count": 679, "statues_count": 12315 }, "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": 12097, "countyName": "Osceola", "cityID": 1224581, "cityName": "Four Corners" } }
+{ "create_at": datetime("2016-01-05T10:00:01.000Z"), "id": 684434188330639360, "text": "Vote for me and my best bud Emily for Most Likely to Be on SNL #senyorsouperlatives https://t.co/0es8XIF9tC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "senyorsouperlatives" }}, "user": { "id": 1700585011, "name": "Callan Marquart", "screen_name": "shreksbutt", "lang": "en", "location": "Castro Valley, CA", "create_at": date("2013-08-25"), "description": "All that and a bag of chips & I look like Grubber from the Gangreen Gang", "followers_count": 137, "friends_count": 298, "statues_count": 798 }, "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-05T10:00:01.000Z"), "id": 684434188498370560, "text": "Thank you�� miss you too! https://t.co/y68kFpOwG6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 98229521, "name": "mads", "screen_name": "MaddiMays", "lang": "en", "location": "sandy eggo CA", "create_at": date("2009-12-20"), "description": "stripper booty and a rack like wow", "followers_count": 400, "friends_count": 243, "statues_count": 33258 }, "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-05T10:00:01.000Z"), "id": 684434188519522304, "text": "Last time on Death of our lives.......�� #Days", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Days" }}, "user": { "id": 389730216, "name": "Kendra", "screen_name": "niki1Kendra", "lang": "en", "location": "simplexity is key ", "create_at": date("2011-10-12"), "description": "I love Soaps & Primetime Television Rebecca Herbst & Candice Patton Sharon Case Kelly Monoco are Queens #LizFF #IrisFF #SharonFF #SamFF #allblackslivesmatter", "followers_count": 989, "friends_count": 1628, "statues_count": 43124 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conyers, GA", "id": "32b057a29a79ad43", "name": "Conyers", "place_type": "city", "bounding_box": rectangle("-84.045724,33.624919 -83.927994,33.702968") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13247, "countyName": "Rockdale", "cityID": 1319336, "cityName": "Conyers" } }
+{ "create_at": datetime("2016-01-05T10:00:01.000Z"), "id": 684434188607434753, "text": "Interested in a #Nursing #job near #Atlanta, GA? This could be a great fit: https://t.co/eSmFFgYrw9 #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.39,33.76"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "Atlanta", "Hiring", "CareerArc" }}, "user": { "id": 22150756, "name": "Atlanta Nursing Jobs", "screen_name": "tmj_atl_nursing", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-02-27"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Atlanta, GA. Need help? Tweet us at @CareerArc!", "followers_count": 624, "friends_count": 338, "statues_count": 807 }, "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-05T10:00:01.000Z"), "id": 684434188611764224, "text": "@vanbruaenealex1 I cannot wait for him to search his name and see this��", "in_reply_to_status": 684409844028289024, "in_reply_to_user": 3299958797, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3299958797 }}, "user": { "id": 3261520375, "name": "Alexys Kay Osorio", "screen_name": "Osorioalexys", "lang": "en", "location": "null", "create_at": date("2015-06-30"), "description": "blessed with the best | #ripEli", "followers_count": 292, "friends_count": 300, "statues_count": 2933 }, "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-01-05T10:00:01.000Z"), "id": 684434188729217024, "text": "@ShawnMontano I'm a reporter, so naturally my phone is attached to my hand ��", "in_reply_to_status": 684433742983598080, "in_reply_to_user": 16579019, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16579019 }}, "user": { "id": 22576348, "name": "Becca Habegger", "screen_name": "BeccaHabegger", "lang": "en", "location": "Knoxville, Tennessee", "create_at": date("2009-03-02"), "description": "TV news reporter, musician, native Minnesotan & Mizzou Tiger! Opinions are mine. **FOLLOW @BeccaWBIR for my Knoxville news tweets!**", "followers_count": 1686, "friends_count": 1343, "statues_count": 6562 }, "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-05T10:00:01.000Z"), "id": 684434188787843073, "text": "I need to sign up for more scholarships. More money 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": 321437486, "name": "RaegalBaegal♥", "screen_name": "KissThis_Ass", "lang": "en", "location": "null", "create_at": date("2011-06-21"), "description": "♡A.L.H #LU19", "followers_count": 2350, "friends_count": 1818, "statues_count": 53295 }, "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-05T10:00:01.000Z"), "id": 684434188901171201, "text": "@DoperByDesign thems is mines!", "in_reply_to_status": 684434095410118656, "in_reply_to_user": 357065078, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 357065078 }}, "user": { "id": 357065078, "name": "Rose Gold®", "screen_name": "DoperByDesign", "lang": "en", "location": "null", "create_at": date("2011-08-17"), "description": "I helped start Rose Gold®", "followers_count": 1275, "friends_count": 295, "statues_count": 54319 }, "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-05T10:00:01.000Z"), "id": 684434189060538368, "text": "Wind 0 mph --. Barometer 30.40 in, Falling slowly. Temperature 45.9 °F. Rain today 0.00 in. Humidity 49%", "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": 19981 }, "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-05T10:00:01.000Z"), "id": 684434189328973826, "text": "@scottvisy https://t.co/mw9JKQVs7A", "in_reply_to_status": -1, "in_reply_to_user": 131977153, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 131977153 }}, "user": { "id": 214770250, "name": "vee-she", "screen_name": "michaelvisy", "lang": "en", "location": "waco", "create_at": date("2010-11-11"), "description": "Jesus knows me, this i love", "followers_count": 387, "friends_count": 389, "statues_count": 12383 }, "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-01-05T10:00:01.000Z"), "id": 684434189467308032, "text": "13:00 34.7°F Feels:29.8°F (Hi34.7°F/Lo18.7°F) Hum:47% Wnd:SE 5.5MPH Baro:30.43in. 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": 159, "friends_count": 261, "statues_count": 24313 }, "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-05T10:00:01.000Z"), "id": 684434189576347649, "text": "@HardLuckHero @BlizzardCareers that one seems to be less buoyant than the in game ones. ��", "in_reply_to_status": 684424608548667392, "in_reply_to_user": 2269291741, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2269291741, 171792810 }}, "user": { "id": 2873532216, "name": "BeccaBoo", "screen_name": "Five_foot_3", "lang": "en", "location": "Texas, USA", "create_at": date("2014-10-23"), "description": "Thanks for All the Fish - US Stormrage. i draw things. sometimes. SoCal native, stuck in texas. iced caramel macchiato. ginger4lyfe.", "followers_count": 167, "friends_count": 534, "statues_count": 1593 }, "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-05T10:00:01.000Z"), "id": 684434189807124480, "text": "Shit Crazy man ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2871683915, "name": "BañdChasèrr™", "screen_name": "Bandchaserjay", "lang": "en", "location": "schoolcraft bitch", "create_at": date("2014-11-10"), "description": "I Couldn't help But to know your Pain What pain it runs deep Share it with Me. Rip Craigo Rip Omar free da GanG", "followers_count": 256, "friends_count": 765, "statues_count": 959 }, "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-05T10:00:01.000Z"), "id": 684434190033678336, "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": 4204963035, "name": "mmmanisha", "screen_name": "mmmanisha", "lang": "en", "location": "null", "create_at": date("2015-11-16"), "description": "null", "followers_count": 87, "friends_count": 129, "statues_count": 138 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Herndon, VA", "id": "33d41d156b557913", "name": "Herndon", "place_type": "city", "bounding_box": rectangle("-77.413471,38.953006 -77.364481,38.988444") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5136648, "cityName": "Herndon" } }
+{ "create_at": datetime("2016-01-05T10:00:01.000Z"), "id": 684434190323073024, "text": "1h 5m wait time at Misericordia 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.570247,53.519353"), "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": 372, "friends_count": 888, "statues_count": 2521 }, "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-05T10:00:01.000Z"), "id": 684434190423556097, "text": "#PolandIsReadyForRevivalTour 31", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "PolandIsReadyForRevivalTour" }}, "user": { "id": 1691377656, "name": "Justin my Angel", "screen_name": "AngelaDziedzic1", "lang": "pl", "location": "Chicago, IL", "create_at": date("2013-08-22"), "description": "I fell in love with a 15 year old guy who dreams fulfilled now I love 21-year-old man,who helps fulfill the dreams of other!18/11/15 Thank You Justin PotterHead", "followers_count": 325, "friends_count": 808, "statues_count": 2601 }, "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-01-05T10:00:01.000Z"), "id": 684434190553620480, "text": "Up next! @panasonic press conference at #CES2016 https://t.co/ypTPJ9yefW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CES2016" }}, "user_mentions": {{ 18974329 }}, "user": { "id": 569243855, "name": "AZ Tech Beat", "screen_name": "AZTechBeat", "lang": "en", "location": "null", "create_at": date("2012-05-02"), "description": "Arizona's news source for the thriving high tech industry. Get the latest on startups, software, gadgets, funding, local events & awesomeness. Editor @TishinD", "followers_count": 4807, "friends_count": 786, "statues_count": 13775 }, "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-05T10:00:01.000Z"), "id": 684434190587179009, "text": "@AHow17 Not my picture. Look at the sign...", "in_reply_to_status": 684433952728166401, "in_reply_to_user": 93273187, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 93273187 }}, "user": { "id": 37106040, "name": "Michael Fabiano", "screen_name": "Michael_Fabiano", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-05-01"), "description": "I'm the Senior Fantasy Analyst for http://NFL.com and NFL Network and a member of the Fantasy Sports Writers Association Hall of Fame.", "followers_count": 259872, "friends_count": 1354, "statues_count": 38276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:01.000Z"), "id": 684434190679556096, "text": "@b_warford the first one will suck because the cartilage takes a really long time to heal, I have two midways and they healed real fast", "in_reply_to_status": 684414332092985344, "in_reply_to_user": 2227642968, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2227642968 }}, "user": { "id": 396829794, "name": "Kalle Kimbrell", "screen_name": "kalle_15", "lang": "en", "location": "null", "create_at": date("2011-10-23"), "description": "#UK20 @TheOriginalBizz", "followers_count": 691, "friends_count": 508, "statues_count": 12734 }, "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-01-05T10:00:01.000Z"), "id": 684434191023357953, "text": "♫ Rocking at @redventures : https://t.co/dBn9bFk85y @GetRockbot", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.847771,35.006561"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17518037, 91192339 }}, "user": { "id": 81997235, "name": "charles canady", "screen_name": "charlescanady", "lang": "en", "location": "ÜT: 35.439535,-80.636722", "create_at": date("2009-10-12"), "description": "null", "followers_count": 208, "friends_count": 364, "statues_count": 1591 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Mill, SC", "id": "fa2609437f549b51", "name": "Fort Mill", "place_type": "city", "bounding_box": rectangle("-80.991523,34.897011 -80.803177,35.107416") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45091, "countyName": "York", "cityID": 4526890, "cityName": "Fort Mill" } }
+{ "create_at": datetime("2016-01-05T10:00:02.000Z"), "id": 684434191438712838, "text": "cannnot waiiiiit theyll be Ⓜ️INE https://t.co/jGT2U1WV5e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4094734581, "name": "January's Very Own.", "screen_name": "AriiiReyyy", "lang": "en", "location": "Middletown, CT", "create_at": date("2015-11-01"), "description": "#RestEasyTita❤️.", "followers_count": 369, "friends_count": 174, "statues_count": 13473 }, "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-01-05T10:00:02.000Z"), "id": 684434191560216577, "text": "@jediishelbz it's never too late to learn!! But if you feel safer in an automatic, I would def go w that, but if anything, I'd go w the ford", "in_reply_to_status": 684433979903090688, "in_reply_to_user": 881320368, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 881320368 }}, "user": { "id": 2699541997, "name": "betha", "screen_name": "bethany_idk", "lang": "en", "location": "Houston, TX", "create_at": date("2014-08-01"), "description": "Grace is what matters. It keeps you from reaching for the gun too quickly.", "followers_count": 293, "friends_count": 321, "statues_count": 25419 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:02.000Z"), "id": 684434191669264384, "text": "Just need to be left alone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3314020136, "name": "Asshøle.", "screen_name": "mcwt_", "lang": "en", "location": "Katy, TX", "create_at": date("2015-08-12"), "description": "don't talk to me.", "followers_count": 71, "friends_count": 80, "statues_count": 2027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:02.000Z"), "id": 684434191761719296, "text": "@HayleyLequire �������� https://t.co/NVJJ6aifPH", "in_reply_to_status": 684433920436350976, "in_reply_to_user": 1376029302, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1376029302 }}, "user": { "id": 1376029302, "name": "Hayley LeQuire", "screen_name": "HayleyLequire", "lang": "en", "location": "Maryland❤️", "create_at": date("2013-04-23"), "description": "Still trying to figure out what to do with my life....For backup please check @Markiah_05 #StayGoldenPonyBoy", "followers_count": 497, "friends_count": 472, "statues_count": 15983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westminster, MD", "id": "46eefbd4521bc929", "name": "Westminster", "place_type": "city", "bounding_box": rectangle("-77.063895,39.533477 -76.9327,39.621259") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24013, "countyName": "Carroll", "cityID": 2483100, "cityName": "Westminster" } }
+{ "create_at": datetime("2016-01-05T10:00:02.000Z"), "id": 684434192076279809, "text": "Wind 3.0 mph E. Barometer 30.607 in, Falling. Temperature 31.3 °F. Rain today 0.00 in. Humidity 62%", "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": 16652 }, "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-05T10:00:02.000Z"), "id": 684434192713650177, "text": "@Forbes That's because in what 6 years there hasn't been a sequel?", "in_reply_to_status": 684389142348840961, "in_reply_to_user": 91478624, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 91478624 }}, "user": { "id": 703904233, "name": "Nolan Hodges", "screen_name": "NolanHodges1", "lang": "en", "location": "Houston, TX", "create_at": date("2012-07-18"), "description": "Hello, I'm Nolan Hodges.", "followers_count": 63, "friends_count": 221, "statues_count": 977 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:02.000Z"), "id": 684434193053396992, "text": "Honestly I'm honored just to represent the progress", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 595260722, "name": "ALPO♤ the Great", "screen_name": "AEllison97", "lang": "en", "location": "Road to Riches", "create_at": date("2012-05-31"), "description": "When you love what you do going to work is more pleasure than work", "followers_count": 664, "friends_count": 827, "statues_count": 6025 }, "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-05T10:00:02.000Z"), "id": 684434193150013440, "text": "Thanks For Nothing / Thanks For Everything is the best back to back song combo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1877707664, "name": "20NEEKTEEN", "screen_name": "nickoxcalderon", "lang": "en", "location": "null", "create_at": date("2013-09-17"), "description": "I'm 21 and I've got a lot to learn. Listen to my band @ChurchTongueIN #DIEWILD", "followers_count": 1100, "friends_count": 769, "statues_count": 29510 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsburg, IN", "id": "00f424d474454ea0", "name": "Brownsburg", "place_type": "city", "bounding_box": rectangle("-86.439701,39.800082 -86.326794,39.912651") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18063, "countyName": "Hendricks", "cityID": 1808416, "cityName": "Brownsburg" } }
+{ "create_at": datetime("2016-01-05T10:00:02.000Z"), "id": 684434193204396032, "text": "Snow this afternoon, high 39 (4 C). Low 33 (1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15953346, "name": "Amarillo Weather", "screen_name": "AmarilloWeather", "lang": "en", "location": "Amarillo, Texas", "create_at": date("2008-08-22"), "description": "Forecasts and advisories for Amarillo, Texas", "followers_count": 4045, "friends_count": 395, "statues_count": 8711 }, "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-01-05T10:00:02.000Z"), "id": 684434193246359552, "text": "Dear Los Angeles, your #uggs WILL NOT survive El Niño #Trustme #wetsheepskin #ElNinoLA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "uggs", "Trustme", "wetsheepskin", "ElNinoLA" }}, "user": { "id": 111413657, "name": "★ Jennifer Wolfe ★", "screen_name": "TheeWolfe", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-02-04"), "description": "Journalist, @KCRW news, former CNNer. Haiku afficionado. Words are weapons, learn to use them properly.", "followers_count": 2771, "friends_count": 2101, "statues_count": 7542 }, "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-05T10:00:02.000Z"), "id": 684434193405865984, "text": "Travelling to Toledo or just twittering about Toledo? https://t.co/IE8zj8dFBT #Toledo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.5552,41.6639"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Toledo" }}, "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": 974, "friends_count": 312, "statues_count": 2513330 }, "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-05T10:00:02.000Z"), "id": 684434193422536704, "text": "I AM disappointed the Minister left a loophole for ill-thinking citizens to continue fire setting. But he may address them at another time.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 58628774, "name": "Damien King", "screen_name": "DamienWKing", "lang": "en", "location": "Kingston, Jamaica", "create_at": date("2009-07-20"), "description": "Campion; New York Univ; Currently, economics lecturer at Univ. of the West Indies.", "followers_count": 3180, "friends_count": 44, "statues_count": 2688 }, "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-01-05T10:00:02.000Z"), "id": 684434193724489728, "text": "@transitorykris what happened?", "in_reply_to_status": 684430920141541377, "in_reply_to_user": 71390929, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 71390929 }}, "user": { "id": 19041154, "name": "ryan", "screen_name": "ryan505", "lang": "en", "location": "san francisco, ca", "create_at": date("2009-01-15"), "description": "made in canada over a few drinks. networks @fastly.", "followers_count": 610, "friends_count": 102, "statues_count": 6700 }, "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-05T10:00:02.000Z"), "id": 684434193753870336, "text": "OK it's 10:00 where's that \"pouring down rain\" the weather people were hyping?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2914409168, "name": "katie", "screen_name": "katerskater23", "lang": "en", "location": "Southern California", "create_at": date("2014-11-29"), "description": "This is my desi..I mean a new account. NSFW. I love Mads and Hugh equally. Once a Fannibal always a Fannibal is what I always say.", "followers_count": 360, "friends_count": 357, "statues_count": 13039 }, "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-05T10:00:02.000Z"), "id": 684434193833537537, "text": "@gxbbayy ft me cus she misses me ����", "in_reply_to_status": -1, "in_reply_to_user": 2505091926, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2505091926 }}, "user": { "id": 1676955986, "name": "Phil Hoops", "screen_name": "phileen_angel", "lang": "en", "location": "San Antonio, TX", "create_at": date("2013-08-16"), "description": "TJHS'16 || Coach Phil || @aide_medrano is mine ❤️", "followers_count": 933, "friends_count": 554, "statues_count": 68655 }, "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-05T10:00:02.000Z"), "id": 684434193942732801, "text": "#SupportOriginMelissa 38.1°F Wind:0.0mph Pressure: 30.53hpa Falling 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": 306278 }, "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-05T10:00:02.000Z"), "id": 684434194047479808, "text": "Temp: 47.0°F Wind:0.8mph Pressure: 30.420hpa 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": 55945 }, "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-05T10:00:02.000Z"), "id": 684434194139889664, "text": "Check out this epic check-in @Swarmapp!\nhttps://t.co/0DHakZPw0K https://t.co/lSlYVs2W7d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.757355,40.717145"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 240676055 }}, "user": { "id": 19257498, "name": "BIG ED", "screen_name": "NYCKING", "lang": "en", "location": "NYC", "create_at": date("2009-01-20"), "description": "NYC product", "followers_count": 4438, "friends_count": 3682, "statues_count": 363531 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Papa John's Pizza", "id": "07d9cababe884003", "name": "Papa John's Pizza", "place_type": "poi", "bounding_box": rectangle("-73.7573551,40.7171449 -73.757355,40.717145") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-01-05T10:00:02.000Z"), "id": 684434194324451328, "text": "Bruh https://t.co/lY7hzThEjO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 22021118, "name": "HARLEMNEXTDOOR™", "screen_name": "SkeezeeWonder", "lang": "en", "location": "From Capitol Heights With Love", "create_at": date("2009-02-26"), "description": "When I was young, my favorite song was Get Money.", "followers_count": 626, "friends_count": 321, "statues_count": 87059 }, "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-01-05T10:00:02.000Z"), "id": 684434194345410560, "text": "Burning rage + heavy caffineination = ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 69503579, "name": "Cassie Buckner", "screen_name": "cassbbuck", "lang": "en", "location": "Knoxville, TN", "create_at": date("2009-08-27"), "description": "Loose lips sink ships all the damn time. Not this time.", "followers_count": 166, "friends_count": 283, "statues_count": 4338 }, "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-05T10:00:02.000Z"), "id": 684434194437554176, "text": "I love sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1578215005, "name": "Michelle", "screen_name": "michellieyshell", "lang": "en", "location": "null", "create_at": date("2013-07-08"), "description": "shopaholic and dog lover", "followers_count": 520, "friends_count": 513, "statues_count": 6113 }, "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-05T10:00:02.000Z"), "id": 684434194521456640, "text": "47.7F (Feels: 47.7F) - Humidity: 65% - Wind: 3.1mph NW - Gust: 5.4mph - Pressure: 1041.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": 221142 }, "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-05T10:00:02.000Z"), "id": 684434195029049344, "text": "Tarantos a sick fuck", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1005060746, "name": "Samantha Sposato", "screen_name": "SamSposatoo", "lang": "en", "location": "null", "create_at": date("2012-12-11"), "description": "Mustard on the beat hoe", "followers_count": 1038, "friends_count": 579, "statues_count": 38345 }, "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-05T10:00:02.000Z"), "id": 684434195050041344, "text": "Where is Toledo on the map? Play the game at https://t.co/IE8zj8dFBT #Toledo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.5552,41.6639"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Toledo" }}, "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": 974, "friends_count": 312, "statues_count": 2513331 }, "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-05T10:00:02.000Z"), "id": 684434195163254784, "text": "Someone make me a dope paddle... Any takers? If you're artsy, I'll pay you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3100892299, "name": "JANUARY 3RD", "screen_name": "RookieRozay", "lang": "en", "location": "East New York, Brooklyn", "create_at": date("2015-03-20"), "description": "ΧΥΣ, Deuce. SUNY New Paltz '17. #CreepSquad", "followers_count": 278, "friends_count": 302, "statues_count": 6822 }, "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-05T10:00:03.000Z"), "id": 684434195687424002, "text": "Hey can someone show me the A & B lunch schedule for heights 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": 981882504, "name": "Carson Lesperance", "screen_name": "carsonjoyy", "lang": "en", "location": "Huntsville, TX", "create_at": date("2012-11-30"), "description": "Sam Houston State University & @dyland95 :)", "followers_count": 712, "friends_count": 721, "statues_count": 11972 }, "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-01-05T10:00:03.000Z"), "id": 684434196039733248, "text": "Hahaha! https://t.co/vCZvQMeYuc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 4237516359, "name": "Kimberly Rose", "screen_name": "KimpossibleRose", "lang": "en", "location": "null", "create_at": date("2015-11-20"), "description": "The Rose That Grew From Concrete", "followers_count": 7, "friends_count": 33, "statues_count": 15 }, "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-05T10:00:03.000Z"), "id": 684434196199112704, "text": "@GeraldYak420 @Vinceee_ny @lupash7 @RandallKraft @Bobbyh214 @CurlyCoyne @chrgdup1973 @OORAHPICS @NoHostagesTaken BYE https://t.co/6wz7035Qb5", "in_reply_to_status": 684430172817231872, "in_reply_to_user": 3018718277, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3018718277, 3130178253, 2786996610, 2329283396, 454515349, 1959741181, 740874799, 3306837340, 2211027127 }}, "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": 8938, "friends_count": 8874, "statues_count": 187551 }, "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-05T10:00:03.000Z"), "id": 684434196295749634, "text": "I need a friend to smoke a blunt with", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2337481460, "name": "nic", "screen_name": "Nikkkkkkkkkkole", "lang": "en", "location": "wonderland", "create_at": date("2014-02-10"), "description": "flower child • Capricorn • in a world of my own • ♈️♻️", "followers_count": 689, "friends_count": 468, "statues_count": 23551 }, "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-01-05T10:00:03.000Z"), "id": 684434196379635712, "text": "#BalondeOro #Cristiano", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "BalondeOro", "Cristiano" }}, "user": { "id": 1626141462, "name": "Kenzo", "screen_name": "kennyescolero", "lang": "en", "location": "NY", "create_at": date("2013-07-27"), "description": "W.I.T", "followers_count": 611, "friends_count": 352, "statues_count": 3609 }, "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-05T10:00:03.000Z"), "id": 684434196543213568, "text": "Lehner, Pysyk heading to the Amerks https://t.co/ezHfryRVzy #ROC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ROC" }}, "user": { "id": 17022537, "name": "Rochester D and C", "screen_name": "DandC", "lang": "en", "location": "Rochester, NY", "create_at": date("2008-10-28"), "description": "Download the D&C app for everything you need to #ROCtheWINTER. iPhone: http://apple.co/1uuMuqe Android: http://bit.ly/1tzAtw7 iPad: http://bit.ly/1tzAtw7", "followers_count": 51910, "friends_count": 4259, "statues_count": 85023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rochester, NY", "id": "2f1fc0d72969452b", "name": "Rochester", "place_type": "city", "bounding_box": rectangle("-92.558347,43.099801 -77.533421,44.108056") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3663000, "cityName": "Rochester" } }
+{ "create_at": datetime("2016-01-05T10:00:03.000Z"), "id": 684434196618678273, "text": "@eddiewhite3 majority of Americans don't believe half the things he says! Because he lies!", "in_reply_to_status": 684433285980631040, "in_reply_to_user": 217053410, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 217053410 }}, "user": { "id": 2455587853, "name": "Jerry Pero", "screen_name": "PeroJerry", "lang": "en", "location": "null", "create_at": date("2014-04-20"), "description": "love my wife and kids,", "followers_count": 32, "friends_count": 104, "statues_count": 585 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Homecroft, IN", "id": "27aa0b995cb547ec", "name": "Homecroft", "place_type": "city", "bounding_box": rectangle("-86.139476,39.666867 -86.12366,39.672264") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18097, "countyName": "Marion", "cityID": 1834420, "cityName": "Homecroft" } }
+{ "create_at": datetime("2016-01-05T10:00:03.000Z"), "id": 684434196660523008, "text": "After this season I'm declaring for the draft ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 353360401, "name": "Connor Gawryszewski", "screen_name": "theONLYwhiteDB", "lang": "en", "location": "Belton, TX", "create_at": date("2011-08-11"), "description": "UMHB DB Transfer/ Sophomore .. Selfie King Snapchat:ConnorAlan123 Insta:Connor_zooski Tivy Highschool Alum", "followers_count": 840, "friends_count": 462, "statues_count": 22010 }, "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-01-05T10:00:03.000Z"), "id": 684434196698234880, "text": "Afternoon showers this afternoon, high 41 (5 C). Low 34 (1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712333, "name": "LubbockWeather", "screen_name": "LubbockWeather", "lang": "en", "location": "Lubbock, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Lubbock, Texas", "followers_count": 1187, "friends_count": 93, "statues_count": 8044 }, "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-05T10:00:03.000Z"), "id": 684434196715024384, "text": "Flayk Family Management App Launches on iOS, Android https://t.co/YeHPxTduqt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "et", "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": 17540, "statues_count": 66628 }, "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-05T10:00:03.000Z"), "id": 684434196815867904, "text": "Wind 1.6 mph N. Barometer 30.64 in, Falling. Temperature 16.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 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": 156442 }, "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-05T10:00:03.000Z"), "id": 684434197121912832, "text": "Cloud Peak Energy #SkilledTrade #Job: Electrician (#Gillette, WY) https://t.co/uJFDvFH9JJ #electrician #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.5022205,44.2910915"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SkilledTrade", "Job", "Gillette", "electrician", "Jobs", "Hiring" }}, "user": { "id": 3011572896, "name": "Cloud Peak Jobs", "screen_name": "CloudPeakJobs", "lang": "en", "location": "null", "create_at": date("2015-02-06"), "description": "Cloud Peak Energy aims to be a leading energy producer operating in a safe, responsible and caring manner. Check here for current job openings.", "followers_count": 58, "friends_count": 270, "statues_count": 15 }, "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-01-05T10:00:03.000Z"), "id": 684434197151363072, "text": "@AtlantaRegional predicting Atlanta will add 2.5M people in the next 20 years", "in_reply_to_status": -1, "in_reply_to_user": 63251803, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 63251803 }}, "user": { "id": 20188209, "name": "Thornton Kennedy", "screen_name": "ATKII", "lang": "en", "location": "null", "create_at": date("2009-02-05"), "description": "PR pro | Buckhead newspaper columnist | Husband | Dad", "followers_count": 719, "friends_count": 800, "statues_count": 4834 }, "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-05T10:00:03.000Z"), "id": 684434197201551360, "text": "@MondoBitch my bad b you knew how I move", "in_reply_to_status": 684433963905990656, "in_reply_to_user": 2883702494, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2883702494 }}, "user": { "id": 350637501, "name": "Hitt$", "screen_name": "_KingShit___", "lang": "en", "location": "Mo City, TX", "create_at": date("2011-08-07"), "description": "MONEYGANGCEO #FreeMontre #MGE #TIO I'm a WeedSmokingShitTalkingBadAttitideHavingAssNiggaAndStillWillFuckYourBitchAssNigga B A L E E", "followers_count": 491, "friends_count": 475, "statues_count": 10473 }, "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-01-05T10:00:03.000Z"), "id": 684434197243650048, "text": "The Clemson #NationalChampionship hype videos are pouring in! This #WeTooDeep video has no chill! https://t.co/56YV0ivok5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NationalChampionship", "WeTooDeep" }}, "user": { "id": 331028550, "name": "Tigertown Graphics", "screen_name": "TTGrafx", "lang": "en", "location": "Clemson, South Carolina", "create_at": date("2011-07-07"), "description": "Established in 1988, we are your source for custom t-shirts, promotional items, and embroidery.", "followers_count": 6636, "friends_count": 1315, "statues_count": 3646 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clemson, SC", "id": "01ff1adb1d047095", "name": "Clemson", "place_type": "city", "bounding_box": rectangle("-82.854023,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-01-05T10:00:03.000Z"), "id": 684434197373665280, "text": "Maui to San Jose Intl on Hawaiian for $318 #CheapFlights #OGG https://t.co/Ri7M78mtX3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-156.430496,20.8986"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CheapFlights", "OGG" }}, "user": { "id": 756129644, "name": "Flight Fishing", "screen_name": "FlightFishing", "lang": "en", "location": "Orlando, FL", "create_at": date("2012-08-13"), "description": "Collecting the internet's best airfares for flexible explorers. Sign up for free at http://www.flightfishing.com", "followers_count": 1141, "friends_count": 1505, "statues_count": 8002 }, "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", "cityID": 1522700, "cityName": "Kahului" } }
+{ "create_at": datetime("2016-01-05T10:00:03.000Z"), "id": 684434197927301120, "text": "@Troy4LSU yep major bullshit.", "in_reply_to_status": 684430966819950592, "in_reply_to_user": 3260339101, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3260339101 }}, "user": { "id": 374673775, "name": "buki.", "screen_name": "BukiWilliams", "lang": "en", "location": "Manhattan, NY", "create_at": date("2011-09-16"), "description": "When the Seagulls follow the trawler, its because they think Sardines will be thrown into the sea. Fighting for PBO. #TeamManUtd London, England.", "followers_count": 529, "friends_count": 511, "statues_count": 52985 }, "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-05T10:00:03.000Z"), "id": 684434198300618757, "text": "Wind 1.4 mph NW. Barometer 30.366 in, Falling. Temperature 46.1 °F. Rain today 0.00 in. Humidity 51%", "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": 55946 }, "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-05T10:00:03.000Z"), "id": 684434198619271170, "text": "@tomverica I'm never giving up on an #Olitz pic from you. #hopelives #2016goal #olitzhope @tonygoldwyn @kerrywashington", "in_reply_to_status": -1, "in_reply_to_user": 793228710, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Olitz", "hopelives", "2016goal", "olitzhope" }}, "user_mentions": {{ 793228710, 499336680, 205302299 }}, "user": { "id": 3252283033, "name": "Hi! I'm Rose", "screen_name": "MessyAngel_Rose", "lang": "en", "location": "City of Angels", "create_at": date("2015-06-21"), "description": "Accountant from Los Angeles❤️Loves SCANDAL.. Olitz..a Gladiator. ❤️Tony's Angels ❤️Loves my Scandal friends❤️Happy Wife..Happy Life❤️", "followers_count": 354, "friends_count": 431, "statues_count": 4514 }, "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-05T10:00:03.000Z"), "id": 684434198627745792, "text": "I tried calling my mom to ask her what good fruit is in season but she didn't answer so now I'm stuck with guessing. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 992242854, "name": "Leah Ruth✡", "screen_name": "leiamoss", "lang": "en", "location": "Mt. Pleasant", "create_at": date("2012-12-05"), "description": "Inspiration and genius - one and the same. Victor Hugo #cmu", "followers_count": 755, "friends_count": 648, "statues_count": 21332 }, "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-05T10:00:03.000Z"), "id": 684434198858457088, "text": "@natstradamus__ you probably deserve it ��", "in_reply_to_status": 684406791006994432, "in_reply_to_user": 2322696349, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2322696349 }}, "user": { "id": 76350105, "name": "ORANGE", "screen_name": "orangeismedia", "lang": "en", "location": "Orlando, Fl", "create_at": date("2009-09-22"), "description": "Owner of #RespectTheMic CEO of #OrangeIsMedia #Artist #Photographer #Videographer", "followers_count": 2220, "friends_count": 2094, "statues_count": 32965 }, "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-05T10:00:03.000Z"), "id": 684434198871064576, "text": "14% of requests for food are for seniors. Many of our senior women neighbors face the same problems of unemployment https://t.co/JZvDzG0pSB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 32401447, "name": "Mid-Ohio Foodbank", "screen_name": "Mid_OHFoodbank", "lang": "en", "location": "Columbus, Ohio", "create_at": date("2009-04-17"), "description": "Providing food to thousands by partnering with more than 650 emergency feeding sites across central and eastern Ohio. Because no one should go hungry.", "followers_count": 12140, "friends_count": 2858, "statues_count": 7589 }, "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-05T10:00:03.000Z"), "id": 684434198892015616, "text": "@queenobey__ What Are You Looking For", "in_reply_to_status": 684434029085638658, "in_reply_to_user": 3000288879, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3000288879 }}, "user": { "id": 2474860022, "name": "u n i q u e ♔", "screen_name": "_MeCorona_", "lang": "en", "location": "null", "create_at": date("2014-05-02"), "description": "It's better to be unhappy alone than unhappy with someone else", "followers_count": 1056, "friends_count": 851, "statues_count": 2765 }, "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-05T10:00:03.000Z"), "id": 684434198988500992, "text": "When the time comes & niggas wanna decide to \"SHAVE THE BEARD \" , Ima be a HAWK On my Man �������� #DONTHONEY ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DONTHONEY" }}, "user": { "id": 1345695960, "name": "Jubilee⚜", "screen_name": "SayheyJubi_", "lang": "en", "location": "null", "create_at": date("2013-04-11"), "description": "ARIEL ❤️", "followers_count": 412, "friends_count": 123, "statues_count": 15484 }, "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-05T10:00:03.000Z"), "id": 684434199072346113, "text": "$EGYH IDIOT GAVE AWAY ur shares @ 0065 LAUGHING AT U! FOOOL #startup #oil #stocks #brent #gold #markets #otc #wallstreet #trading #silver", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "startup", "oil", "stocks", "brent", "gold", "markets", "otc", "wallstreet", "trading", "silver" }}, "user": { "id": 25762392, "name": "Obi", "screen_name": "aaaamhim", "lang": "en", "location": "Boston", "create_at": date("2009-03-21"), "description": "Liars never figure, figures don't lie. Boston RE pro, Stocks maniac, nonconformist. Here 2learn &2help others anyway I can. Everything I post is JUST MY OPINION", "followers_count": 10864, "friends_count": 10640, "statues_count": 35702 }, "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-05T10:00:03.000Z"), "id": 684434199194021888, "text": "Cat degiance. \"Difference is vital\" #publicart #catface #StAugustine #FL @ The Casa Monica Hotel https://t.co/wWxadmD5uT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.31364075,29.89197503"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "publicart", "catface", "StAugustine", "FL" }}, "user": { "id": 46443894, "name": "Michelle Souliere", "screen_name": "msouliere", "lang": "en", "location": "Portland, Maine", "create_at": date("2009-06-11"), "description": "Owner @GreenHandBooks.Editor:Strange Maine Gazette&blog. Illustrator. Bibliofiend. Love monsters, Forteana & Lovecraft. Ex-lib. Other 1/2 of @coastcitycomics", "followers_count": 656, "friends_count": 529, "statues_count": 4442 }, "place": { "country": "United States", "country_code": "United States", "full_name": "St Augustine, FL", "id": "01b31e336c002ed7", "name": "St Augustine", "place_type": "city", "bounding_box": rectangle("-81.344147,29.856582 -81.26716,29.940326") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12109, "countyName": "St. Johns", "cityID": 1262500, "cityName": "St. Augustine" } }
+{ "create_at": datetime("2016-01-05T10:00:03.000Z"), "id": 684434199248371712, "text": "Yesterday, when I was here and everyone else was deterred by the snow, including the employees. #portlandfail https://t.co/gEhRMJz8vw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "portlandfail" }}, "user": { "id": 1007442128, "name": "John Cal", "screen_name": "MrJohnCal", "lang": "en", "location": "Portland, OR", "create_at": date("2012-12-12"), "description": "Candy Collector, Postcard Enthusiast, and Bow Tie Connoisseur", "followers_count": 20, "friends_count": 43, "statues_count": 44 }, "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-05T10:00:04.000Z"), "id": 684434199751729152, "text": "Morning showers this afternoon, high 51 (11 C). Low 40 (4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712562, "name": "ElPasoWX", "screen_name": "ElPasoWX", "lang": "en", "location": "El Paso, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for El Paso, Texas", "followers_count": 526, "friends_count": 93, "statues_count": 8102 }, "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-05T10:00:04.000Z"), "id": 684434199881728000, "text": "@FOXLA what a great coincidence, we've heard great statements from @POTUS and @ericgarcetti today. Great job to them", "in_reply_to_status": -1, "in_reply_to_user": 9648652, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 9648652, 1536791610, 17358750 }}, "user": { "id": 2514831335, "name": "Paul Baker", "screen_name": "paulhobiebaker", "lang": "en", "location": "San Clemente CA", "create_at": date("2014-04-28"), "description": "Im a college graduate and have aspergers syndrome. I enjoy hanging out with friends, loyal fan of Gdla, playing on my xbox, helping out, and an awesome person.", "followers_count": 51, "friends_count": 71, "statues_count": 7517 }, "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-01-05T10:00:04.000Z"), "id": 684434199894450176, "text": "house 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": 50214909, "name": "KD", "screen_name": "krisdionisio_", "lang": "en", "location": "si // nyc", "create_at": date("2009-06-23"), "description": "the kind of girl u steal from the football team //", "followers_count": 431, "friends_count": 330, "statues_count": 16024 }, "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-05T10:00:04.000Z"), "id": 684434199974039552, "text": "Ladies and gentlemen, welcome to #TimesSquare #NYC #US where @DXBMediaOffice is thanked and… https://t.co/QjnhgXBbHP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.8289935,40.7568765"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TimesSquare", "NYC", "US" }}, "user_mentions": {{ 37599685 }}, "user": { "id": 256495109, "name": "عمر البوسعيدي", "screen_name": "omaralbusaidy", "lang": "en", "location": "UAE, Abu Dhabi & Dubai", "create_at": date("2011-02-23"), "description": "Muslim Emarati Author of Just Read It and member of @Aud_aa @amchamabudhabi @globalshapersad . support #hayatech Views are my own ,retweets are not endorsements", "followers_count": 3303, "friends_count": 2195, "statues_count": 25827 }, "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-05T10:00:04.000Z"), "id": 684434200032706560, "text": "I know exactly what I'm getting my sister for her birthday ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 306282137, "name": "Brandon jensen", "screen_name": "tooters14", "lang": "en", "location": "null", "create_at": date("2011-05-27"), "description": "Portland Oregon, bowler, fisherman. HIT ME UP!!!!", "followers_count": 203, "friends_count": 273, "statues_count": 2772 }, "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-05T10:00:04.000Z"), "id": 684434200032874497, "text": "Im selfish af, I don't share what'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": 509005339, "name": "Kasey N. Watkins", "screen_name": "Kasey_Watkins7", "lang": "en", "location": "null", "create_at": date("2012-02-29"), "description": "Kaiden Elijah.", "followers_count": 358, "friends_count": 496, "statues_count": 18731 }, "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-05T10:00:04.000Z"), "id": 684434200427020288, "text": "#Mario's 2 For 1 Pizza #The Bowl #Prince George https://t.co/K4ITxrRimY Excellent https://t.co/vJ2hiFV35n", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.79079,53.899013"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Mario", "The", "Prince" }}, "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": 3645, "friends_count": 507, "statues_count": 162178 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Prince George, British Columbia", "id": "0582c884a3ab504d", "name": "Prince George", "place_type": "city", "bounding_box": rectangle("-122.900929,53.812891 -122.604368,54.043014") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-05T10:00:04.000Z"), "id": 684434200598986752, "text": "It didn't rain once for the first 3 months I was here. Now it rains every day. Thanks SF.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 549890992, "name": "Andy Creeth", "screen_name": "andycreeth", "lang": "en", "location": "San Francisco, CA", "create_at": date("2012-04-10"), "description": "Engineer at http://Academia.edu | Computing Enthusiast | Mountaineer in Training", "followers_count": 28, "friends_count": 60, "statues_count": 70 }, "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-05T10:00:04.000Z"), "id": 684434200863354881, "text": "I tend to be relaxed when someone is drilling and poking at my teeth. Average heart rate in the 60's. https://t.co/SMTkOzn1ag", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 95909224, "name": "TazCat", "screen_name": "Taz_Cat1", "lang": "en", "location": "Boston, Ma", "create_at": date("2009-12-10"), "description": "null", "followers_count": 247, "friends_count": 216, "statues_count": 13151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston Common, Boston", "id": "332fde136a1694b6", "name": "Boston Common", "place_type": "neighborhood", "bounding_box": rectangle("-71.069469,42.352362 -71.06196,42.35772") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-01-05T10:00:04.000Z"), "id": 684434200892542976, "text": "@koguma_chann then you're garnet!", "in_reply_to_status": 684378229893935105, "in_reply_to_user": 3856235235, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3856235235 }}, "user": { "id": 1934764105, "name": "ashley ✨", "screen_name": "shellypoop", "lang": "en", "location": "null", "create_at": date("2013-10-04"), "description": "she/her, 22, sasuke, fandom shitpost account is @planktongf", "followers_count": 124, "friends_count": 161, "statues_count": 3975 }, "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-05T10:00:04.000Z"), "id": 684434201005944832, "text": "@_SluttyBoi_ @mike_golden55 look up online and just mess around with some designs", "in_reply_to_status": 684434010467086336, "in_reply_to_user": 4021443919, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4021443919, 587801673 }}, "user": { "id": 488498968, "name": "JaySmoove", "screen_name": "TyVaughnFos", "lang": "en", "location": "Reidsville, NC", "create_at": date("2012-02-10"), "description": "#TooNiceForRice\n#InSafeHouseWeTrust\n\n\n\n\n\n\n\n\n\n\n\n\n\n\nstay gold ponyboy", "followers_count": 470, "friends_count": 507, "statues_count": 11657 }, "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-01-05T10:00:04.000Z"), "id": 684434201051922432, "text": "We're #hiring! Click to apply: Clinical Assistant - Resource Team - https://t.co/51ek0eA4Zd #Healthcare #Honolulu, HI #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8480364,21.3136151"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Healthcare", "Honolulu", "Job", "Jobs", "CareerArc" }}, "user": { "id": 23526678, "name": "TMJ- HON Health Jobs", "screen_name": "tmj_hon_health", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Healthcare job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 406, "friends_count": 304, "statues_count": 188 }, "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-05T10:00:04.000Z"), "id": 684434201345527808, "text": "Ah Bueno ustedes Traian a Sean Penn pajuo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 115715447, "name": "Luis Calandriello M.", "screen_name": "Calandriellol", "lang": "es", "location": "Zulia-Venezuela", "create_at": date("2010-02-19"), "description": "jugare Futbol hasta que me muera, te amo Venezuela.", "followers_count": 691, "friends_count": 803, "statues_count": 7914 }, "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-01-05T10:00:04.000Z"), "id": 684434201513336832, "text": "Check out Anna Wiseman's listing in #Lincolnton #NC https://t.co/b6lnPR5j97 #realestate #realtor https://t.co/ixuqsNXGqb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.179311,35.487857"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Lincolnton", "NC", "realestate", "realtor" }}, "user": { "id": 2445654229, "name": "RE/MAX Crossroads", "screen_name": "lincolntonremax", "lang": "en", "location": "Lincolnton NC", "create_at": date("2014-04-15"), "description": "null", "followers_count": 71, "friends_count": 43, "statues_count": 6128 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boger City, NC", "id": "00cc455e739ccdf1", "name": "Boger City", "place_type": "city", "bounding_box": rectangle("-81.213666,35.462566 -81.159325,35.51609") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37109, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-05T10:00:04.000Z"), "id": 684434201551192064, "text": "Pegula reportedly issues an ultimatum https://t.co/8O85j8okxX #ROC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ROC" }}, "user": { "id": 17022537, "name": "Rochester D and C", "screen_name": "DandC", "lang": "en", "location": "Rochester, NY", "create_at": date("2008-10-28"), "description": "Download the D&C app for everything you need to #ROCtheWINTER. iPhone: http://apple.co/1uuMuqe Android: http://bit.ly/1tzAtw7 iPad: http://bit.ly/1tzAtw7", "followers_count": 51910, "friends_count": 4259, "statues_count": 85024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rochester, NY", "id": "2f1fc0d72969452b", "name": "Rochester", "place_type": "city", "bounding_box": rectangle("-92.558347,43.099801 -77.533421,44.108056") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3663000, "cityName": "Rochester" } }
+{ "create_at": datetime("2016-01-05T10:00:04.000Z"), "id": 684434201555394560, "text": "Which is why she's on a podium with her fake tears and nicely done eyebrows and extensions... At 14 �� https://t.co/xbY7X0CYui", "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": ".CraiG.", "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": 2024, "friends_count": 1276, "statues_count": 117365 }, "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-05T10:00:04.000Z"), "id": 684434201614000128, "text": "��i hate spending money.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2311349672, "name": "Great Value Biggie", "screen_name": "fatboiislimm", "lang": "en", "location": "null", "create_at": date("2014-01-25"), "description": "BUFFET BOYZ. FAT KIDS EAT FIRST Phoenix,AZ", "followers_count": 220, "friends_count": 258, "statues_count": 6231 }, "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-05T10:00:04.000Z"), "id": 684434201764954112, "text": "Im not even supposed to be here oops lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 838122260, "name": "Monica", "screen_name": "ShadesOfMonica", "lang": "en", "location": "ur mom", "create_at": date("2012-09-21"), "description": "J.Cole❤️ Kendrick Lamar❤️ LDR❤️ #TeamRob", "followers_count": 1388, "friends_count": 694, "statues_count": 42993 }, "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-05T10:00:04.000Z"), "id": 684434202155216896, "text": "If ya bitch don't want a busy niggghaa she must want a broke one ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 333840072, "name": "إشراق", "screen_name": "CallMeMaximo", "lang": "en", "location": "Nassau Bahamas ", "create_at": date("2011-07-11"), "description": "Perfectly Imperfect2015♥ AVA Instagram:@Bahamian_thug", "followers_count": 966, "friends_count": 624, "statues_count": 69292 }, "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-01-05T10:00:04.000Z"), "id": 684434202255818752, "text": "I don't own a gun.\nI don't ever want I gun.\n\nBut I am an advocate for those w/ mental illnesses. They were stigmatized\n\n #StopGunViolence", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "StopGunViolence" }}, "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": 2583, "friends_count": 150, "statues_count": 95549 }, "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-05T10:00:04.000Z"), "id": 684434202297679872, "text": "Cloudy this afternoon, high 50 (10 C). Low 43 (6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712512, "name": "AbileneWeather", "screen_name": "AbileneWeather", "lang": "en", "location": "Abilene, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Abilene, Texas", "followers_count": 929, "friends_count": 93, "statues_count": 8142 }, "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-05T10:00:04.000Z"), "id": 684434202331234307, "text": "Its coo.. Ill be good��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 614691878, "name": "TEYonce❤️", "screen_name": "kiilla_tay", "lang": "en", "location": "The Trap, Oregon. ", "create_at": date("2012-06-21"), "description": "curiousity killed the cat", "followers_count": 328, "friends_count": 337, "statues_count": 4632 }, "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-05T10:00:04.000Z"), "id": 684434202763243521, "text": "@kristenkaleal thanks for the idea! That's a great idea!", "in_reply_to_status": 684360812245291009, "in_reply_to_user": 15921793, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15921793 }}, "user": { "id": 348158461, "name": "Jaclyn Seymour", "screen_name": "SeymourJacOnTV", "lang": "en", "location": "null", "create_at": date("2011-08-03"), "description": "Faith & Family. Television Reporter. Ginger. Ohio native. SoDak resident. CSU Alum. Peanut butter addict!", "followers_count": 462, "friends_count": 570, "statues_count": 913 }, "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-05T10:00:04.000Z"), "id": 684434202800947200, "text": "Holy crap, Tumblr is silent.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 916596266, "name": "竹下E本", "screen_name": "takenji_ebooks", "lang": "en", "location": "The Past", "create_at": date("2012-10-31"), "description": "Eh, close enough.", "followers_count": 147, "friends_count": 1, "statues_count": 99495 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "f42a863798156617", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.459696,47.491912 -122.224433,47.734145") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-01-05T10:00:04.000Z"), "id": 684434202842894336, "text": "Want to work in #KAPOLEI, HI? View our latest opening: https://t.co/1DIrG17wfX #Labor #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-158.056892,21.3354034"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "KAPOLEI", "Labor", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 118676555, "name": "TMJ-HI Labor Jobs", "screen_name": "tmj_HI_LABOR", "lang": "en", "location": "Hawaii", "create_at": date("2010-03-01"), "description": "Follow this account for geo-targeted General Labor job tweets in Hawaii Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 283, "friends_count": 279, "statues_count": 4 }, "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-05T10:00:04.000Z"), "id": 684434203090391041, "text": "what a beautiful gloomy day to BE FUCKING 21 IM FUCKING 21 ITS MY BIRTHDAAAAAYYYYYYY BIIIIIIIIITTTTCCCCHHHHHHH ����������������������������������������������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 95834349, "name": "nice & outgoing guy", "screen_name": "yoloswagmomen", "lang": "en", "location": "null", "create_at": date("2009-12-09"), "description": "i put the angst in gangsta", "followers_count": 293, "friends_count": 457, "statues_count": 21185 }, "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-05T10:00:04.000Z"), "id": 684434203140702208, "text": "I just wanna tell both of em to stfu I can't hear the 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": 40040533, "name": "djweirdnasty", "screen_name": "TheWeirdDJ", "lang": "en", "location": "Somewhere in the air", "create_at": date("2009-05-14"), "description": "#ScreenWriter #Comedian #Poet #Dancer #Musician #Actor #discjockey multi talented weirdo for booking call 2674286238 or Email: djweirdnastyphilly@gmail.com", "followers_count": 3217, "friends_count": 1599, "statues_count": 28688 }, "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-05T10:00:04.000Z"), "id": 684434203241484288, "text": "@evelyn_vigueras pero aver como me va hoy si no I am dropping it ��", "in_reply_to_status": 684433973909389312, "in_reply_to_user": 3594365114, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 3594365114 }}, "user": { "id": 407236194, "name": "yasmin", "screen_name": "yassminibarra", "lang": "en", "location": "Durham, NC", "create_at": date("2011-11-07"), "description": "dgo, mx ❣", "followers_count": 1249, "friends_count": 560, "statues_count": 50820 }, "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-05T10:00:04.000Z"), "id": 684434203333660672, "text": "elohel", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "et", "is_retweet": false, "user": { "id": 717457838, "name": "⠀", "screen_name": "oohrosemary", "lang": "en", "location": "miami✈️hou$ton", "create_at": date("2012-07-25"), "description": "dominican mamí✨ #TXST", "followers_count": 1348, "friends_count": 298, "statues_count": 6294 }, "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-05T10:00:04.000Z"), "id": 684434203438661632, "text": "Only my car would break down while trying to take someone else to get their car fixed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 52748914, "name": "Job 33", "screen_name": "intellectuWOAH", "lang": "en", "location": "Wherever the lames stay ", "create_at": date("2009-07-01"), "description": "His imagination is wild and extravagant, it escapes incessantly from every restraint of reason and taste... #BGSU", "followers_count": 1126, "friends_count": 1019, "statues_count": 102402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Groveport, OH", "id": "0028e98e6d4016e8", "name": "Groveport", "place_type": "city", "bounding_box": rectangle("-82.93875,39.800159 -82.860512,39.890319") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3932606, "cityName": "Groveport" } }
+{ "create_at": datetime("2016-01-05T10:00:04.000Z"), "id": 684434203497218048, "text": "@karrencajayon lol, these birthday candles drip too much and ruin my bowl �� #hurting", "in_reply_to_status": 684433190140772353, "in_reply_to_user": 1695007244, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hurting" }}, "user_mentions": {{ 1695007244 }}, "user": { "id": 24468373, "name": "kaydog", "screen_name": "KatieTeeh", "lang": "en", "location": "I'm just moving about my ways ", "create_at": date("2009-03-14"), "description": "rise up", "followers_count": 415, "friends_count": 821, "statues_count": 208 }, "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-05T10:00:04.000Z"), "id": 684434203589529600, "text": "#Zidane confirma que sera DT Del #RealMadrid hasta Junio del 2018.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "Zidane", "RealMadrid" }}, "user": { "id": 1398435163, "name": "Jhannet Sanchez", "screen_name": "JhannetSanchez", "lang": "en", "location": "null", "create_at": date("2013-05-02"), "description": "@UTAustin Journalism Broadcast Alumni- LOVE sports- Current Univision Fellow- Texas borned & raised- Dallas ✈️ Austin opinions are my own", "followers_count": 394, "friends_count": 689, "statues_count": 1533 }, "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-05T10:00:05.000Z"), "id": 684434203870494720, "text": "King's Creek Plantation #Construction #Job: PAINTER (#Williamsburg, VA) https://t.co/VQZvreDRS0 #va #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.6350285,37.2668356"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Construction", "Job", "Williamsburg", "va", "Jobs", "Hiring" }}, "user": { "id": 3015648286, "name": "King's Creek Jobs", "screen_name": "KingsCreekJobs", "lang": "en", "location": "null", "create_at": date("2015-02-03"), "description": "Check here for current job opportunities with King's Creek Plantation, in beautiful Williamsburg, VA.", "followers_count": 21, "friends_count": 46, "statues_count": 5 }, "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": 51199, "countyName": "York" } }
+{ "create_at": datetime("2016-01-05T10:00:05.000Z"), "id": 684434204021542912, "text": "Vpn be saving 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": 318701051, "name": "Niqueee ➰", "screen_name": "Glogirlniquee", "lang": "en", "location": "HTX ", "create_at": date("2011-06-16"), "description": "| Rest in peace Arthur ❤️ . Never been the type to play tag , bitch I been IT ‼️", "followers_count": 671, "friends_count": 636, "statues_count": 15767 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:05.000Z"), "id": 684434204709502977, "text": "Welcome to Miami ☀️�� @ Bayside - Port of Miami https://t.co/WEeZaqrjbF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.1861047,25.77787366"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 128940232, "name": "A Real Mermaid", "screen_name": "CaliforniaBuzz_", "lang": "en", "location": "The Sea, Atlantis", "create_at": date("2010-04-02"), "description": "FMT Daytona Fall '13 | Zoe Girl | ReKonstruKtion | II-XXIV-MCMXCV | SC & IG: California_Buzz | #10Reasons | #DoIt4Dame | I'm a Mermaid I'm meant to swim freely", "followers_count": 1764, "friends_count": 1415, "statues_count": 60939 }, "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-05T10:00:05.000Z"), "id": 684434204734521345, "text": "Rip to my ughs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3285997818, "name": "gg", "screen_name": "GizzMagana", "lang": "en", "location": "null", "create_at": date("2015-07-20"), "description": "null", "followers_count": 129, "friends_count": 118, "statues_count": 2788 }, "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-05T10:00:05.000Z"), "id": 684434204747239424, "text": "I swear I feel like the Quad is the Suburbs & Rogers, Gresham, & Whitehurst the trap ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 69970596, "name": "GHOST", "screen_name": "raymonta", "lang": "en", "location": "null", "create_at": date("2009-08-29"), "description": "ODU 19'", "followers_count": 570, "friends_count": 321, "statues_count": 33324 }, "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-05T10:00:05.000Z"), "id": 684434204768235520, "text": "if you want to be mad, making a murder on Netflix is for you 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": 948664524, "name": "chlo ✨", "screen_name": "chloeeberghorst", "lang": "en", "location": "null", "create_at": date("2012-11-14"), "description": "fox enthusiast. average picture taking potato.", "followers_count": 349, "friends_count": 131, "statues_count": 8827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jenison, MI", "id": "49e7adaebc60092a", "name": "Jenison", "place_type": "city", "bounding_box": rectangle("-85.921923,42.885184 -85.781818,42.936233") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26139, "countyName": "Ottawa", "cityID": 2641680, "cityName": "Jenison" } }
+{ "create_at": datetime("2016-01-05T10:00:05.000Z"), "id": 684434205124726785, "text": "I have to steal this caption https://t.co/mEzaFYwUk4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 44774988, "name": "Lizzy Liz✨", "screen_name": "KM_Heartbreaker", "lang": "en", "location": "null", "create_at": date("2009-06-04"), "description": "Veni, vidi, vici❤️", "followers_count": 1103, "friends_count": 1193, "statues_count": 64339 }, "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-01-05T10:00:05.000Z"), "id": 684434205187518464, "text": "@shawnuscainius @pplswar @20committee His FP political \"friends\" (IRA, Hamas, Hezbolla, Putin) are no great humanists", "in_reply_to_status": 684381829730832384, "in_reply_to_user": 1092583993, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1092583993, 3343387330, 635608354 }}, "user": { "id": 2968369291, "name": "Nils van Otterloo", "screen_name": "OlekeBolekeKnol", "lang": "en", "location": "LA, CA", "create_at": date("2015-01-08"), "description": "Master of Social Work. Father of Mila and Isadora. Musician. Cyclist. Sartorialist. Neurophiliac.", "followers_count": 58, "friends_count": 144, "statues_count": 1116 }, "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-05T10:00:05.000Z"), "id": 684434205191704576, "text": "Stahp https://t.co/oNMstp9i39", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 244798887, "name": "jecelle", "screen_name": "jaycelle_", "lang": "en", "location": "null", "create_at": date("2011-01-29"), "description": "18 / schs '16 / ☧242", "followers_count": 258, "friends_count": 111, "statues_count": 32775 }, "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-05T10:00:05.000Z"), "id": 684434205405765633, "text": "@cillastaacks happy birthday ��", "in_reply_to_status": -1, "in_reply_to_user": 2764482280, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2764482280 }}, "user": { "id": 758259150, "name": "✨کریستال", "screen_name": "CryssAlyssa", "lang": "en", "location": "traveling ", "create_at": date("2012-08-14"), "description": "luxury lover , dancer ... kinda.", "followers_count": 1446, "friends_count": 1070, "statues_count": 49966 }, "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-05T10:00:05.000Z"), "id": 684434205606973440, "text": "Can you recommend anyone for this #Hospitality #job? https://t.co/YKwQ1pTTq5 #TheWoodlands, TX #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.458492,30.164915"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "TheWoodlands", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 22485197, "name": "TMJ-HOU HRTA Jobs", "screen_name": "tmj_hou_hrta", "lang": "en", "location": "Houston, TX", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Houston, TX. Need help? Tweet us at @CareerArc!", "followers_count": 459, "friends_count": 299, "statues_count": 1601 }, "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-05T10:00:05.000Z"), "id": 684434205669855232, "text": "Express Factory Outlet-Store Manager-Camarillo Outlet-Camarillo, CA - Express: (#Camarillo, CA) https://t.co/OiCz0g4iHl #Retail #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.0376023,34.2163937"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Camarillo", "Retail", "Job", "Jobs" }}, "user": { "id": 2565844555, "name": "ExpressCareers", "screen_name": "expresscareers", "lang": "en", "location": "Columbus, OH", "create_at": date("2014-06-13"), "description": "Love everything fashion and the latest trends and styles? Why not mix business with pleasure - work at Express, we're hiring!", "followers_count": 260, "friends_count": 60, "statues_count": 856 }, "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-05T10:00:05.000Z"), "id": 684434205674094592, "text": "@Wirlander @Trudgin time for a margarita!", "in_reply_to_status": 684434092155383808, "in_reply_to_user": 195678379, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 195678379, 77464236 }}, "user": { "id": 627363601, "name": "Freddie", "screen_name": "fredfalcone", "lang": "en", "location": "Austin, Texas", "create_at": date("2012-07-05"), "description": "Cycling and technology (the chip kind). UT grad. Day job, tech VP. Tweets r own", "followers_count": 301, "friends_count": 568, "statues_count": 24426 }, "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-05T10:00:05.000Z"), "id": 684434205682577408, "text": "I hate immature people 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": 1890329436, "name": "vict☮ria", "screen_name": "vicmongiello", "lang": "en", "location": "rip creamer♡", "create_at": date("2013-09-21"), "description": "all i need is within me • john scott", "followers_count": 933, "friends_count": 698, "statues_count": 20680 }, "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-05T10:00:05.000Z"), "id": 684434205787303936, "text": "Cloudy this afternoon, high 51 (11 C). Low 42 (6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712530, "name": "MidlandWeather", "screen_name": "MidlandWeather", "lang": "en", "location": "Midland/Odessa, TX", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Midland/Odessa, TX", "followers_count": 879, "friends_count": 93, "statues_count": 8126 }, "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-01-05T10:00:05.000Z"), "id": 684434205888135170, "text": "SOME1'S WATCHING US~ 4SOME REASON OR ANOTHER~JUST KNOW EYES ARE ALWAYS ON US~RATHER 4GOOD /BAD~OR JUST PLAIN NOSEY! https://t.co/pY2t1IdHjo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 131702362, "name": "THEROYALMC", "screen_name": "GODS_ANGEL2", "lang": "en", "location": "GIVEN ENLARGED TERRITORY 4GOD", "create_at": date("2010-04-10"), "description": "SENT BY GOD 4 GOD A MERE VESSEL 4THE GOD IN ME 2PREACH AND TEACH CALLED &CHOSEN BY GOD 2EXPOUND ON THE REVELATION OF HIS WORD~CHANGED FROM GLORY2GLORY2GLORY...", "followers_count": 5573, "friends_count": 873, "statues_count": 5812 }, "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-05T10:00:05.000Z"), "id": 684434206173184000, "text": "God damn it's rain like a mofo����������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 447439316, "name": "J-®od", "screen_name": "Jerrincoontz", "lang": "en", "location": "Villa Rica, GA", "create_at": date("2011-12-26"), "description": "How We Got Here Ohh It All Started With A Ice Cold Beer", "followers_count": 395, "friends_count": 284, "statues_count": 15935 }, "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-05T10:00:05.000Z"), "id": 684434206659760128, "text": "@MarkEOrtega @BenHarper @jimmykimmel tfti brah", "in_reply_to_status": 684431295439474688, "in_reply_to_user": 21990431, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 21990431, 30938602, 26053643 }}, "user": { "id": 244840833, "name": "Adam Kadourhe", "screen_name": "AdamKadourhe", "lang": "en", "location": "Bay Area ", "create_at": date("2011-01-29"), "description": "Covering Bay Area sports w/ @RyanLeong for Metro Networks | Site Coverage Editor @Pac12Networks | Founder of #PuppySquad", "followers_count": 1559, "friends_count": 705, "statues_count": 80736 }, "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-05T10:00:05.000Z"), "id": 684434206861033472, "text": "I have a love/hate relationship with routine.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 292713832, "name": "Cailin Harper", "screen_name": "Cailin_Harper", "lang": "en", "location": "null", "create_at": date("2011-05-03"), "description": "null", "followers_count": 784, "friends_count": 279, "statues_count": 19953 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Groves, TX", "id": "9af98223aefe3151", "name": "Groves", "place_type": "city", "bounding_box": rectangle("-93.944772,29.924735 -93.890543,29.96429") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48245, "countyName": "Jefferson", "cityID": 4831328, "cityName": "Groves" } }
+{ "create_at": datetime("2016-01-05T10:00:05.000Z"), "id": 684434206903013378, "text": "The Deseret News gives UNFORGETTABLE a great review. Come to the launch party tonight at 7pm. #BNAuthorSigning #EricJamesStone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BNAuthorSigning", "EricJamesStone" }}, "user": { "id": 4352173993, "name": "Barnes & Noble", "screen_name": "BN_Orem", "lang": "en", "location": "null", "create_at": date("2015-11-24"), "description": "null", "followers_count": 35, "friends_count": 97, "statues_count": 47 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murray, UT", "id": "5811c847bb5f82b6", "name": "Murray", "place_type": "city", "bounding_box": rectangle("-111.932066,40.623952 -111.862386,40.684428") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4953230, "cityName": "Murray" } }
+{ "create_at": datetime("2016-01-05T10:00:05.000Z"), "id": 684434206915563520, "text": "Can you recommend anyone for this #Healthcare #job? https://t.co/cx4h2QvuoI #Madison, WI #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.4012302,43.0730517"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "Madison", "Hiring", "CareerArc" }}, "user": { "id": 24579293, "name": "TM-MAD Health Jobs", "screen_name": "tmj_mad_health", "lang": "en", "location": "Madison, WI", "create_at": date("2009-03-15"), "description": "Follow this account for geo-targeted Healthcare job tweets in Madison, WI. Need help? Tweet us at @CareerArc!", "followers_count": 375, "friends_count": 303, "statues_count": 48 }, "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.491889 -86.24955,47.309715") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55025, "countyName": "Dane", "cityID": 5548000, "cityName": "Madison" } }
+{ "create_at": datetime("2016-01-05T10:00:05.000Z"), "id": 684434206945062913, "text": "@DanicaDelRey hehehe", "in_reply_to_status": 684433758003572736, "in_reply_to_user": 30331175, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 30331175 }}, "user": { "id": 440437280, "name": "anna", "screen_name": "annabb09", "lang": "en", "location": "NJ", "create_at": date("2011-12-18"), "description": "find something you would die for, and live for it", "followers_count": 291, "friends_count": 298, "statues_count": 7902 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wharton, NJ", "id": "015a38893c7657ef", "name": "Wharton", "place_type": "city", "bounding_box": rectangle("-74.611183,40.885987 -74.535882,40.951251") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34027, "countyName": "Morris", "cityID": 3480390, "cityName": "Wharton" } }
+{ "create_at": datetime("2016-01-05T10:00:05.000Z"), "id": 684434207129534464, "text": "I think I'm dying", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 396660671, "name": "Mamacita™", "screen_name": "Babie_Babich", "lang": "en", "location": "VIII.XX.MMXIV", "create_at": date("2011-10-23"), "description": "Paul Mitchell Future Professional............... Girlfriend/Property of @Drewyour15", "followers_count": 1333, "friends_count": 410, "statues_count": 18957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shelby, MI", "id": "5c8c81a4fef94364", "name": "Shelby", "place_type": "city", "bounding_box": rectangle("-83.095428,42.623997 -82.973298,42.715777") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26127, "countyName": "Oceana", "cityID": 2672840, "cityName": "Shelby" } }
+{ "create_at": datetime("2016-01-05T10:00:05.000Z"), "id": 684434207234457600, "text": "@SlimROZAY you real ��", "in_reply_to_status": 684433641678712832, "in_reply_to_user": 20925907, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20925907 }}, "user": { "id": 2289018352, "name": "Papi Mike", "screen_name": "__yadad", "lang": "en", "location": "Pittsburgh✈️Tampa", "create_at": date("2014-01-17"), "description": "I'm a hopeless romantic who lives in the hood. What kinda sick joke you tryna play God?", "followers_count": 955, "friends_count": 702, "statues_count": 30443 }, "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-01-05T10:00:05.000Z"), "id": 684434207377076224, "text": "13:00:02 |Temp: 41.7ºF | Wind Chill 41.7ºF |Dew Point 20.8ºF | Rain today: 0.00 inches | Wind: 3.0 mph from the ENE, Gusting to 7.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": 88045 }, "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-05T10:00:05.000Z"), "id": 684434207393746944, "text": "@iMaikL @zeinabnajm Everything he does is put under a microscope & nit picked. It's ridiculous. #FACTS is fun & something to listen to 4 now", "in_reply_to_status": 684433385251598337, "in_reply_to_user": 84930270, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FACTS" }}, "user_mentions": {{ 84930270, 171781820 }}, "user": { "id": 51994808, "name": "AP", "screen_name": "Kiddie_A", "lang": "en", "location": "93640", "create_at": date("2009-06-29"), "description": "I am the Martin Luther King Jr. of Generation Y.", "followers_count": 1008, "friends_count": 994, "statues_count": 81781 }, "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-01-05T10:00:05.000Z"), "id": 684434207456677888, "text": "Not feeling it today:(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4461094752, "name": "patrizia warren", "screen_name": "patriziaawarren", "lang": "en", "location": "null", "create_at": date("2015-12-04"), "description": "pvhs", "followers_count": 97, "friends_count": 108, "statues_count": 240 }, "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-05T10:00:05.000Z"), "id": 684434207548952577, "text": "I live for Socratic seminars", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315474070, "name": "Phia (:", "screen_name": "sophiadaniielle", "lang": "en", "location": "California, USA", "create_at": date("2013-03-29"), "description": "smile", "followers_count": 749, "friends_count": 594, "statues_count": 14808 }, "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-05T10:00:05.000Z"), "id": 684434207590998018, "text": "@kevin_cloen @cjones_814 @SchaeferTony Kevin you not even a good dj fr, you a wannabe Dj Khaled", "in_reply_to_status": 684434010542596096, "in_reply_to_user": 1050886794, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1050886794, 2732831098, 857225917 }}, "user": { "id": 912980780, "name": "Josh Robertson", "screen_name": "itsyaboyj_rob", "lang": "en", "location": "Elyria, OH", "create_at": date("2012-10-29"), "description": "Elyria Senior / U.S Army", "followers_count": 722, "friends_count": 328, "statues_count": 35770 }, "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-01-05T10:00:05.000Z"), "id": 684434207599230976, "text": "Thas how this lil girl at hais named isabella is. https://t.co/SdSWMqNw4d", "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": 1578, "friends_count": 1547, "statues_count": 37004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:06.000Z"), "id": 684434208027103232, "text": "I leave for Disneyland tomorrow and my birthday is Friday. I should not feel this depressed.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 182529240, "name": "mariah", "screen_name": "mariahpapaya15", "lang": "en", "location": "null", "create_at": date("2010-08-24"), "description": "I wish I was in Disneyland", "followers_count": 191, "friends_count": 415, "statues_count": 10364 }, "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-01-05T10:00:06.000Z"), "id": 684434208035442688, "text": "@BarackObama #StopGunViolence what's wrong with common sense laws to control the violence", "in_reply_to_status": 684426196138893312, "in_reply_to_user": 813286, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "StopGunViolence" }}, "user_mentions": {{ 813286 }}, "user": { "id": 4357794372, "name": "Rhodrick J Haralson", "screen_name": "RhodrickH", "lang": "en", "location": "null", "create_at": date("2015-11-25"), "description": "null", "followers_count": 30, "friends_count": 70, "statues_count": 663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:06.000Z"), "id": 684434208463417344, "text": "Should Jim Caldwell stay, or should Jim Caldwell go? THAT is THE… https://t.co/dKTE7fYn9w", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.48105627,42.72164361"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 112727250, "name": "Al Martin", "screen_name": "AlMartinWKAR", "lang": "en", "location": "Detroit, MI", "create_at": date("2010-02-09"), "description": "WKAR Radio and TV Sports Reporter/Host and MSU School of Journalism ALUM! Emmy nominated and MAB Best in Category winner. 'Greatest is what you make it.'", "followers_count": 1655, "friends_count": 1394, "statues_count": 34197 }, "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-05T10:00:06.000Z"), "id": 684434208916295681, "text": "First world probs but how am I expected to have a 10 am class in a room with no drinks or food allowed? I need me my starbs ☕️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 49502198, "name": "Ellen Bakira", "screen_name": "ella_bakira", "lang": "en", "location": "null", "create_at": date("2009-06-21"), "description": "What would Olivia Pope do? #lifegoals", "followers_count": 119, "friends_count": 162, "statues_count": 1836 }, "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-05T10:00:06.000Z"), "id": 684434209084067840, "text": "Wind 4 mph SE. Barometer 1011.8 hPa, Falling. Temperature 81.0 °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": 87, "friends_count": 265, "statues_count": 140350 }, "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-05T10:00:06.000Z"), "id": 684434209398587394, "text": "Mostly cloudy this afternoon, high 62 (17 C). Low 56 (13 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712593, "name": "CorpusChristiWX", "screen_name": "CorpusChristiWX", "lang": "en", "location": "Corpus Christi, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Corpus Christi, Texas", "followers_count": 1013, "friends_count": 93, "statues_count": 8117 }, "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-05T10:00:06.000Z"), "id": 684434209461514241, "text": "Goes to meet with VP of Student Affairs soaked from 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": 132363279, "name": "mg", "screen_name": "mikeyyocean", "lang": "en", "location": "△ m.A.A.d city △", "create_at": date("2010-04-12"), "description": "welcome to the Hotel California", "followers_count": 423, "friends_count": 413, "statues_count": 21583 }, "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-05T10:00:06.000Z"), "id": 684434209583316992, "text": "Good morning, Disney. We coming for you. #BingBoomRoehl @ Orlando-Kissimmee https://t.co/05WVHg4rTX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.49917516,28.33358789"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BingBoomRoehl" }}, "user": { "id": 23357327, "name": "Angela Sullivan", "screen_name": "astoldbyang", "lang": "en", "location": "Chicago, IL", "create_at": date("2009-03-08"), "description": "Twenty-one. Nashville transplant who misses living in London. Love God. Love people. Love self. #hawkey Hates Mondays, but loves lasagna.", "followers_count": 508, "friends_count": 593, "statues_count": 20099 }, "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": 12097, "countyName": "Osceola" } }
+{ "create_at": datetime("2016-01-05T10:00:06.000Z"), "id": 684434209612513280, "text": "some of the niggas are so deceptive using my style like a contraceptive", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2240089736, "name": "cutie", "screen_name": "chyneeezy", "lang": "en", "location": "inglewood, CA", "create_at": date("2013-12-10"), "description": "such a 90s hip hop head", "followers_count": 1466, "friends_count": 694, "statues_count": 48673 }, "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-05T10:00:06.000Z"), "id": 684434209679785984, "text": "Tuesday Leadership... (@ Calvary Chapel Golden Springs in Diamond Bar, CA) https://t.co/tuboehdWht", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.82320634,34.0028158"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17548379, "name": "Sammy Aceves", "screen_name": "sammyaceves", "lang": "en", "location": "Brea, CA.", "create_at": date("2008-11-21"), "description": "Jesus • Worship Leader • Singer • Guitars • Married to my love, Tricia • Family • Friends • Maps -- that's what it's all about.", "followers_count": 634, "friends_count": 992, "statues_count": 9462 }, "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-05T10:00:06.000Z"), "id": 684434209763504128, "text": ".@PonsSublicius @maxnrgmax OBAMA WAS A minority, trained at Harvard, to meet some quota. Most likely never went to class he didn't enjoy", "in_reply_to_status": 684432372301959169, "in_reply_to_user": 621219247, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 621219247, 234419508 }}, "user": { "id": 265666449, "name": "mark wonderful", "screen_name": "markwonderful", "lang": "en", "location": "Phoenix, Az", "create_at": date("2011-03-13"), "description": "Christian, Ex-1%er, Divorced 2003, Asperger's Az-Born-Boomer Conservative Pro-Israel Fotos Actor Writer Reader 13yrs protecting 1%ers. RETIRED", "followers_count": 1863, "friends_count": 1544, "statues_count": 130418 }, "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-05T10:00:06.000Z"), "id": 684434209818152960, "text": "@FaradayFuture unveils its concept car... Will it ever hit the roads? I'll have more from #CES2016 @NBCLA at Noon https://t.co/AMSihRM6uq", "in_reply_to_status": -1, "in_reply_to_user": 65585297, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CES2016" }}, "user_mentions": {{ 65585297, 17379685 }}, "user": { "id": 16191064, "name": "Mekahlo Medina", "screen_name": "MekahloNBCLA", "lang": "en", "location": "Los Angeles, California", "create_at": date("2008-09-08"), "description": "@NBCLA Tech/Social Media Anchor/Reporter, @NAHJ Nat'l Pres; Proud Native New Mexican. Follows\\Retweets=not endorsements. Tweets to me may end up on TV/online.", "followers_count": 59283, "friends_count": 22013, "statues_count": 12419 }, "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-05T10:00:06.000Z"), "id": 684434209893560321, "text": "@young_barney @cftallent got to put the definition into urban dictionary so others aren't confused.", "in_reply_to_status": 684433950387900417, "in_reply_to_user": 55075300, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 55075300, 260787960 }}, "user": { "id": 2260606616, "name": "island•queen", "screen_name": "Maiolu_Payne7", "lang": "en", "location": "O'ahu, Hawai'i ", "create_at": date("2013-12-24"), "description": "#KCCO Necromancer of Regret #ChooseyLover Ai Aida Pi\nViolet Belle 10/15/15", "followers_count": 121, "friends_count": 125, "statues_count": 16086 }, "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-01-05T10:00:06.000Z"), "id": 684434210224877568, "text": "#Hospitality #Job in #Honolulu, Hawaii: New Store Opening - Restaurant at Nordstrom https://t.co/rq3KN3qOzy #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "Job", "Honolulu", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 23897767, "name": "TMJ-HON Hosp. Jobs", "screen_name": "tmj_hon_hrta", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-11"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 412, "friends_count": 292, "statues_count": 267 }, "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-05T10:00:06.000Z"), "id": 684434210325569536, "text": "I'm really craving donuts", "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": 657, "friends_count": 378, "statues_count": 34724 }, "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-05T10:00:06.000Z"), "id": 684434210497560576, "text": "Temp: 29.5°F - Dew Point: 19.9° - Wind: 13.3 mph - Gust: 19.7 - Rain Today: 0.00in. - Pressure: 30.49in, - 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": 11567 }, "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-05T10:00:06.000Z"), "id": 684434210510209028, "text": "Thank you @peanuttillman for everything this season. Recover fast. #respect #Panthers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "respect", "Panthers" }}, "user_mentions": {{ 519795737 }}, "user": { "id": 34071492, "name": "Jake Lancaster", "screen_name": "NCJake2013", "lang": "en", "location": "Greensboro, NC", "create_at": date("2009-04-21"), "description": "Jake. College grad. Lover of music, movies, anything Carolina, and exercise. I'm in the Coke business.", "followers_count": 268, "friends_count": 974, "statues_count": 14010 }, "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-05T10:00:06.000Z"), "id": 684434210723991552, "text": "#Honolulu, HI #Manufacturing #Job: Aluminum Welder at Aerotek https://t.co/l6rMf1Gw2Z #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "Honolulu", "Manufacturing", "Job", "Jobs", "Hiring" }}, "user": { "id": 23527770, "name": "TMJ - HON Manuf Jobs", "screen_name": "tmj_hon_manuf", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Manufacturing job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 224, "friends_count": 202, "statues_count": 1 }, "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-05T10:00:06.000Z"), "id": 684434210782724096, "text": "look at that boy Jo! https://t.co/2GyuHhOYhm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 98070190, "name": "T . G R A Y", "screen_name": "TGrayWINNING", "lang": "en", "location": "H O U $ T O N , F L E X A $", "create_at": date("2009-12-19"), "description": "|Winning Royalty™ | #iProduceRoyalMusic", "followers_count": 1107, "friends_count": 1492, "statues_count": 27061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:06.000Z"), "id": 684434210891812864, "text": "@ChinaDiamond_ it ok ����", "in_reply_to_status": 684434076770545664, "in_reply_to_user": 4049294235, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4049294235 }}, "user": { "id": 373735463, "name": "Since '96", "screen_name": "Ocho_600", "lang": "en", "location": "Chicago, IL", "create_at": date("2011-09-14"), "description": "#RichAuraBenjiBoy✈️", "followers_count": 4096, "friends_count": 1709, "statues_count": 61889 }, "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-05T10:00:06.000Z"), "id": 684434210904383490, "text": "The wolld is sick of your lies! All #business owners are!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "business" }}, "user": { "id": 456726486, "name": "HOLIDAY HEADQUARTERS", "screen_name": "sportparadise", "lang": "en", "location": "null", "create_at": date("2012-01-06"), "description": "#Holiday #Shop #Gifts #Sales #Deals #Bargains #Ebay #Clothing #Fashion #Jewelry #Hunting #Electronics #HomeDecor #Toys #Automotive #teamfollowback", "followers_count": 101898, "friends_count": 94152, "statues_count": 935512 }, "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-05T10:00:06.000Z"), "id": 684434210908581888, "text": "��Balifornia", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2827581649, "name": "♛DG", "screen_name": "DylGivens", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-09-22"), "description": "GOD Bless #TheCircle™ 20 • Sapient Visionary • Athlete Model Actor Writer • CoOwner Brand/Apparel (@preclothing_) #Pre™ • Booking:iyern93@gmail.com IG:Dgivens_", "followers_count": 525, "friends_count": 273, "statues_count": 9245 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:06.000Z"), "id": 684434210916978688, "text": "Retail Sales Associate - OfficeMax: (#Kahului, HI) https://t.co/vcIdmFhmBT #Veterans #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-156.454708,20.881571"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Kahului", "Veterans", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 38082277, "name": "TMJ-MAU Jobs", "screen_name": "tmj_mau_jobs", "lang": "en", "location": "Maui, HI", "create_at": date("2009-05-05"), "description": "Follow this account for geo-targeted Other job tweets in Maui, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 547, "friends_count": 535, "statues_count": 16 }, "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-05T10:00:06.000Z"), "id": 684434210971586560, "text": "annoyed asf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 410416730, "name": "1|23♒️", "screen_name": "cocobabyy_", "lang": "en", "location": "null", "create_at": date("2011-11-11"), "description": "#UNCG20✨", "followers_count": 1324, "friends_count": 1012, "statues_count": 37479 }, "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-01-05T10:00:06.000Z"), "id": 684434211542036481, "text": "@KWkq7 @Debi129 @musicnews_facts I don't want to read people rants, save that for facebook.", "in_reply_to_status": 684433683802140673, "in_reply_to_user": 1668150649, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1668150649, 16444544, 2958642883 }}, "user": { "id": 543780273, "name": "Mr. Weeks", "screen_name": "MrDane1982", "lang": "en", "location": "Bronx, New York", "create_at": date("2012-04-02"), "description": "We need realistic policies, Smart power and someone willing to make Hard choices. We need LEADERS! @HillaryClinton Supporter Democrats2016 TeamObama", "followers_count": 2529, "friends_count": 2414, "statues_count": 43896 }, "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-05T10:00:06.000Z"), "id": 684434211663667200, "text": "Wind 1.3 mph S. Barometer 30.721 in, Falling. Temperature 19.0 °F. Rain today 0.00 in. Humidity 66%", "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": 5590 }, "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-05T10:00:06.000Z"), "id": 684434211843883008, "text": "@business @POTUS Obama\"NUMBED\"to his #gunviolence wRaids onLatino BrownFAMILIES acrossAmerica WE FEEL HIS \"NUMB\"bro! https://t.co/oRMCUEhIso", "in_reply_to_status": 684431709971038209, "in_reply_to_user": 34713362, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "gunviolence" }}, "user_mentions": {{ 34713362, 1536791610 }}, "user": { "id": 21237679, "name": "WISE LATINA", "screen_name": "EusebiaAq", "lang": "en", "location": "America", "create_at": date("2009-02-18"), "description": "My mission is to end hate in America! You can can be part of the human solution or part of the injustice painful reality.GOD Bless America and our President!", "followers_count": 2142, "friends_count": 2061, "statues_count": 153622 }, "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-05T10:00:06.000Z"), "id": 684434212049391616, "text": "Morning clouds/afternoon sun this afternoon, high 43 (6 C). Low 32 (0 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712706, "name": "TulsaWX", "screen_name": "TulsaWX", "lang": "en", "location": "Tulsa, Oklahoma", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Tulsa, Oklahoma", "followers_count": 517, "friends_count": 93, "statues_count": 8179 }, "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-05T10:00:06.000Z"), "id": 684434212049428480, "text": "@NCogsmith @XylariaVG @wishingtroll they just took additional x-rays. Oh boy.", "in_reply_to_status": 684432909239894017, "in_reply_to_user": 309292791, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 309292791, 17910014, 2310634763 }}, "user": { "id": 20335491, "name": "Michael Sparks", "screen_name": "Ponyzord", "lang": "en", "location": "Yakima, WA", "create_at": date("2009-02-07"), "description": "Musician, Humorist, Babydangler. Rogue extraordinaire. Sold my soul for three Yodels and half a Ding-Dong. Battletag: ponyzord#1825", "followers_count": 422, "friends_count": 1100, "statues_count": 24420 }, "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-05T10:00:07.000Z"), "id": 684434212280123392, "text": "If I can reach the heart and soul in a tweet, surely Twitter Executives in their Infinite Wisdom can do better. No?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "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": 1079, "friends_count": 987, "statues_count": 57764 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:07.000Z"), "id": 684434212292694016, "text": "Can you recommend anyone for this #job? Primavera Developer - https://t.co/CWK7Rf5Czs #SanDiego, CA #IT #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.1572551,32.7153292"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "SanDiego", "IT", "Hiring", "CareerArc" }}, "user": { "id": 21697389, "name": "San Diego IT Jobs", "screen_name": "tmj_san_it", "lang": "en", "location": "San Diego, CA", "create_at": date("2009-02-23"), "description": "Follow this account for geo-targeted Software Dev. - General/IT job tweets in San Diego, CA. Need help? Tweet us at @CareerArc!", "followers_count": 803, "friends_count": 330, "statues_count": 152 }, "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-05T10:00:07.000Z"), "id": 684434212448038912, "text": "10 minutes into class and my teacher has already cursed thrice i think 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": 136818065, "name": "arielle victoria", "screen_name": "ayveetee", "lang": "en", "location": "depaul so hard", "create_at": date("2010-04-24"), "description": "killer swag virus", "followers_count": 245, "friends_count": 153, "statues_count": 11174 }, "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-05T10:00:07.000Z"), "id": 684434212489981952, "text": "Just posted a photo @ Atlanta, Georgia https://t.co/CW3ZMXuL9R", "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, "user": { "id": 252418195, "name": "Damon Bruce Wayne", "screen_name": "Damon_NewEra", "lang": "en", "location": "Pennsylvania, USA Atlanta,Ga", "create_at": date("2011-02-14"), "description": "On A mission to prestige as a king #YoungKingzMovement #Hydroteam add me #SoicalMediaWorldwidedaedae", "followers_count": 675, "friends_count": 886, "statues_count": 9538 }, "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-05T10:00:07.000Z"), "id": 684434212615798784, "text": "@jaltuiter mito orbigafa jolti", "in_reply_to_status": 684433929726722048, "in_reply_to_user": 2606097865, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user_mentions": {{ 2606097865 }}, "user": { "id": 167205262, "name": "Morena Tropicana™", "screen_name": "lulliromero", "lang": "pt", "location": "toca gando e andando", "create_at": date("2010-07-15"), "description": "babuinos bobocas balbuciando em bando", "followers_count": 2894, "friends_count": 410, "statues_count": 137226 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland Beach, FL", "id": "4a29c5d001f991a6", "name": "Highland Beach", "place_type": "city", "bounding_box": rectangle("-80.070171,26.39072 -80.062653,26.426347") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1230200, "cityName": "Highland Beach" } }
+{ "create_at": datetime("2016-01-05T10:00:07.000Z"), "id": 684434213802643456, "text": "Asian man going off on his wife in whole ass public ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1378494782, "name": "T⚡️", "screen_name": "TMFnClay_", "lang": "en", "location": "Dallas, Flexa$", "create_at": date("2013-04-24"), "description": "23| Scorpio| #CowboysNation| 1738 | #UNTForHoweverLong| Snapchat: tdot_14", "followers_count": 1671, "friends_count": 999, "statues_count": 124711 }, "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-05T10:00:07.000Z"), "id": 684434213819449345, "text": "���� It makes me want to color my hair dark again. https://t.co/lXv2VHw9Ok", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 186290631, "name": "Skye", "screen_name": "Skye_Carrington", "lang": "en", "location": "your dad's house ", "create_at": date("2010-09-02"), "description": "I gave 50 Cent a dollar.", "followers_count": 1130, "friends_count": 666, "statues_count": 59246 }, "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-05T10:00:07.000Z"), "id": 684434214054432768, "text": "DSS 26 carrier lock on STEREO B\nFrequency: 8.4462GHz\nIDLE OFF 1 TURBO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-116.89,35.426667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3380828067, "name": "Deep Space Network", "screen_name": "dsn_status", "lang": "en", "location": "Goldstone, Madrid, Canberra", "create_at": date("2015-07-17"), "description": "Tracking what @NASA's Deep Space Network listens to. (An unofficial bot by @russss.)", "followers_count": 138, "friends_count": 13, "statues_count": 15773 }, "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" } }
+{ "create_at": datetime("2016-01-05T10:00:07.000Z"), "id": 684434214335459329, "text": "6 Words That Every Travel Junkie Should Know https://t.co/bD7nxC3fWM via @hellogiggles https://t.co/CDcLxtXAYs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 219682445 }}, "user": { "id": 21897390, "name": "Jade Stewart Nolie", "screen_name": "travelagent4u", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2009-02-25"), "description": "Passionate travel agent! Lover of music~ appletinis~conversation! Proud wife and mother. Love to assist w/ travel plans. 702-215-5935.", "followers_count": 1134, "friends_count": 2112, "statues_count": 1818 }, "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-05T10:00:07.000Z"), "id": 684434214637432833, "text": "I AM THEY - King Of Love: Song Sessions\n#SongOfTheDay �� https://t.co/wUjudWNA8y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SongOfTheDay" }}, "user": { "id": 2853552457, "name": "Noah H.", "screen_name": "NoahHazen24", "lang": "en", "location": "Intents Harbors", "create_at": date("2014-10-12"), "description": "He Turned My Mess into a Miracle, While Simultaneously Setting the Prisoner Free in Me, Therefore, In the Power of Love and Grace, Anything is Possible.", "followers_count": 165, "friends_count": 1559, "statues_count": 2203 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plainwell, MI", "id": "9cd2ae8c73365927", "name": "Plainwell", "place_type": "city", "bounding_box": rectangle("-85.661534,42.432399 -85.622067,42.47505") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26005, "countyName": "Allegan", "cityID": 2664740, "cityName": "Plainwell" } }
+{ "create_at": datetime("2016-01-05T10:00:07.000Z"), "id": 684434214746460160, "text": "My moms grandmother told me in my dream all you need is 6 And now I know what it means �� six of your realest souls you know to carry you ⚰��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 420931578, "name": "Khallid Pierce", "screen_name": "TheCoolalove", "lang": "en", "location": "Roaming 4 it ", "create_at": date("2011-11-25"), "description": "Wanna play Clark Kent you better have ya cape on! UHS C/O 1⃣3⃣ Alum, Student Athlete #9️⃣9️⃣ #BethanyCollege #AOAA Roaming for Success. Strong Believer in God", "followers_count": 1072, "friends_count": 899, "statues_count": 33067 }, "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-01-05T10:00:07.000Z"), "id": 684434214834442240, "text": "@ellie_lacy so sad all those are gone (not rly) but thanks ✌��", "in_reply_to_status": 684429068473774080, "in_reply_to_user": 2559590720, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2559590720 }}, "user": { "id": 940566338, "name": "Hannah Elaine", "screen_name": "HannahELacy", "lang": "en", "location": "null", "create_at": date("2012-11-10"), "description": "MO STATE || ΑΧΩ", "followers_count": 406, "friends_count": 520, "statues_count": 688 }, "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-05T10:00:07.000Z"), "id": 684434214855426048, "text": "Mostly sunny this afternoon, high 42 (6 C). Low 35 (2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712679, "name": "OKCWX", "screen_name": "OKCWX", "lang": "en", "location": "Oklahoma City, Oklahoma", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Oklahoma City, Oklahoma", "followers_count": 186, "friends_count": 93, "statues_count": 8206 }, "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-05T10:00:07.000Z"), "id": 684434214888931328, "text": "1/5/2016 - 12:00\nTemp: 41.6F \nHum: 56%\nWind: 1.0 mph\nBaro: 30.460in. & 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": 105, "friends_count": 52, "statues_count": 48718 }, "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-05T10:00:07.000Z"), "id": 684434215220473856, "text": "This https://t.co/CiuRXsruEE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 250643783, "name": "Rising Apple", "screen_name": "RisingAppleBlog", "lang": "en", "location": "New York, NY", "create_at": date("2011-02-11"), "description": "New York Mets site that's part of the @FanSided Network. We're informative, opinionated, & accurate. Tweets usually from Danny Abriano: @D_Abriano", "followers_count": 4643, "friends_count": 987, "statues_count": 41791 }, "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-05T10:00:07.000Z"), "id": 684434215258046464, "text": "this was being sent to me https://t.co/FK4ya9WvEs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1258250484, "name": "cry baby", "screen_name": "GoatSlaughterer", "lang": "en", "location": "sevierville, tn", "create_at": date("2013-03-10"), "description": "i love kyle simpson so very much. cupcake hair princess. one of those crazy girls. glo girl who doesnt need a garment.", "followers_count": 196, "friends_count": 17, "statues_count": 6764 }, "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-05T10:00:07.000Z"), "id": 684434215283208193, "text": "I miss green and purple ketchup.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28073881, "name": "Nick Maillet", "screen_name": "NickMeteorain", "lang": "en", "location": "LA/Wherever I get sent", "create_at": date("2009-04-01"), "description": "TV Post Production. Editor/Colorist. Vidya. Gym. Coffee Addict. video editor for @techraptr", "followers_count": 3863, "friends_count": 158, "statues_count": 3581 }, "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-05T10:00:07.000Z"), "id": 684434215635685376, "text": "Finalized schedule for @Bison_Cheer Invite! Lots of great teams this Sunday! DM us your entrance song requests! ���� https://t.co/svMt5SYk6Z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3024501022 }}, "user": { "id": 834169148, "name": "Double B Ent. Inc.", "screen_name": "DoubleBEntInc", "lang": "en", "location": "Darien, IL", "create_at": date("2012-09-19"), "description": "Double B Entertainment Inc. is a premier disc jockey, sound and lighting entertainment company serving the Chicagoland Metro areas.", "followers_count": 494, "friends_count": 305, "statues_count": 717 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Darien, IL", "id": "001dfcdd3eb26fdb", "name": "Darien", "place_type": "city", "bounding_box": rectangle("-88.030611,41.685721 -87.940587,41.768777") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1718628, "cityName": "Darien" } }
+{ "create_at": datetime("2016-01-05T10:00:07.000Z"), "id": 684434215799230466, "text": "someone who knows how to do eyeshadow really well plz do mine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1003127737, "name": "Kayla", "screen_name": "_kwhitman", "lang": "en", "location": "MD", "create_at": date("2012-12-10"), "description": "null", "followers_count": 549, "friends_count": 266, "statues_count": 23996 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chesapeake Ranch Estates, MD", "id": "4dd2cff80a3e3636", "name": "Chesapeake Ranch Estates", "place_type": "city", "bounding_box": rectangle("-76.451763,38.319092 -76.380792,38.387749") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24009, "countyName": "Calvert", "cityID": 2416050, "cityName": "Chesapeake Ranch Estates" } }
+{ "create_at": datetime("2016-01-05T10:00:07.000Z"), "id": 684434215870447616, "text": "I really miss the feeling of having closure with someone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2675390784, "name": "Marisa Barcia", "screen_name": "marisabarcia_", "lang": "en", "location": "Blackwood, NJ", "create_at": date("2014-07-23"), "description": "null", "followers_count": 739, "friends_count": 528, "statues_count": 50709 }, "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-05T10:00:07.000Z"), "id": 684434215899901953, "text": "@kash2396 ^^^^^^god damn comedian!!!!!", "in_reply_to_status": 684430047495745536, "in_reply_to_user": 341071479, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 341071479 }}, "user": { "id": 302385622, "name": "jordan spellis", "screen_name": "JordanSpellis", "lang": "en", "location": "UT, once had 70 favorites ", "create_at": date("2011-05-20"), "description": "hey", "followers_count": 499, "friends_count": 402, "statues_count": 6498 }, "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-05T10:00:07.000Z"), "id": 684434215933313025, "text": "@viratise you're welcome", "in_reply_to_status": 684016437502017541, "in_reply_to_user": 2456617136, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2456617136 }}, "user": { "id": 66562172, "name": "LJ Aviles", "screen_name": "LifeCoachLJ", "lang": "en", "location": "New Jersey, USA", "create_at": date("2009-08-17"), "description": "FREE - How To Get More Followers➡http://LifeCoachLJ.com/FreeTwitter⬅ Entrepreneur & Wrestler help ppl Grow Brand & Business w/ Social Media Marketing", "followers_count": 4039, "friends_count": 4739, "statues_count": 30679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bridgeton, NJ", "id": "78fa23c1159f23d3", "name": "Bridgeton", "place_type": "city", "bounding_box": rectangle("-75.25581,39.403136 -75.183652,39.490858") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34011, "countyName": "Cumberland", "cityID": 3407600, "cityName": "Bridgeton" } }
+{ "create_at": datetime("2016-01-05T10:00:07.000Z"), "id": 684434216067596290, "text": "Wind 2.2 mph ESE. Barometer 29.92 in, Steady. Temperature 54.7 °F. Rain today 0.02 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.13472222,33.81666667"), "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": 94, "statues_count": 13444 }, "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-01-05T10:00:07.000Z"), "id": 684434216180924417, "text": "���� https://t.co/IWtIglF202", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2307152977, "name": "Sophia", "screen_name": "sophiakateri97", "lang": "en", "location": "Masshole", "create_at": date("2014-01-23"), "description": "WMHS '16 | 3•21•15", "followers_count": 340, "friends_count": 536, "statues_count": 5194 }, "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-05T10:00:08.000Z"), "id": 684434216453423104, "text": "This Oregon thing is actually pretty rad. Go farmers.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 108772888, "name": "New Huntyear", "screen_name": "HHavins", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-01-26"), "description": "pew pew pew!!!!! -lasers, probably", "followers_count": 243, "friends_count": 242, "statues_count": 1681 }, "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-01-05T10:00:08.000Z"), "id": 684434216470188033, "text": "@genexporter @SenSanders are you fucking retarded planned parenthood screens women for cancer and gives prenatals and check ups to women", "in_reply_to_status": 684433744803921920, "in_reply_to_user": 17301193, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17301193, 29442313 }}, "user": { "id": 3029709375, "name": "Syd♉️", "screen_name": "arden_sydney", "lang": "en", "location": "null", "create_at": date("2015-02-10"), "description": "You probably hate me and I definitely don't give a shit", "followers_count": 236, "friends_count": 155, "statues_count": 3709 }, "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-05T10:00:08.000Z"), "id": 684434216545730561, "text": "COME DOWN TO GLENDALE AT THE ROAR ROOM ON JANUARY 24TH 2016 AND SEE YOUNG KERMIT PERFORM HIS NEW… https://t.co/JMcY1NnDmv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.14461924,33.80187259"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1292110422, "name": "Young Kermit", "screen_name": "youngkermitbib", "lang": "en", "location": "Long Beach, CA", "create_at": date("2013-03-23"), "description": "null", "followers_count": 950, "friends_count": 2031, "statues_count": 6948 }, "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-05T10:00:08.000Z"), "id": 684434216612843520, "text": "@dodo Haven't seen anything like the coyote in a downtown Chicago convenience store cooler. If humans no longer exist, coyotes will rule...", "in_reply_to_status": 684388401341165568, "in_reply_to_user": 1604444052, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1604444052 }}, "user": { "id": 2481255624, "name": "sunny long-kaake", "screen_name": "cobalttash", "lang": "en", "location": "null", "create_at": date("2014-05-06"), "description": "null", "followers_count": 98, "friends_count": 195, "statues_count": 9592 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Bend, WA", "id": "67edb63739f2bce0", "name": "North Bend", "place_type": "city", "bounding_box": rectangle("-121.813119,47.478968 -121.763542,47.512483") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5349485, "cityName": "North Bend" } }
+{ "create_at": datetime("2016-01-05T10:00:08.000Z"), "id": 684434216872878080, "text": "See our latest #LIVERPOOL, NY #job and click to apply: Tax Preparer - https://t.co/3YZiWvvt9V #Accounting #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.2177046,43.106456"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LIVERPOOL", "job", "Accounting", "Hiring", "CareerArc" }}, "user": { "id": 318736689, "name": "Syracuse Acct Jobs", "screen_name": "tmj_NYS_ACCT", "lang": "en", "location": "Syracuse, NY", "create_at": date("2011-06-16"), "description": "Follow this account for geo-targeted Accounting job tweets in Syracuse, NY from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 285, "friends_count": 274, "statues_count": 14 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Liverpool, NY", "id": "b52b7a0e2263db4b", "name": "Liverpool", "place_type": "city", "bounding_box": rectangle("-76.254017,43.082506 -76.183767,43.138185") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36067, "countyName": "Onondaga", "cityID": 3642884, "cityName": "Liverpool" } }
+{ "create_at": datetime("2016-01-05T10:00:08.000Z"), "id": 684434216973692928, "text": "Hit me up with some sweet #photography themed #shirts, #mugs etc that I can #feature on the #blog this month! #comment", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "photography", "shirts", "mugs", "feature", "blog", "comment" }}, "user": { "id": 3438798358, "name": "Amanda Pratt", "screen_name": "ILVArtistry", "lang": "en", "location": "Madeira Beach, FL", "create_at": date("2015-08-24"), "description": "Portrait Photographer in Madeira Beach, Florida", "followers_count": 606, "friends_count": 1380, "statues_count": 245 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pinellas Park, FL", "id": "d598cb5ad6f7a09f", "name": "Pinellas Park", "place_type": "city", "bounding_box": rectangle("-82.752428,27.824633 -82.663128,27.897396") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1256975, "cityName": "Pinellas Park" } }
+{ "create_at": datetime("2016-01-05T10:00:08.000Z"), "id": 684434217267118080, "text": "@waters_brennen @AndrewCorona6 Brennen I will kick your ass, you got beat up by Morgan drane", "in_reply_to_status": 684434009938464768, "in_reply_to_user": 796525578, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 796525578, 751026576 }}, "user": { "id": 3314008224, "name": "Jim Morrison", "screen_name": "Bailey_P97", "lang": "en", "location": "null", "create_at": date("2015-08-12"), "description": "GANG! NTX Pirates/ Excluders", "followers_count": 322, "friends_count": 303, "statues_count": 1357 }, "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-05T10:00:08.000Z"), "id": 684434217283915776, "text": "Partly cloudy this afternoon, high 57 (14 C). Low 46 (8 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712488, "name": "SanAntonioWX", "screen_name": "SanAntonioWX", "lang": "en", "location": "San Antonio, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for San Antonio, Texas", "followers_count": 251, "friends_count": 93, "statues_count": 8196 }, "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-05T10:00:08.000Z"), "id": 684434217326014464, "text": "my problem was I got too attached to something that was never mines to begin with .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 69821051, "name": "BRIANA .", "screen_name": "BRE_utiful_", "lang": "en", "location": "60th st (Southside of Market) ", "create_at": date("2009-08-29"), "description": "null", "followers_count": 1777, "friends_count": 1355, "statues_count": 69609 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Darby, PA", "id": "442f8732107cab6d", "name": "Darby", "place_type": "city", "bounding_box": rectangle("-75.275294,39.912157 -75.247034,39.930325") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4218152, "cityName": "Darby" } }
+{ "create_at": datetime("2016-01-05T10:00:08.000Z"), "id": 684434217409720320, "text": "@sacca Saturday! So please keep some fresh powder for me.", "in_reply_to_status": 684431787787956224, "in_reply_to_user": 586, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 586 }}, "user": { "id": 17646335, "name": "Christoph Messagie", "screen_name": "ChristophM", "lang": "en", "location": "San Francisco", "create_at": date("2008-11-26"), "description": "Stuff that matters: Startups, VCs, Angels, Data & Analytics, Technology & Finance + anything M&A related. @ FactSet", "followers_count": 228, "friends_count": 772, "statues_count": 1410 }, "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-05T10:00:08.000Z"), "id": 684434217476964352, "text": "Shit gets crazy like I'm getting iced coffee and may buy some NyQuil to turn up aka go to sleep early like @KThild @mirandaheberlig", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 358920199, 805557474 }}, "user": { "id": 412291979, "name": "Cara Toniola", "screen_name": "_grammycara", "lang": "en", "location": "somewhere between pa ga and al", "create_at": date("2011-11-14"), "description": "Millersville University grad. I like tequila, cheese and dogs. #dONTpANIC", "followers_count": 412, "friends_count": 277, "statues_count": 10104 }, "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-05T10:00:08.000Z"), "id": 684434218018025472, "text": "Happy birthday loser @KyleHanford", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 338990926 }}, "user": { "id": 1969971222, "name": "kenzz", "screen_name": "mackenziebaldi9", "lang": "en", "location": "null", "create_at": date("2013-10-18"), "description": "STL", "followers_count": 583, "friends_count": 544, "statues_count": 3641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bel-Nor, MO", "id": "a53be29890e92ae4", "name": "Bel-Nor", "place_type": "city", "bounding_box": rectangle("-90.325816,38.69292 -90.309667,38.710193") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2904348, "cityName": "Bel-Nor" } }
+{ "create_at": datetime("2016-01-05T10:00:08.000Z"), "id": 684434218227732486, "text": "@TFairchild69 I know son lol I said yah", "in_reply_to_status": 684433975318818816, "in_reply_to_user": 506671585, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 506671585 }}, "user": { "id": 437135069, "name": "Gavin Failla", "screen_name": "gavinfailla", "lang": "en", "location": "null", "create_at": date("2011-12-14"), "description": "snapchat: gavinfailla", "followers_count": 893, "friends_count": 369, "statues_count": 12299 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pearl River, LA", "id": "4a7b876876f80502", "name": "Pearl River", "place_type": "city", "bounding_box": rectangle("-89.771127,30.351691 -89.736169,30.391999") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22103, "countyName": "St. Tammany", "cityID": 2259445, "cityName": "Pearl River" } }
+{ "create_at": datetime("2016-01-05T10:00:08.000Z"), "id": 684434218277933056, "text": "Supplemental Health Care #Healthcare #Job: Travel RNs needed - ICU / CVOR (#Salem, MA) https://t.co/Qdf2Kiazq4 #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.8952337,42.5235097"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "Job", "Salem", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 20831558, "name": "TMJ- BOS Health Jobs", "screen_name": "tmj_bos_health", "lang": "en", "location": "Boston, MA", "create_at": date("2009-02-13"), "description": "Follow this account for geo-targeted Healthcare job tweets in Boston, MA. Need help? Tweet us at @CareerArc!", "followers_count": 625, "friends_count": 1212, "statues_count": 1167 }, "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-01-05T10:00:08.000Z"), "id": 684434218303139840, "text": "shit happens to me sometimes JJ �������� https://t.co/qimozR5Lgv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3074448601, "name": "Yung DC", "screen_name": "theboi_yung", "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 im also a Cavs fan", "followers_count": 284, "friends_count": 517, "statues_count": 7345 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:08.000Z"), "id": 684434218341003265, "text": "Dawg why were me and my boy @TrillyOcean so in sync last night at game night ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 438056199 }}, "user": { "id": 459921576, "name": "Bogart Lagunes", "screen_name": "ChampagneBogey", "lang": "en", "location": "Corpus Christi, Tx", "create_at": date("2012-01-09"), "description": "Dicks in our hands", "followers_count": 556, "friends_count": 542, "statues_count": 8648 }, "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-05T10:00:08.000Z"), "id": 684434218370207744, "text": "I told my mama and uncle about hot boxing my car with t", "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": "null", "create_at": date("2014-03-02"), "description": "If it don't make dollars then it don't make sense ...... #TrueStory", "followers_count": 1262, "friends_count": 772, "statues_count": 51196 }, "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-05T10:00:08.000Z"), "id": 684434218424745984, "text": "See a virtual tour of our listing on 422 S Avon Street #Gastonia #NC https://t.co/bQRuyTacht #realestate https://t.co/KpHGV0Khcj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.173264,35.256904"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Gastonia", "NC", "realestate" }}, "user": { "id": 2445654229, "name": "RE/MAX Crossroads", "screen_name": "lincolntonremax", "lang": "en", "location": "Lincolnton NC", "create_at": date("2014-04-15"), "description": "null", "followers_count": 71, "friends_count": 43, "statues_count": 6129 }, "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-05T10:00:08.000Z"), "id": 684434218693206016, "text": "\"Is It Not From The Mouth Of The Most High..?\" Lamentations 3:38", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.3196699,34.26005071"), "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": 393, "friends_count": 347, "statues_count": 61802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:08.000Z"), "id": 684434218798153728, "text": "I'm at Confluence in Pittsburgh, PA https://t.co/JF12KaflbR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.99766531,40.4492004"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2312681299, "name": "timothy", "screen_name": "tturner122", "lang": "en", "location": "home", "create_at": date("2014-01-26"), "description": "IT Helpdesk, Servers, Network Technician", "followers_count": 18, "friends_count": 70, "statues_count": 577 }, "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-05T10:00:08.000Z"), "id": 684434218827538432, "text": "@Dterrible07 understandable.", "in_reply_to_status": 684433951830728705, "in_reply_to_user": 358867663, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 358867663 }}, "user": { "id": 301153388, "name": "bryce montes", "screen_name": "bryce_montes2", "lang": "en", "location": "Columbus, OH", "create_at": date("2011-05-18"), "description": "Future stay at home father of two.", "followers_count": 759, "friends_count": 525, "statues_count": 16062 }, "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-01-05T10:00:08.000Z"), "id": 684434218986934272, "text": "Trade show trade show trade show trade show trade show trade show @ Orange County Convention Center https://t.co/xv2bB6Fnj8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4667928,28.42575973"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 314989973, "name": "Ben Reaves", "screen_name": "reavesben", "lang": "en", "location": "Statesboro, GA", "create_at": date("2011-06-10"), "description": "Unworthy. // Guitarist, keyboardist, audio engineer, and producer. // photo:@livefreekennyg // http://soundcloud.com/benreaves", "followers_count": 329, "friends_count": 268, "statues_count": 11915 }, "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-01-05T10:00:08.000Z"), "id": 684434219146186752, "text": "I miss gospel choir. https://t.co/JUrlrd0dZj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2785109970, "name": "Michelle", "screen_name": "michellemarket", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-09-01"), "description": "in case i don't see you, good afternoon, good evening, and good night.", "followers_count": 114, "friends_count": 192, "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-05T10:00:08.000Z"), "id": 684434219171508224, "text": "Anyone want a joke type in @NRA and @POTUS read the replies and I bet you'll laugh!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21829541, 1536791610 }}, "user": { "id": 295524093, "name": "Jackal Queen of Rust", "screen_name": "anfeline", "lang": "en", "location": "west michigain ", "create_at": date("2011-05-08"), "description": "somewhere in my life I became a trans jackal, with a bad habbit for poor choices, beer, and beater cars, and general ghetto shit.", "followers_count": 73, "friends_count": 165, "statues_count": 2348 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Haven, MI", "id": "00f907000c450422", "name": "Grand Haven", "place_type": "city", "bounding_box": rectangle("-86.251592,42.942868 -86.154898,43.076904") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26139, "countyName": "Ottawa", "cityID": 2633340, "cityName": "Grand Haven" } }
+{ "create_at": datetime("2016-01-05T10:00:08.000Z"), "id": 684434219335028736, "text": "It's the new wave.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3290753787, "name": "Kenny K", "screen_name": "KDK_supreme", "lang": "en", "location": "null", "create_at": date("2015-05-19"), "description": "VA || 757", "followers_count": 176, "friends_count": 238, "statues_count": 799 }, "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-05T10:00:08.000Z"), "id": 684434219364446208, "text": "Totally agree, plus whiskey, beer, wine, cheese and good crusty bread. @ Monticello Apartments https://t.co/TvDqozlADz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.25895894,38.64322949"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 49991692, "name": "Dr Mick Maurer", "screen_name": "WoodhullIrish", "lang": "en", "location": "St Louis, MO; USA", "create_at": date("2009-06-23"), "description": "DMin, MTS, MA, now1st yr F'15-S'16 MDiv/Anglican Studies at Eden Theological Seminary. Disabled Army Vet.Trinity Episcopal in CWE St Louis, Mo.", "followers_count": 470, "friends_count": 1677, "statues_count": 4877 }, "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.320464,38.533149 -90.175132,38.774349") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29510, "countyName": "St. Louis", "cityID": 2965000, "cityName": "St. Louis" } }
+{ "create_at": datetime("2016-01-05T10:00:08.000Z"), "id": 684434219574161408, "text": "@mannythedrummer HAPPY BIRTHDAY MANNY!!!! LUV YOU!", "in_reply_to_status": -1, "in_reply_to_user": 26572044, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 26572044 }}, "user": { "id": 2808678007, "name": "Gigi Justine", "screen_name": "UchihaGigi", "lang": "en", "location": "null", "create_at": date("2014-09-13"), "description": "im just a short, music & anime loving, weird, crazy, emotional, broken ,but healing, learning home cooking, guitar playing, madly in love with my bf teenage gal", "followers_count": 175, "friends_count": 300, "statues_count": 2000 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bogota, NJ", "id": "e363ac0012f77f07", "name": "Bogota", "place_type": "city", "bounding_box": rectangle("-74.039549,40.865727 -74.020909,40.884448") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3406490, "cityName": "Bogota" } }
+{ "create_at": datetime("2016-01-05T10:00:08.000Z"), "id": 684434219897065472, "text": "@LifeI_ButADream heat https://t.co/pDdEqAFlHj", "in_reply_to_status": -1, "in_reply_to_user": 1316391158, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1316391158 }}, "user": { "id": 90680228, "name": "uncle bitches", "screen_name": "BenjaminBitches", "lang": "en", "location": "Cincinnat OHIO", "create_at": date("2009-11-17"), "description": "Contact me at EZduzit513@Gmail.com #GKFAM #DeadSociety #TheU", "followers_count": 1302, "friends_count": 2427, "statues_count": 26701 }, "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-05T10:00:08.000Z"), "id": 684434219917905921, "text": "Partly cloudy this afternoon, high 48 (9 C). Low 40 (4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712452, "name": "FortWorthWX", "screen_name": "FortWorthWX", "lang": "en", "location": "Fort Worth, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Fort Worth, Texas", "followers_count": 218, "friends_count": 93, "statues_count": 8183 }, "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-05T10:00:08.000Z"), "id": 684434220123578368, "text": "\" are you ready to go trick-or-treating now \" - one of the kids sleep taking in my class", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1388074004, "name": "Holley❥", "screen_name": "Holley_Banks", "lang": "en", "location": "null", "create_at": date("2013-04-28"), "description": "• MSG •", "followers_count": 293, "friends_count": 322, "statues_count": 11022 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grovetown, GA", "id": "00fd461f4df791dc", "name": "Grovetown", "place_type": "city", "bounding_box": rectangle("-82.245686,33.422339 -82.169188,33.519205") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13073, "countyName": "Columbia", "cityID": 1335716, "cityName": "Grovetown" } }
+{ "create_at": datetime("2016-01-05T10:00:08.000Z"), "id": 684434220261847041, "text": "Everyone's like super punchy and excited and talkative and I'm sitting here like https://t.co/E97CuOLQLm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2518659265, "name": "shadont", "screen_name": "chboishado", "lang": "en", "location": "the abyss", "create_at": date("2014-05-23"), "description": "calling all goddesses", "followers_count": 21, "friends_count": 28, "statues_count": 2960 }, "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-01-05T10:00:08.000Z"), "id": 684434220345774080, "text": "Hawaii Pacific Health #Marketing #Job: Manager - Marketing (#Honolulu, HI) https://t.co/yb0KMxJxks #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8480364,21.3136151"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Marketing", "Job", "Honolulu", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 23493115, "name": "TMJ-HON Advert Jobs", "screen_name": "tmj_hon_adv", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Marketing/Ad/PR job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 290, "friends_count": 193, "statues_count": 9 }, "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-05T10:00:08.000Z"), "id": 684434220425588736, "text": "Ms. Nolan has the damn heat on power blast", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2333791734, "name": "Ethan Hutsell", "screen_name": "HutsellEthan", "lang": "en", "location": "Lewisville, TX", "create_at": date("2014-02-08"), "description": "nghs/cec 2016 , Physically in Georgia, Mentally in Texas", "followers_count": 3488, "friends_count": 457, "statues_count": 21062 }, "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-05T10:00:09.000Z"), "id": 684434220958101504, "text": "Stay in your lane.....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 245573654, "name": "Doe", "screen_name": "YG_Lando", "lang": "en", "location": "Shreveport/Monroe ", "create_at": date("2011-01-31"), "description": "University of Louisiana Monroe", "followers_count": 1594, "friends_count": 1393, "statues_count": 27710 }, "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-05T10:00:09.000Z"), "id": 684434221130121218, "text": "https://t.co/mh8V2Ksc5a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1104661386, "name": "2️⃣0️⃣1️⃣6️⃣", "screen_name": "Gerbygerbz", "lang": "en", "location": "halloweentown", "create_at": date("2013-01-19"), "description": "life has a høpeful undertone", "followers_count": 363, "friends_count": 322, "statues_count": 18686 }, "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-01-05T10:00:09.000Z"), "id": 684434221172142084, "text": "Wind 3.0 mph WNW. Barometer 30.494 in, Falling. Temperature 26.3 °F. Rain today 0.00 in. Humidity 35%", "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": 3393 }, "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-05T10:00:09.000Z"), "id": 684434221318975489, "text": "Scared but excited!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 95673733, "name": "Gigi", "screen_name": "_Giki_", "lang": "en", "location": "New York, USA", "create_at": date("2009-12-09"), "description": "Games, coffee, art & design #ESO #FFXIV #CamelotUnchained http://gikii.imgur.com/all/", "followers_count": 179, "friends_count": 206, "statues_count": 3959 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lockport, NY", "id": "45009687ba062971", "name": "Lockport", "place_type": "city", "bounding_box": rectangle("-78.744599,43.138056 -78.618744,43.205149") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36063, "countyName": "Niagara", "cityID": 3643082, "cityName": "Lockport" } }
+{ "create_at": datetime("2016-01-05T10:00:09.000Z"), "id": 684434221402841090, "text": "@HardWorkinNELL duh Nell I know", "in_reply_to_status": 684434182169341952, "in_reply_to_user": 143235865, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 143235865 }}, "user": { "id": 2184781852, "name": "Pretty Kee", "screen_name": "Keesoprettyyy", "lang": "en", "location": "null", "create_at": date("2013-11-17"), "description": "Parts of my are in Heaven 10.13.15 Remy you changed my life forever G4ever", "followers_count": 348, "friends_count": 339, "statues_count": 14882 }, "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.022437") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2407125, "cityName": "Bethesda" } }
+{ "create_at": datetime("2016-01-05T10:00:09.000Z"), "id": 684434221469974528, "text": "@ab_morales530 @bricen_thorn @BricenThorn nooooo ����", "in_reply_to_status": 684433862164754433, "in_reply_to_user": 794578680, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 794578680, 926840208, 396613506 }}, "user": { "id": 64852391, "name": "Johneis Winston", "screen_name": "JohnnyBoy_38", "lang": "en", "location": "null", "create_at": date("2009-08-11"), "description": "John Hagelberg -EBHS RIP Dor❤️ WARNING: I tweet about sports.. a lot | Bucs | Gators | Warriors | Magic | Rays | Lightning |Orlando City. Snapchat : ayyyitsjohn", "followers_count": 648, "friends_count": 921, "statues_count": 17403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gibsonton, FL", "id": "00c23e64064f65f5", "name": "Gibsonton", "place_type": "city", "bounding_box": rectangle("-82.412765,27.777215 -82.326561,27.868938") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1225900, "cityName": "Gibsonton" } }
+{ "create_at": datetime("2016-01-05T10:00:09.000Z"), "id": 684434221579014145, "text": "����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1072564729, "name": "Alex N", "screen_name": "alex_norr18", "lang": "en", "location": "null", "create_at": date("2013-01-08"), "description": "2phoneshawty", "followers_count": 450, "friends_count": 362, "statues_count": 8663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Odenton, MD", "id": "3795cbef20412c50", "name": "Odenton", "place_type": "city", "bounding_box": rectangle("-76.734394,39.019327 -76.643899,39.103254") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24003, "countyName": "Anne Arundel", "cityID": 2458300, "cityName": "Odenton" } }
+{ "create_at": datetime("2016-01-05T10:00:09.000Z"), "id": 684434221641764865, "text": "If you're a #BusinessMgmt professional in #Orlando, FL, check out this #job: https://t.co/Kwgom7DSky #management #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3792365,28.5383355"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BusinessMgmt", "Orlando", "job", "management", "Hiring" }}, "user": { "id": 2228943546, "name": "Ghirardelli Careers", "screen_name": "ghirardellijobs", "lang": "en", "location": "null", "create_at": date("2013-12-03"), "description": "Ghirardelli invites you to explore our open #jobs and discover a place where your ideas come to life, as part of our delicious legacy.", "followers_count": 52, "friends_count": 1, "statues_count": 84 }, "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-05T10:00:09.000Z"), "id": 684434221650296832, "text": "When cuh look into the camera �� https://t.co/XI8nlXS1dP", "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": "Ivy", "screen_name": "xcallmepoison", "lang": "en", "location": "256 ✈️ 404 Spelman College", "create_at": date("2011-05-26"), "description": "18. my DMs are closed.", "followers_count": 2466, "friends_count": 2381, "statues_count": 17158 }, "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-01-05T10:00:09.000Z"), "id": 684434221910237184, "text": "Wind 2.3 mph NE. Barometer 30.363 in, Falling slowly. Temperature 51.2 °F. Rain today 0.00in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.11555556,29.6725"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25210383, "name": "New Braunfels WX", "screen_name": "jeffreycentexwx", "lang": "en", "location": "New Braunfels, TX", "create_at": date("2009-03-18"), "description": "Weather Station located on County Line Road in New Braunfels, TX.\r\n\r\nRealtime Weather @ http://www.nbtx-wx.com\r\nFollow me @ http://twitter.com/jeffreycentex", "followers_count": 379, "friends_count": 24, "statues_count": 160094 }, "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-01-05T10:00:09.000Z"), "id": 684434222010863616, "text": "@brittanydailey 30 year campaign to create mental illness and remove me from society at https://t.co/Kglv865MJc", "in_reply_to_status": 684434159016652801, "in_reply_to_user": 26902613, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 26902613 }}, "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": 2045, "friends_count": 2370, "statues_count": 804951 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:09.000Z"), "id": 684434222581338112, "text": "Partly cloudy this afternoon, high 48 (9 C). Low 39 (4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712424, "name": "DallasWX", "screen_name": "DallasWX", "lang": "en", "location": "Dallas, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Dallas, Texas", "followers_count": 194, "friends_count": 93, "statues_count": 8160 }, "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-05T10:00:09.000Z"), "id": 684434222702964741, "text": "Want to work in #WilkesBarre, PA? View our latest opening: https://t.co/BV7sJwH8t5 #Sales #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.8813075,41.2459149"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WilkesBarre", "Sales", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22023656, "name": "TMJ-PA Sales Jobs", "screen_name": "tmj_pa_sales", "lang": "en", "location": "Pennsylvania", "create_at": date("2009-02-26"), "description": "Follow this account for geo-targeted Sales job tweets in Pennsylvania Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 444, "friends_count": 310, "statues_count": 816 }, "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-01-05T10:00:09.000Z"), "id": 684434222963146753, "text": "Im 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": 475445797, "name": "ava rollins ♕", "screen_name": "_avarollins", "lang": "en", "location": "null", "create_at": date("2012-01-26"), "description": "@selena21100 // snapchat @ avajanelle", "followers_count": 553, "friends_count": 264, "statues_count": 11441 }, "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("2016-01-05T10:00:09.000Z"), "id": 684434223093043200, "text": "@matthewebel @Patreon My issue is my card expiration changed and there’s no manual way for the patron to have their new card charged now.", "in_reply_to_status": 684432108731953152, "in_reply_to_user": 3101811, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3101811, 1228325660 }}, "user": { "id": 16235784, "name": "Casidhe Farrell", "screen_name": "casidhe", "lang": "en", "location": "San Jose, CA", "create_at": date("2008-09-11"), "description": "Your favorite fun-loving, modern, geeky, irresistible vixen. For my adult twitter find me on @badcasidhe.", "followers_count": 1619, "friends_count": 965, "statues_count": 76196 }, "place": { "country": "United States", "country_code": "United States", "full_name": "City Center, Santa Clara", "id": "34e7bdc0eefdae37", "name": "City Center", "place_type": "neighborhood", "bounding_box": rectangle("-121.959618,37.352283 -121.942146,37.363612") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 669084, "cityName": "Santa Clara" } }
+{ "create_at": datetime("2016-01-05T10:00:09.000Z"), "id": 684434223307059200, "text": "@Sir_quay passed out on the scene", "in_reply_to_status": 684433713195827200, "in_reply_to_user": 2261829786, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2261829786 }}, "user": { "id": 53237338, "name": "Big Zo°", "screen_name": "_blackgatsby", "lang": "en", "location": "901°", "create_at": date("2009-07-02"), "description": "| Global Grind | #APSU18 |", "followers_count": 941, "friends_count": 780, "statues_count": 14972 }, "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-05T10:00:09.000Z"), "id": 684434223604736000, "text": "Keep your head up your almsot done with school", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2434649540, "name": "Aaron D'Allesandro™", "screen_name": "aaron_DONNY21", "lang": "en", "location": "Sartell", "create_at": date("2014-04-08"), "description": "SeaBee in the United States Navy \n \n R.I.P Taylor", "followers_count": 284, "friends_count": 232, "statues_count": 1131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sartell, MN", "id": "344aab839ec7e1a8", "name": "Sartell", "place_type": "city", "bounding_box": rectangle("-94.255707,45.585516 -94.168736,45.679451") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27145, "countyName": "Stearns", "cityID": 2758612, "cityName": "Sartell" } }
+{ "create_at": datetime("2016-01-05T10:00:09.000Z"), "id": 684434223759929344, "text": "@iChrissyLTE ����������", "in_reply_to_status": 684433944947769344, "in_reply_to_user": 277487036, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 277487036 }}, "user": { "id": 319283425, "name": "ThatGuy_Dash", "screen_name": "exotic_heat", "lang": "en", "location": "The Moon", "create_at": date("2011-06-17"), "description": "Welcome To The World Of Dash... I Can Only Be Myself...New Orleans Made!! Just A Regular Guy Snapchat:prince.dash Instagram: pulseof_heat", "followers_count": 449, "friends_count": 630, "statues_count": 12164 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:09.000Z"), "id": 684434223843901440, "text": "My papa said you don't need no nigga if he can't do what I do 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": 3306344368, "name": "9.26♎️", "screen_name": "derricka______", "lang": "en", "location": "null", "create_at": date("2015-06-01"), "description": "H A T E D B Y M A N Y ™", "followers_count": 534, "friends_count": 424, "statues_count": 4630 }, "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-05T10:00:09.000Z"), "id": 684434224615714818, "text": "The Twitter thing can't be real", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 94467597, "name": "Tyler Felix", "screen_name": "flexme4", "lang": "en", "location": "Foxboro", "create_at": date("2009-12-03"), "description": "the real Tyler Felix sometimes in life all you have to do is bunt on", "followers_count": 209, "friends_count": 455, "statues_count": 9740 }, "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-05T10:00:09.000Z"), "id": 684434224661815296, "text": "I can't stand you https://t.co/opuybTrD7j", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 419920271, "name": "savage ninz", "screen_name": "thatgirl_ninaa", "lang": "en", "location": "954/305", "create_at": date("2011-11-23"), "description": "dubstep & tequila", "followers_count": 3264, "friends_count": 791, "statues_count": 133855 }, "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-01-05T10:00:10.000Z"), "id": 684434224724615168, "text": "@SamBarkersn I like what I see here Sam ��", "in_reply_to_status": 684433266770710529, "in_reply_to_user": 2538112338, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2538112338 }}, "user": { "id": 771662623, "name": "Skipopotamus", "screen_name": "buzzdliteyear", "lang": "en", "location": "Colorado, USA", "create_at": date("2012-08-21"), "description": "CU Boulder | soundcloud:buzzdliteyear| Illuminator 2015|", "followers_count": 582, "friends_count": 706, "statues_count": 7648 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Castle Rock, CO", "id": "0fd0097acd635907", "name": "Castle Rock", "place_type": "city", "bounding_box": rectangle("-104.92104,39.322269 -104.773048,39.451319") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8035, "countyName": "Douglas", "cityID": 812415, "cityName": "Castle Rock" } }
+{ "create_at": datetime("2016-01-05T10:00:10.000Z"), "id": 684434225005633536, "text": "Imma make a groupchat with this girl in it. Whos down to roast?������ https://t.co/CFhNHaFdn5", "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": 638, "friends_count": 778, "statues_count": 2978 }, "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-05T10:00:10.000Z"), "id": 684434225139970048, "text": "Good to be back @ Whataburger Restaurants https://t.co/SJS5l5mK4P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.3089371,30.4486904"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2360375850, "name": "michael", "screen_name": "MichaelBluto", "lang": "en", "location": "null", "create_at": date("2014-02-24"), "description": "757 ➡️ Florida State University", "followers_count": 348, "friends_count": 350, "statues_count": 4054 }, "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-05T10:00:10.000Z"), "id": 684434225144008705, "text": "#fakelivesmatter #crisisactors #nwo #falseflag #guncontrol #isis @ Inland Regional Center https://t.co/xKAOoDfury", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.27783651,34.07533894"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "hashtags": {{ "fakelivesmatter", "crisisactors", "nwo", "falseflag", "guncontrol", "isis" }}, "user": { "id": 424078033, "name": "The Freedom Fighter", "screen_name": "LibertaColumbia", "lang": "en", "location": "Muskogee, OK", "create_at": date("2011-11-29"), "description": "Read and contribute your thoughts to The Freedom Fighter Magazine. Let's pursue the truth together.", "followers_count": 63, "friends_count": 213, "statues_count": 484 }, "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-05T10:00:10.000Z"), "id": 684434225311789056, "text": "Partly cloudy this afternoon, high 56 (13 C). Low 43 (6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712472, "name": "AustinWX", "screen_name": "AustinWX", "lang": "en", "location": "Austin, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Austin, Texas", "followers_count": 1011, "friends_count": 1366, "statues_count": 8021 }, "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-05T10:00:10.000Z"), "id": 684434225353887744, "text": "Yall I'm so excited", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3098392752, "name": "Marissa Thompson", "screen_name": "marissat15", "lang": "en", "location": "instagram: _marissat_", "create_at": date("2015-03-19"), "description": "•I may be a great sinner, but I have a greater savior•", "followers_count": 528, "friends_count": 497, "statues_count": 4333 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winchester, KY", "id": "5de0c44aa57aa526", "name": "Winchester", "place_type": "city", "bounding_box": rectangle("-84.292805,37.94511 -84.118785,38.037115") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21049, "countyName": "Clark", "cityID": 2183676, "cityName": "Winchester" } }
+{ "create_at": datetime("2016-01-05T10:00:10.000Z"), "id": 684434225366495234, "text": "This would've been us on Mommas @itskarenxx @EstherDeanBitch https://t.co/x3gciDz952", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 413575729, 272119089 }}, "user": { "id": 182677618, "name": "Maybelline♥️", "screen_name": "MayFromTheBando", "lang": "en", "location": "Pakistan, NJ", "create_at": date("2010-08-24"), "description": "20 . Leo♌ . $ingle . iG: maybeits_maybelline", "followers_count": 543, "friends_count": 552, "statues_count": 17225 }, "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-05T10:00:10.000Z"), "id": 684434225370509314, "text": "@_mouseface_ I know :-/", "in_reply_to_status": 684434133351706624, "in_reply_to_user": 634132069, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 634132069 }}, "user": { "id": 2699541997, "name": "betha", "screen_name": "bethany_idk", "lang": "en", "location": "Houston, TX", "create_at": date("2014-08-01"), "description": "Grace is what matters. It keeps you from reaching for the gun too quickly.", "followers_count": 293, "friends_count": 321, "statues_count": 25420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:10.000Z"), "id": 684434225378902017, "text": "It's crazy no titles but your my baby my idol��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 571395907, "name": "jess", "screen_name": "Jessie_brooke14", "lang": "en", "location": "null", "create_at": date("2012-05-04"), "description": "courage above all things, is the first quality of a warrior", "followers_count": 747, "friends_count": 683, "statues_count": 26396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reading, OH", "id": "1f70979508a8aceb", "name": "Reading", "place_type": "city", "bounding_box": rectangle("-84.454726,39.204223 -84.407099,39.241837") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3965732, "cityName": "Reading" } }
+{ "create_at": datetime("2016-01-05T10:00:10.000Z"), "id": 684434225768996864, "text": "This #Manufacturing #job might be a great fit for you: Plant Manager - https://t.co/Km4TDxAUsU #Toledo, OH #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.555212,41.6639383"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Manufacturing", "job", "Toledo", "Hiring" }}, "user": { "id": 28447324, "name": "Cameron Craig Group", "screen_name": "cameroncraig", "lang": "en", "location": "USA", "create_at": date("2009-04-02"), "description": "Thousands of jobs, one click away. Find a job here today.", "followers_count": 25929, "friends_count": 25812, "statues_count": 28693 }, "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-05T10:00:10.000Z"), "id": 684434226482003968, "text": "If you're a #Retail professional in #CrestHill, IL, check out this #job: https://t.co/FT01RNpF5Z #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.128776,41.5526032"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "CrestHill", "job", "Hiring", "CareerArc" }}, "user": { "id": 59785313, "name": "TMJ-IL Retail Jobs", "screen_name": "tmj_il_retail", "lang": "en", "location": "Illinois", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in Illinois Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 462, "friends_count": 310, "statues_count": 1197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crest Hill, IL", "id": "493011471f6f2a96", "name": "Crest Hill", "place_type": "city", "bounding_box": rectangle("-88.147188,41.540723 -88.07334,41.596485") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1717458, "cityName": "Crest Hill" } }
+{ "create_at": datetime("2016-01-05T10:00:10.000Z"), "id": 684434226645630976, "text": "Want to work at Performance Team? We're #hiring in #Riverside, CA! Click for details: https://t.co/d19of7xHul #EngagePT #transportation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.3961564,33.9533487"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Riverside", "EngagePT", "transportation" }}, "user": { "id": 3271973436, "name": "PerformanceTeam Jobs", "screen_name": "PTGTCareers", "lang": "en", "location": "null", "create_at": date("2015-07-08"), "description": "PT is an industry leading 3PL with over 28 years of experience in warehousing, distribution, e-commerce fulfillment, transportation and supply chain logistics.", "followers_count": 23, "friends_count": 114, "statues_count": 30 }, "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-05T10:00:10.000Z"), "id": 684434226666577920, "text": "Ted Snyder is looking for a buyer on 422 S Avon Street #Gastonia #NC https://t.co/s07xyNhsYm #realestate https://t.co/bBCluPBL0u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.173264,35.256904"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Gastonia", "NC", "realestate" }}, "user": { "id": 2445654229, "name": "RE/MAX Crossroads", "screen_name": "lincolntonremax", "lang": "en", "location": "Lincolnton NC", "create_at": date("2014-04-15"), "description": "null", "followers_count": 71, "friends_count": 43, "statues_count": 6130 }, "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-05T10:00:10.000Z"), "id": 684434226754797569, "text": "���� https://t.co/M1Q0VzkTNZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2598872563, "name": "❣سيدة", "screen_name": "eazyn_", "lang": "en", "location": "9⃣1⃣0⃣", "create_at": date("2014-07-01"), "description": "carefree black girl☀️....addicted to trapping✌️✨ #NCAT19 #KanyeForPresident2020", "followers_count": 903, "friends_count": 716, "statues_count": 14620 }, "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-05T10:00:10.000Z"), "id": 684434226863714306, "text": "Dividend hoodies are the softest things ever next to my 2010-2012 On The Byas hoodies and the fleece blankets 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": 485142309, "name": "Lord D-Bear", "screen_name": "Darrian_Morgan", "lang": "en", "location": "In the Streets of Warren, OH", "create_at": date("2012-02-06"), "description": "W.G.H Raider '15 ⚽️ #12 @alena_012 YSU '19 Digital Media Major Aspiring Dj, Photo/Videographer Skateboarder and Gamer http://d-bear12.tumblr.com", "followers_count": 584, "friends_count": 290, "statues_count": 15813 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Girard, OH", "id": "b631c06684127309", "name": "Girard", "place_type": "city", "bounding_box": rectangle("-80.719166,41.13365 -80.670391,41.184764") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39155, "countyName": "Trumbull", "cityID": 3930198, "cityName": "Girard" } }
+{ "create_at": datetime("2016-01-05T10:00:10.000Z"), "id": 684434227006279680, "text": "Earlier today on @1stsports \n@theejamieson chats up #NBA news and notes.\nAudio at the link: https://t.co/GjI455hIiE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NBA" }}, "user_mentions": {{ 215363022, 23736539 }}, "user": { "id": 72649928, "name": "Steve Bortstein", "screen_name": "Fox1340AM", "lang": "en", "location": "Farmington, NM", "create_at": date("2009-09-08"), "description": "Host of First Sports on FOX AM1340... a man of many talents!", "followers_count": 607, "friends_count": 761, "statues_count": 14093 }, "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-01-05T10:00:10.000Z"), "id": 684434227136466944, "text": "Thank You @saspinall Inspired to be part of the @virtustream #SAPPosse Happy New Year!!", "in_reply_to_status": 684275238050377729, "in_reply_to_user": 39811765, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SAPPosse" }}, "user_mentions": {{ 39811765, 111702496 }}, "user": { "id": 287994254, "name": "Henrik Wagner", "screen_name": "henrikwagner73", "lang": "en", "location": "Phoenix, AZ via Stockholm", "create_at": date("2011-04-25"), "description": "Passionate about SAP & HANA / Love Innovation / Work @Virtustream / ☁️ Fan / Customer Engagement incl #SAPWeek / #EMCElect '14 & '15 / #SAPPosse / Tennis", "followers_count": 2765, "friends_count": 2493, "statues_count": 12638 }, "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-05T10:00:10.000Z"), "id": 684434227207614464, "text": "I've met the BEST humans on @Tinder. Half of my best friends are from the app. https://t.co/FgdVU6HOpW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 625298957 }}, "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": 9224, "friends_count": 159, "statues_count": 28853 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:10.000Z"), "id": 684434227325186049, "text": "#littlelucas #southSanfrancisco #pastrami #sandwich @ Little Lucca Specialty Sandwich Shop https://t.co/luyGpPaz3c", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.4305573,37.6509094"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "littlelucas", "southSanfrancisco", "pastrami", "sandwich" }}, "user": { "id": 30614165, "name": "Tony RuQusS Carbetta", "screen_name": "TonyRuQusS", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-04-11"), "description": "Producer, Recording Artist http://TonyRuQusSMusic.com", "followers_count": 401, "friends_count": 386, "statues_count": 7508 }, "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-05T10:00:10.000Z"), "id": 684434227337691137, "text": "We're #hiring! Click to apply: Advance Registered Nurse Practitioner - https://t.co/sMT2txz9DB #Nursing #Shelby, MT #Veterans #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.859769,48.506945"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Nursing", "Shelby", "Veterans", "Job", "Jobs" }}, "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": 322, "friends_count": 292, "statues_count": 392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shelby, MT", "id": "b5e1f550eca64caf", "name": "Shelby", "place_type": "city", "bounding_box": rectangle("-111.899639,48.495408 -111.839106,48.525211") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30101, "countyName": "Toole", "cityID": 3067450, "cityName": "Shelby" } }
+{ "create_at": datetime("2016-01-05T10:00:10.000Z"), "id": 684434227580944384, "text": "\"Usually judges Retire at like 50 and die at like 52\" -@ornett_juliana", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-116.959144,32.796674"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1171083109 }}, "user": { "id": 939846996, "name": "breneneim", "screen_name": "BrendenBarta", "lang": "en", "location": "null", "create_at": date("2012-11-10"), "description": "El Cap", "followers_count": 184, "friends_count": 170, "statues_count": 380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior Court of CA - El Cajon", "id": "07d9db678f084003", "name": "Superior Court of CA - El Cajon", "place_type": "poi", "bounding_box": rectangle("-116.95914409999999,32.7966739 -116.959144,32.796674") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 621712, "cityName": "El Cajon" } }
+{ "create_at": datetime("2016-01-05T10:00:10.000Z"), "id": 684434227635421184, "text": "@HCrewzy88 @hunterj_1417 @ducky2k thanks to that one other person who actually likes me", "in_reply_to_status": 684434096613797889, "in_reply_to_user": 528970532, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 528970532, 277280501, 577049941 }}, "user": { "id": 2746838819, "name": "Troy Walker", "screen_name": "twalk_05", "lang": "en", "location": "null", "create_at": date("2014-08-18"), "description": "Averett University business major OlllllllO Nothing beats flyin down a back road, with some buddies from back home.", "followers_count": 187, "friends_count": 223, "statues_count": 6089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Danville, VA", "id": "a58a872123811d63", "name": "Danville", "place_type": "city", "bounding_box": rectangle("-79.519308,36.540813 -79.317136,36.673579") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51590, "countyName": "Danville", "cityID": 5121344, "cityName": "Danville" } }
+{ "create_at": datetime("2016-01-05T10:00:10.000Z"), "id": 684434227702665217, "text": "I'm at Casa Smith Our Little Bit Of Heaven in NJ https://t.co/mm4wQl9pb7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.22380201,40.14815945"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 299073418, "name": "SuzAnne Smith", "screen_name": "Meanie2011", "lang": "en", "location": "Howell, NJ", "create_at": date("2011-05-15"), "description": "Just saying it like it is....", "followers_count": 86, "friends_count": 370, "statues_count": 1669 }, "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-05T10:00:10.000Z"), "id": 684434227815968768, "text": "@luminous_aura esa serie me da vida y simultáneamente me la quita", "in_reply_to_status": 684420538068905984, "in_reply_to_user": 40045920, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 40045920 }}, "user": { "id": 290180700, "name": "Mer", "screen_name": "jeanne_maarie", "lang": "en", "location": "P.R.", "create_at": date("2011-04-29"), "description": "Crecí entre el verde límite y la distancia azul. Siempre, siempre mirando el horizonte • •phhhoto: jeannemariee", "followers_count": 244, "friends_count": 560, "statues_count": 12065 }, "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-05T10:00:10.000Z"), "id": 684434227945967616, "text": "@RepMikeRogers 2/2 any replacement plan for the ACA or gun safety in 7 yrs????", "in_reply_to_status": -1, "in_reply_to_user": 156703580, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 156703580 }}, "user": { "id": 392159449, "name": "U Lar", "screen_name": "LarryMaskell", "lang": "en", "location": "baltimore", "create_at": date("2011-10-16"), "description": "69 yrs old, last of the '60 LIBERALS, veteran of USMC 1966-1970, Viet vet, 1000% Obama supporter, FAUX hater, Orioles and Ravens fan", "followers_count": 73, "friends_count": 147, "statues_count": 9187 }, "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-05T10:00:10.000Z"), "id": 684434228310896641, "text": "@Kelly_McCool ❤️ https://t.co/07uPEVJ2WP", "in_reply_to_status": -1, "in_reply_to_user": 340961505, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 340961505 }}, "user": { "id": 67709535, "name": "GMONEY™", "screen_name": "elr3ygil", "lang": "en", "location": "New Jersey", "create_at": date("2009-08-21"), "description": "#SetonHall Bass/Vocals for @the_rumples #taken by the wonderful, beautiful, and amazing @kelly_mccool", "followers_count": 1321, "friends_count": 928, "statues_count": 12125 }, "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-01-05T10:00:10.000Z"), "id": 684434228327497728, "text": "Last thing we need is 10,000 characters about how men need to take women on $200 dates", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 476420777, "name": "Tre", "screen_name": "Tre_Potts", "lang": "en", "location": "Maryland✈️Schertz✈️The Army", "create_at": date("2012-01-27"), "description": "22, #Tamucc16, God | US Army future officer | LSU, UT & Saints | Spurs | tryna make it everyday", "followers_count": 1553, "friends_count": 1894, "statues_count": 99689 }, "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-01-05T10:00:10.000Z"), "id": 684434228394737667, "text": "I have to much stress 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": 2806205632, "name": "K", "screen_name": "Kebannnn", "lang": "en", "location": "null", "create_at": date("2014-10-04"), "description": "@haley_johnstone", "followers_count": 337, "friends_count": 250, "statues_count": 3871 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westchase, FL", "id": "44c12398792e903e", "name": "Westchase", "place_type": "city", "bounding_box": rectangle("-82.648577,28.038662 -82.578263,28.08544") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1276062, "cityName": "Westchase" } }
+{ "create_at": datetime("2016-01-05T10:00:10.000Z"), "id": 684434228432515077, "text": "Wind 3 mph NE. Barometer 30.18 in, Falling. Temperature 70.2 °F. Rain today 0.00 in. Humidity 28%", "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": 51, "friends_count": 70, "statues_count": 24753 }, "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-05T10:00:10.000Z"), "id": 684434228830797824, "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": 3012630889, "name": "Ang(:", "screen_name": "l0l4Real", "lang": "en", "location": "null", "create_at": date("2015-02-07"), "description": "you thought I was coal my friend I'm gold can't ya tell?", "followers_count": 352, "friends_count": 733, "statues_count": 299 }, "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-05T10:00:10.000Z"), "id": 684434228868677632, "text": "@carmelle_reyna @_deanosaur_ it's an honor i love it����", "in_reply_to_status": 684433957283217408, "in_reply_to_user": 3789683656, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3789683656, 93325527 }}, "user": { "id": 353392657, "name": "Jacob Wade", "screen_name": "jac0bwade", "lang": "en", "location": "NYC", "create_at": date("2011-08-11"), "description": "SUNY Purchase Class of 2019 instagram/snapchat: jac0bwade", "followers_count": 465, "friends_count": 624, "statues_count": 5292 }, "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-05T10:00:11.000Z"), "id": 684434229015347201, "text": "Partly cloudy this afternoon, high 55 (13 C). Low 44 (7 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712388, "name": "HoustonWX", "screen_name": "HoustonWX", "lang": "en", "location": "Houston, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Houston, Texas", "followers_count": 1816, "friends_count": 77, "statues_count": 8030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:11.000Z"), "id": 684434229229432832, "text": "give me ape tit for 200.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3347941228, "name": "Erika", "screen_name": "sad_eyezzzz", "lang": "en", "location": "Columbus, OH", "create_at": date("2015-06-27"), "description": "null", "followers_count": 61, "friends_count": 51, "statues_count": 915 }, "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-05T10:00:11.000Z"), "id": 684434229233496065, "text": "#PolandIsReadyForRevivalTour 32", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "PolandIsReadyForRevivalTour" }}, "user": { "id": 1691377656, "name": "Justin my Angel", "screen_name": "AngelaDziedzic1", "lang": "pl", "location": "Chicago, IL", "create_at": date("2013-08-22"), "description": "I fell in love with a 15 year old guy who dreams fulfilled now I love 21-year-old man,who helps fulfill the dreams of other!18/11/15 Thank You Justin PotterHead", "followers_count": 325, "friends_count": 808, "statues_count": 2602 }, "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-01-05T10:00:11.000Z"), "id": 684434229350903808, "text": "Menorah Park Center for Senior L... #Nursing #Job: Resident Associate (#Beachwood, OH) https://t.co/9EY5F7XbRC #STNA #training #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4899166,41.5011199"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "Job", "Beachwood", "STNA", "training", "Jobs", "Hiring" }}, "user": { "id": 214122801, "name": "MenorahParkJobs", "screen_name": "MenorahParkJobs", "lang": "en", "location": "Beachwood, Ohio", "create_at": date("2010-11-10"), "description": "Menorah Park Center for Senior Living is one of the largest senior living communities in the state of Ohio.", "followers_count": 133, "friends_count": 82, "statues_count": 1385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beachwood, OH", "id": "fa5bd676191716ee", "name": "Beachwood", "place_type": "city", "bounding_box": rectangle("-81.532635,41.449669 -81.48037,41.501448") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3904500, "cityName": "Beachwood" } }
+{ "create_at": datetime("2016-01-05T10:00:11.000Z"), "id": 684434229573189632, "text": "Gods 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": 3739444393, "name": "Lex.", "screen_name": "naija_baddie", "lang": "en", "location": "Norman, OK", "create_at": date("2015-09-30"), "description": "ou'19 ❤️ I like fried shrimp", "followers_count": 84, "friends_count": 84, "statues_count": 280 }, "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-05T10:00:11.000Z"), "id": 684434229661401088, "text": ".....Sophia got $600 for losing a tooth........ I was happy when I got $20 to go to the damn movies wth", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25719172, "name": "Melissa Pike", "screen_name": "MelissaaPike", "lang": "en", "location": "Long Island l WVU l ΔΓ", "create_at": date("2009-03-21"), "description": "Don't let your mind keep you from having a good time", "followers_count": 609, "friends_count": 694, "statues_count": 24299 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merrick, NY", "id": "65340fdf9175b0ab", "name": "Merrick", "place_type": "city", "bounding_box": rectangle("-73.576227,40.624816 -73.529515,40.680072") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3646668, "cityName": "Merrick" } }
+{ "create_at": datetime("2016-01-05T10:00:11.000Z"), "id": 684434229996879872, "text": "Want to work at Sterling Jewelers? We're #hiring in #Richmond, VA! Click for details: https://t.co/Tffz9qJSMO #Retail #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.6153498,37.6526027"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Richmond", "Retail", "Job", "Jobs" }}, "user": { "id": 3248848340, "name": "Jobs at Jared", "screen_name": "JobsAtJared", "lang": "en", "location": "null", "create_at": date("2015-06-18"), "description": "Your Future Starts Here.", "followers_count": 13, "friends_count": 1, "statues_count": 504 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Short Pump, VA", "id": "018325d17666f4e3", "name": "Short Pump", "place_type": "city", "bounding_box": rectangle("-77.65493,37.624458 -77.582259,37.689361") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51087, "countyName": "Henrico", "cityID": 5172272, "cityName": "Short Pump" } }
+{ "create_at": datetime("2016-01-05T10:00:11.000Z"), "id": 684434230076674048, "text": "What do you think of Martha?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30892619, "name": "Darth Thea ➡️NYE 360", "screen_name": "thetheaster", "lang": "en", "location": "New York", "create_at": date("2009-04-13"), "description": "IONA'16 #IWL #42 #BASSHEAD I'll see you on the dark side of the moon.", "followers_count": 589, "friends_count": 617, "statues_count": 13154 }, "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-01-05T10:00:11.000Z"), "id": 684434230269497344, "text": "Packing sucks", "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": "unknown", "create_at": date("2011-07-13"), "description": "Let go and let God | #OOB | #FARWG | Race; Oriental | lifes a salad, toss it | Sophomore at Oklahoma State |", "followers_count": 2077, "friends_count": 980, "statues_count": 50203 }, "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-05T10:00:11.000Z"), "id": 684434230294786048, "text": "Seashore Surgic is hiring! OR Surgical Tec #jobs in BRICK Apply today https://t.co/CKzYqPRRZZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.109236,40.045766"), "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": 169, "friends_count": 42, "statues_count": 72159 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedarwood Park, NJ", "id": "01d06fb5242190dd", "name": "Cedarwood Park", "place_type": "city", "bounding_box": rectangle("-74.160521,40.018539 -74.104529,40.060658") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean", "cityID": 3473110, "cityName": "Toms River" } }
+{ "create_at": datetime("2016-01-05T10:00:11.000Z"), "id": 684434230311452672, "text": "@CMcCurdy3 @bethvalasek4 boat oar to the face??? https://t.co/CrRFhLdpEZ", "in_reply_to_status": 683888954043543553, "in_reply_to_user": 361469495, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 361469495, 247776631 }}, "user": { "id": 540337040, "name": "shelby blake", "screen_name": "shelby_blake8", "lang": "en", "location": "null", "create_at": date("2012-03-29"), "description": "snapchat: shelby.blake", "followers_count": 397, "friends_count": 480, "statues_count": 406 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Osage, IA", "id": "2bb77c8a5545ac3a", "name": "Osage", "place_type": "city", "bounding_box": rectangle("-92.827151,43.267784 -92.791243,43.295103") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19131, "countyName": "Mitchell", "cityID": 1959745, "cityName": "Osage" } }
+{ "create_at": datetime("2016-01-05T10:00:11.000Z"), "id": 684434230370123777, "text": "#Transportation #Job in #MAQUOKETA, IA: Parts Delivery at O'Reilly Auto Parts https://t.co/7WQLY7qwO0 #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.6788339,42.0694502"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Transportation", "Job", "MAQUOKETA", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 149711487, "name": "TMJ-IA Transport.", "screen_name": "tmj_IA_transp", "lang": "en", "location": "Indiana", "create_at": date("2010-05-29"), "description": "Follow this account for geo-targeted Transportation job tweets in Iowa Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 309, "friends_count": 282, "statues_count": 171 }, "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-01-05T10:00:11.000Z"), "id": 684434230479224834, "text": "@WOAPRadio @happy @Socialolio @vikingsrule185 @Alyssaa @basque \n#HaveABeautifulDay #Snow @WolfCreekSki2 #PagosaSprings #MostSnow #Colorado", "in_reply_to_status": 684426058913988608, "in_reply_to_user": 1101081968, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "HaveABeautifulDay", "Snow", "PagosaSprings", "MostSnow", "Colorado" }}, "user_mentions": {{ 1101081968, 491585749, 550343318, 2327149594, 38714287, 17708620, 71064211 }}, "user": { "id": 600658656, "name": "HXH@HollySquared", "screen_name": "HXHRadio", "lang": "en", "location": "null", "create_at": date("2012-06-05"), "description": "null", "followers_count": 876, "friends_count": 1442, "statues_count": 6644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pagosa Springs, CO", "id": "0179084714c39084", "name": "Pagosa Springs", "place_type": "city", "bounding_box": rectangle("-107.094306,37.254218 -106.996025,37.276903") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8007, "countyName": "Archuleta", "cityID": 856860, "cityName": "Pagosa Springs" } }
+{ "create_at": datetime("2016-01-05T10:00:11.000Z"), "id": 684434230642884608, "text": "I've been about me & I'll always be 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": 370431477, "name": "иєℓℓιє", "screen_name": "Nell_WoodNorman", "lang": "en", "location": "Atlanta, Georgia", "create_at": date("2011-09-08"), "description": "Criminal Justice Major CSU'17", "followers_count": 15859, "friends_count": 10922, "statues_count": 43544 }, "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-05T10:00:11.000Z"), "id": 684434230802120704, "text": "JUST ANNOUNCED! @omgRainMan at @themidchicago on Friday, Feb. 12th! https://t.co/9gTfx4ot4K #RainMan https://t.co/ZINyWE12eP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RainMan" }}, "user_mentions": {{ 243125110, 221796405 }}, "user": { "id": 221796405, "name": "theMID", "screen_name": "themidchicago", "lang": "en", "location": "Chicago", "create_at": date("2010-12-01"), "description": "306 N Halsted Chicago. 312.265.3990. tickets at http://www.clubtix.com", "followers_count": 14521, "friends_count": 736, "statues_count": 8977 }, "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-05T10:00:11.000Z"), "id": 684434230810525697, "text": "Will do https://t.co/Rd4ylfktsA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 57173351, "name": "Corduroy Handz", "screen_name": "HughHandz", "lang": "en", "location": "Goonies Never Say Die", "create_at": date("2009-07-15"), "description": "I Sell Dreams IG - HUGHHANDZ", "followers_count": 983, "friends_count": 99, "statues_count": 66910 }, "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-01-05T10:00:11.000Z"), "id": 684434230999265280, "text": "We're #hiring! Read about our latest #job opening here: Lpn Lvn (Every other weekend. Every other friday and... - https://t.co/FQ1uqQBMME", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.026088,38.9906657"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job" }}, "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": 314, "friends_count": 9, "statues_count": 2924 }, "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-05T10:00:11.000Z"), "id": 684434231137710082, "text": "@_JB35 et es, no exaggeration https://t.co/8WHdBkGWkP", "in_reply_to_status": 684433819185713152, "in_reply_to_user": 271254581, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user_mentions": {{ 271254581 }}, "user": { "id": 1051927436, "name": "robinquinanola", "screen_name": "robinquinanola", "lang": "en", "location": "null", "create_at": date("2012-12-31"), "description": "That's what I love, she got that Spanish blood.. I know y'all feeling me though, Mexican or Filipino", "followers_count": 453, "friends_count": 26, "statues_count": 24047 }, "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-05T10:00:11.000Z"), "id": 684434231267848192, "text": "Varsity with a great, gritty win over a very tough Moses Brown team last night. Back to 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": 789996846, "name": "CharihoBBall", "screen_name": "CharihoBBall", "lang": "en", "location": "Wood River JCT Rhode Island", "create_at": date("2012-08-29"), "description": "What are you doing RIGHT NOW in becoming a better basketball player? It's great to be a Charger. IG: Chariho_BBall", "followers_count": 678, "friends_count": 822, "statues_count": 3565 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pawcatuck, CT", "id": "c9cdbca03a00179b", "name": "Pawcatuck", "place_type": "city", "bounding_box": rectangle("-71.891122,41.335773 -71.830086,41.408715") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9011, "countyName": "New London", "cityID": 959140, "cityName": "Pawcatuck" } }
+{ "create_at": datetime("2016-01-05T10:00:11.000Z"), "id": 684434231632752641, "text": "Seashore Surgic is hiring! Sterile Process #jobs in BRICK Apply today https://t.co/B2gCuY1iOV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.109236,40.045766"), "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": 169, "friends_count": 42, "statues_count": 72160 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedarwood Park, NJ", "id": "01d06fb5242190dd", "name": "Cedarwood Park", "place_type": "city", "bounding_box": rectangle("-74.160521,40.018539 -74.104529,40.060658") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean", "cityID": 3473110, "cityName": "Toms River" } }
+{ "create_at": datetime("2016-01-05T10:00:11.000Z"), "id": 684434231645356032, "text": "I wonder if any of those murderers we call police served in the military.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 137204642, "name": "16-19", "screen_name": "AlexCles", "lang": "en", "location": "Student of the game ", "create_at": date("2010-04-25"), "description": "Stay the course, light the way.", "followers_count": 450, "friends_count": 68, "statues_count": 10551 }, "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-05T10:00:11.000Z"), "id": 684434231699832832, "text": "I'm at @HMartOfficial in Hartsdale, NY https://t.co/AVOvPwU3OO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.79004036,41.02956413"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 337774726 }}, "user": { "id": 74095751, "name": "Jerome Gentolia", "screen_name": "jeromegentolia", "lang": "en", "location": "New York, NY", "create_at": date("2009-09-13"), "description": "Computer Programmer, Web Developer, Web Marketing Consultant. Founder of Web Marketing Evanglists and Social Media Pundits.", "followers_count": 1234, "friends_count": 405, "statues_count": 80804 }, "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-01-05T10:00:11.000Z"), "id": 684434231771021313, "text": "My hitta , my hitta . https://t.co/0UiLflsoy9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 219557515, "name": "Drew", "screen_name": "TeamOtterPop", "lang": "en", "location": "U.S.A ", "create_at": date("2010-11-24"), "description": "R.I.P TORO", "followers_count": 218, "friends_count": 205, "statues_count": 6706 }, "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-05T10:00:11.000Z"), "id": 684434231771045888, "text": "Happy birthday! Hope your day is going absolutely wonderful! ☺️����@dgarza_25", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 290792955 }}, "user": { "id": 1634120364, "name": "katelyn.", "screen_name": "katelynmariebb", "lang": "en", "location": "null", "create_at": date("2013-07-30"), "description": "EHS | senior | 17. | everything happens for a reason.", "followers_count": 737, "friends_count": 579, "statues_count": 8397 }, "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-05T10:00:11.000Z"), "id": 684434231775358980, "text": "@jenphehey https://t.co/ETJrBlXfpz", "in_reply_to_status": 684433325000376321, "in_reply_to_user": 498500865, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 498500865 }}, "user": { "id": 498500865, "name": "Jenny Phelan", "screen_name": "jenphehey", "lang": "en", "location": "null", "create_at": date("2012-02-20"), "description": "We all have a fighter in us |Lipscomb University Volleyball| Nashville, TN", "followers_count": 526, "friends_count": 328, "statues_count": 3486 }, "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-05T10:00:11.000Z"), "id": 684434232010092544, "text": "NASCAR: Quality Assurance Tester (#Charlotte, NC) https://t.co/LhzmDdzFja #Marketing #QA #Agile #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.8431267,35.2270869"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "Charlotte", "Marketing", "QA", "Agile", "Job", "Jobs", "Hiring" }}, "user": { "id": 2935068602, "name": "NASCAR Jobs", "screen_name": "NASCARJobs", "lang": "en", "location": "null", "create_at": date("2014-12-18"), "description": "The National Association for Stock Car Auto Racing, Inc. (NASCAR) is the sanctioning body for one of North America's premier sports.", "followers_count": 1108, "friends_count": 14, "statues_count": 160 }, "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-05T10:00:11.000Z"), "id": 684434232098304000, "text": "I love Twitter https://t.co/Gva2azF1fX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 455978424, "name": "Mike", "screen_name": "MichaelSanchize", "lang": "en", "location": "Miami, FL", "create_at": date("2012-01-05"), "description": "UF Alum. Law School TBA. Miami Hurricane. Miami Dolphin. #Heat #Marlins #FlaPanthers #Everton. I'm just 'bout that action,boss. I tweet a lot about Alex Morgan", "followers_count": 1013, "friends_count": 966, "statues_count": 22919 }, "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-05T10:00:11.000Z"), "id": 684434232442105856, "text": "Cloudy this afternoon, high 54 (12 C). Low 44 (7 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575081, "name": "San Angelo Weather", "screen_name": "SanAngeloWX", "lang": "en", "location": "San Angelo, TX", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for San Angelo, TX", "followers_count": 1944, "friends_count": 92, "statues_count": 7920 }, "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-05T10:00:11.000Z"), "id": 684434232660353026, "text": "Growth Restaura is hiring! Line Cooks and #jobs in FLORHAM PARK Apply today https://t.co/KqOtDydO4L", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.401,40.774552"), "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": 169, "friends_count": 42, "statues_count": 72161 }, "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-01-05T10:00:11.000Z"), "id": 684434232781832192, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 484487002, "name": "j✨", "screen_name": "JadeChea", "lang": "en", "location": "Houston, TX", "create_at": date("2012-02-05"), "description": "big as the eiffel tower || IG- theyluvjade|| red|", "followers_count": 652, "friends_count": 503, "statues_count": 5101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:12.000Z"), "id": 684434233188724736, "text": "@FezAdree Thanks brother , rock on !", "in_reply_to_status": -1, "in_reply_to_user": 4280203574, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4280203574 }}, "user": { "id": 155703786, "name": "Sound Of Curves", "screen_name": "SoundOfCurves", "lang": "en", "location": "Texas, USA", "create_at": date("2010-06-14"), "description": "Alternative Rock by way of San Antonio ,TX @soundofroger @soundofleonel @joshsatx @AMontanoTeague. New Single Gone Gatsby out now on itunes!", "followers_count": 761, "friends_count": 1852, "statues_count": 642 }, "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.604641") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4833146, "cityName": "Helotes" } }
+{ "create_at": datetime("2016-01-05T10:00:12.000Z"), "id": 684434233264177152, "text": "See our latest #Lubbock, TX #job and click to apply: Registered Nurse (RN)-CVICU - https://t.co/psqTodhVra #cardiaccare #nurse #Nursing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-101.8919288,33.5881366"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Lubbock", "job", "cardiaccare", "nurse", "Nursing" }}, "user": { "id": 2940007148, "name": "UMC Nursing Jobs", "screen_name": "UMCNursingJobs", "lang": "en", "location": "null", "create_at": date("2014-12-22"), "description": "UMC Health System is the leader in comprehensive healthcare delivery in West Texas and Eastern New Mexico. Follow us to learn about our Nursing Jobs!", "followers_count": 90, "friends_count": 278, "statues_count": 76 }, "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-05T10:00:12.000Z"), "id": 684434233658482689, "text": "i strongly dislike dramatic pauses...especially when what the person is saying isnt deep at all", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 306448927, "name": "Uncle Obi", "screen_name": "Dr_Ezekwesili", "lang": "en", "location": "Nigeria✈️Miami✈️SA {UTSA'15}", "create_at": date("2011-05-27"), "description": "Future Oga at the Top", "followers_count": 1216, "friends_count": 857, "statues_count": 38064 }, "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-05T10:00:12.000Z"), "id": 684434233834733572, "text": "Cafasso's Fairw is hiring! Cashier #jobs in FORT LEE Apply today https://t.co/q6c4dhDmwB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.969654,40.848516"), "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": 169, "friends_count": 42, "statues_count": 72162 }, "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-01-05T10:00:12.000Z"), "id": 684434233868201984, "text": "I do know one day I'll be for sure ready and he will be too ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 164145046, "name": "hey bitches im", "screen_name": "LondyAutumn", "lang": "en", "location": "inthe90s", "create_at": date("2010-07-07"), "description": "londy baby.. please say the baby londyautumn@yahoo.com", "followers_count": 4225, "friends_count": 818, "statues_count": 101635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:12.000Z"), "id": 684434234027544576, "text": "Can you recommend anyone for this #job? Busser - https://t.co/zupJxmoZW2 #Framingham, MA #Labor #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.3908904,42.303146"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Framingham", "Labor", "Hiring", "CareerArc" }}, "user": { "id": 22733179, "name": "TMJ-BOS Labor Jobs", "screen_name": "tmj_bos_labor", "lang": "en", "location": "Boston, MA", "create_at": date("2009-03-03"), "description": "Follow this account for geo-targeted General Labor job tweets in Boston, MA. Need help? Tweet us at @CareerArc!", "followers_count": 317, "friends_count": 298, "statues_count": 36 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Framingham, MA", "id": "dbd1411661903bb0", "name": "Framingham", "place_type": "city", "bounding_box": rectangle("-71.496927,42.257263 -71.374193,42.352931") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2524960, "cityName": "Framingham" } }
+{ "create_at": datetime("2016-01-05T10:00:12.000Z"), "id": 684434234124009472, "text": "@MaxTollensJr @DocThompsonShow @POTUS // He is crying because he knows that he can't do anything but smoke and mirrors!! We want him crying!", "in_reply_to_status": 684432750095560704, "in_reply_to_user": 341815101, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 341815101, 267719395, 1536791610 }}, "user": { "id": 2171200586, "name": "Edwin Mobley", "screen_name": "wyomobe", "lang": "en", "location": "Cheyenne, WY", "create_at": date("2013-11-02"), "description": "Retired AF NCO. Conservative! Standing for strict adherence the U.S. Constitution. FLAT TAX!! Any thing libs like I AM AGAINST!! Pro-Israel, Anti-Islam, Pro-2A.", "followers_count": 2380, "friends_count": 2081, "statues_count": 32865 }, "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-05T10:00:12.000Z"), "id": 684434234342117376, "text": "second test tweet", "in_reply_to_status": 684434179853975552, "in_reply_to_user": 208572289, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 208572289, "name": "Zac Barnett", "screen_name": "ZacLBarnett", "lang": "en", "location": "Detroit - Columbus", "create_at": date("2010-10-27"), "description": "Student at The Ohio State University. Better free throw shooter than @DwightHoward. Joe Biden's best friend.", "followers_count": 256, "friends_count": 378, "statues_count": 12534 }, "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-05T10:00:12.000Z"), "id": 684434234426015748, "text": "Can you recommend anyone for this #job? Dishwasher - https://t.co/VALUalZbnx #McLean, VA #Hospitality #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.224982,38.923259"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "McLean", "Hospitality", "Hiring", "CareerArc" }}, "user": { "id": 99812697, "name": "TMJ-VAV HRTA Jobs", "screen_name": "tmj_VAV_HRTA", "lang": "en", "location": "Tysons, VA", "create_at": date("2009-12-27"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Tysons, VA. Need help? Tweet us at @CareerArc!", "followers_count": 295, "friends_count": 273, "statues_count": 162 }, "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-01-05T10:00:12.000Z"), "id": 684434234505850880, "text": "Bitches came over yeah we threw a party, I was just calling cause they were just leaving.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3326004867, "name": "AJ", "screen_name": "OBJumperr", "lang": "en", "location": "Champaign, IL", "create_at": date("2015-06-14"), "description": "RIP Uncle Josh, you were the realist. #IDontAnswerToFeminists.", "followers_count": 1057, "friends_count": 186, "statues_count": 664 }, "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-05T10:00:12.000Z"), "id": 684434234828800000, "text": "I just realized I turn 21 in less than a week omg GN!!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 286487594, "name": "Ryan Kiernan", "screen_name": "rxjk13", "lang": "en", "location": "Pennsylvania, USA", "create_at": date("2011-04-22"), "description": "Here you can find what I do on the daily such as stress about school and obsess over Taylor Swift. Everything is fine, I swear.", "followers_count": 607, "friends_count": 707, "statues_count": 27248 }, "place": { "country": "United States", "country_code": "United States", "full_name": "King of Prussia, PA", "id": "23b12afb6fe7507a", "name": "King of Prussia", "place_type": "city", "bounding_box": rectangle("-75.420016,40.071936 -75.343479,40.117633") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42091, "countyName": "Montgomery", "cityID": 4239736, "cityName": "King of Prussia" } }
+{ "create_at": datetime("2016-01-05T10:00:12.000Z"), "id": 684434234895761408, "text": "Waking up with strep ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 442548955, "name": "K Cole", "screen_name": "KendallCole1", "lang": "en", "location": "Anchorage, AK", "create_at": date("2011-12-20"), "description": "Owner of Sublime Esthetics • I specialize in eyelash extensions and full body waxing • Follow my business page and message me for an appointment :)", "followers_count": 500, "friends_count": 308, "statues_count": 11142 }, "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-05T10:00:12.000Z"), "id": 684434235101327360, "text": "I need coffee��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1005193716, "name": "january14th❤️", "screen_name": "adoremee_ayre", "lang": "en", "location": "null", "create_at": date("2012-12-11"), "description": "IG : Air.bearrr", "followers_count": 1553, "friends_count": 1843, "statues_count": 12035 }, "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-05T10:00:12.000Z"), "id": 684434235256471552, "text": "We're #hiring! Read about our latest #job opening here: Assistant Manager PT - https://t.co/VTVWZkH5N1 #LakeElsinore, CA #Retail #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.3272615,33.6680772"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "LakeElsinore", "Retail", "CareerArc" }}, "user": { "id": 59784845, "name": "TMJ-CA Retail Jobs", "screen_name": "tmj_ca_retail", "lang": "en", "location": "California", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 517, "friends_count": 316, "statues_count": 3400 }, "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-05T10:00:12.000Z"), "id": 684434235361378304, "text": "“@pepsilexxxi: I need a back massage, and a good vent session” YES ����", "in_reply_to_status": 684433917240188928, "in_reply_to_user": 842782579, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 842782579 }}, "user": { "id": 1560133435, "name": "Qυєєη Lαηα", "screen_name": "k_maelana", "lang": "en", "location": "LongLiveZo ☥ LongLiveKB ", "create_at": date("2013-07-01"), "description": "unbothered | 17", "followers_count": 1206, "friends_count": 570, "statues_count": 18731 }, "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-05T10:00:12.000Z"), "id": 684434235369754624, "text": "Mostly sunny this afternoon, high 52 (11 C). Low 40 (4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575147, "name": "Baton Rouge Weather", "screen_name": "BatonRougeWX", "lang": "en", "location": "Baton Rouge, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Baton Rouge, LA", "followers_count": 2875, "friends_count": 92, "statues_count": 7873 }, "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-05T10:00:12.000Z"), "id": 684434235499741185, "text": "Join the Sutter Coast Hospital, Crescent ... team! See our latest #Nursing #job opening here: https://t.co/YbHFuXlmeB #Brookings, OR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-124.2839819,42.0526114"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "Brookings" }}, "user": { "id": 66586174, "name": "TMJ-OR Nursing Jobs", "screen_name": "tmj_or_nursing", "lang": "en", "location": "Oregon", "create_at": date("2009-08-17"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Oregon Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 343, "friends_count": 298, "statues_count": 400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brookings, OR", "id": "5c1f798a253092fa", "name": "Brookings", "place_type": "city", "bounding_box": rectangle("-124.321768,42.042174 -124.254769,42.080967") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41015, "countyName": "Curry", "cityID": 4108650, "cityName": "Brookings" } }
+{ "create_at": datetime("2016-01-05T10:00:12.000Z"), "id": 684434235503947776, "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": 631371002, "name": "Ahmed Alsabban", "screen_name": "Ahmed_Alsbban", "lang": "en", "location": "USA ,OKC AND KSA ,Jeddah", "create_at": date("2012-07-09"), "description": "OSU student, majoring in Industrial Engineering ..Instgram :aalsabban90.. Alahli first ... #12 #okstate", "followers_count": 131, "friends_count": 197, "statues_count": 916 }, "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-05T10:00:12.000Z"), "id": 684434235663331328, "text": "No brainer! Our country does best under democratic leaders ship! VOTE BLUE! https://t.co/GYAtkmhd2e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.15108893,40.17934977"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 297191567, "name": "vooda", "screen_name": "vooda1", "lang": "en", "location": "null", "create_at": date("2011-05-11"), "description": "pro-choice crusader, equal rights for all, democrat, President Obama supporter, love to laugh, love my family. I have ZERO time for trolls...BLOCKED!", "followers_count": 2206, "friends_count": 2243, "statues_count": 35637 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Longmont, CO", "id": "2736a5db074e8201", "name": "Longmont", "place_type": "city", "bounding_box": rectangle("-105.178564,40.125504 -105.045869,40.206357") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8013, "countyName": "Boulder", "cityID": 845970, "cityName": "Longmont" } }
+{ "create_at": datetime("2016-01-05T10:00:12.000Z"), "id": 684434235860455424, "text": "Can you recommend anyone for this #job? Retail Store Shift Supervisor - https://t.co/gR0RFd7vBx #Danville, IN #Retail #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.5263879,39.7606013"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Danville", "Retail", "Hiring", "CareerArc" }}, "user": { "id": 28542508, "name": "TMJ-IND Retail Jobs", "screen_name": "tmj_ind_retail", "lang": "en", "location": "Indianapolis, IN", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Indianapolis, IN. Need help? Tweet us at @CareerArc!", "followers_count": 385, "friends_count": 310, "statues_count": 809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Danville, IN", "id": "e6db906395cb3381", "name": "Danville", "place_type": "city", "bounding_box": rectangle("-86.551319,39.750045 -86.457249,39.777285") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18063, "countyName": "Hendricks", "cityID": 1816804, "cityName": "Danville" } }
+{ "create_at": datetime("2016-01-05T10:00:12.000Z"), "id": 684434236598824964, "text": "Good day folks!! ������ https://t.co/UBfcPhxXXj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 42497648, "name": "Amy Williams", "screen_name": "amayzingamy", "lang": "en", "location": "Orlando Florida ", "create_at": date("2009-05-25"), "description": "Love life", "followers_count": 65, "friends_count": 167, "statues_count": 54 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Longwood, FL", "id": "f3c295d74e525431", "name": "Longwood", "place_type": "city", "bounding_box": rectangle("-81.383289,28.676878 -81.31933,28.721956") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12117, "countyName": "Seminole", "cityID": 1241250, "cityName": "Longwood" } }
+{ "create_at": datetime("2016-01-05T10:00:12.000Z"), "id": 684434236686745600, "text": "@pornlaw in AZ I can sell a gun to anyone with zero paperwork, but we also don't have a mass shooting issue.", "in_reply_to_status": 684426084503257088, "in_reply_to_user": 29149748, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 29149748 }}, "user": { "id": 268149034, "name": "T. Nikolas", "screen_name": "TerryNikolas", "lang": "en", "location": "Scottsdale, Arizona ", "create_at": date("2011-03-18"), "description": "Legends dont retire, they reinvent themselves.", "followers_count": 475, "friends_count": 138, "statues_count": 6912 }, "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-05T10:00:12.000Z"), "id": 684434236925853697, "text": "@desmondsaubeljr Guess what... https://t.co/XqW03gFH0f", "in_reply_to_status": -1, "in_reply_to_user": 1291820426, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1291820426 }}, "user": { "id": 626949993, "name": "the great gabsy", "screen_name": "TheWinningApple", "lang": "en", "location": "null", "create_at": date("2012-07-04"), "description": "I travel a lot | They're stripes, not bars.", "followers_count": 139, "friends_count": 139, "statues_count": 6347 }, "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-05T10:00:12.000Z"), "id": 684434237177507840, "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": 2312893993, "name": "sc:harielleb", "screen_name": "hermosahari_", "lang": "en", "location": "w/myself", "create_at": date("2014-01-26"), "description": "she used to be the sweetest girl..", "followers_count": 1365, "friends_count": 547, "statues_count": 55446 }, "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-05T10:00:12.000Z"), "id": 684434237190205440, "text": "Um. Same?\n https://t.co/PBND7XmBDB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1649274902, "name": "|-/Julie\\-|", "screen_name": "julie12490", "lang": "en", "location": "null", "create_at": date("2013-08-05"), "description": "18•Chicago•Spotify•SXU'19• TØP•Pansexual•Lumineers•Foster the People•Hozier•I have a playlist called Soundtrack of my life•AHS•♈•", "followers_count": 471, "friends_count": 378, "statues_count": 18374 }, "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-05T10:00:13.000Z"), "id": 684434237928259584, "text": "#Columbus, OH #SupplyChain #Job: Purchase Order Assistant at Express https://t.co/00m3ArZD4E #Jobs #Hiring #CareerArc", "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": {{ "Columbus", "SupplyChain", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 188185944, "name": "ColumbusOH S-Chain", "screen_name": "tmj_CHH_schn", "lang": "en", "location": "Columbus, OH", "create_at": date("2010-09-07"), "description": "Follow this account for geo-targeted Supply Chain job tweets in Columbus, OH from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 285, "friends_count": 264, "statues_count": 31 }, "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-05T10:00:13.000Z"), "id": 684434238012129281, "text": "#Retail #Job alert: Sales Associate | Sterling Jewelers | #Altoona, PA https://t.co/875lKc0RuK #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.4089381,40.4694508"), "retweet_count": 0, "lang": "nl", "is_retweet": false, "hashtags": {{ "Retail", "Job", "Altoona", "Jobs", "Hiring" }}, "user": { "id": 3248774184, "name": "Kay Jewelers", "screen_name": "JobsAtKay", "lang": "en", "location": "null", "create_at": date("2015-06-18"), "description": "Your Future Starts Here.", "followers_count": 6, "friends_count": 0, "statues_count": 1110 }, "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": 42013, "countyName": "Blair", "cityID": 4202184, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-01-05T10:00:13.000Z"), "id": 684434238213623808, "text": "I honestly just don't want to talk to anyone anymore ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 65522379, "name": "Megan Cummings", "screen_name": "Meggsss_", "lang": "en", "location": "null", "create_at": date("2009-08-13"), "description": "Mississippi State • alpha delta pi", "followers_count": 1625, "friends_count": 1184, "statues_count": 20834 }, "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-05T10:00:13.000Z"), "id": 684434238288998400, "text": "Mostly cloudy this afternoon, high 52 (11 C). Low 48 (9 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575204, "name": "New Orleans Weather", "screen_name": "NewOrleansWX", "lang": "en", "location": "New Orleans, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for New Orleans, LA", "followers_count": 2899, "friends_count": 92, "statues_count": 7891 }, "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-05T10:00:13.000Z"), "id": 684434238402265088, "text": "It's raining 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": 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": 310, "statues_count": 50317 }, "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-05T10:00:13.000Z"), "id": 684434238565908480, "text": "Shut up I told you to watch it but noooooo ���� https://t.co/WnY37Syzsf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 135974142, "name": "Michael Chan-Lok", "screen_name": "MrChantastic", "lang": "en", "location": "Springfield VA", "create_at": date("2010-04-22"), "description": "26. Mauritian blood. Northern VA bred. WVU kid. Anti Hello Kitty, like really anti. IG/snapchat: @mrchantastic", "followers_count": 621, "friends_count": 564, "statues_count": 34652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Falls Church, VA", "id": "077e6b03695c28ee", "name": "Falls Church", "place_type": "city", "bounding_box": rectangle("-77.19419,38.872266 -77.149803,38.896296") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51610, "countyName": "Falls Church", "cityID": 5127200, "cityName": "Falls Church" } }
+{ "create_at": datetime("2016-01-05T10:00:13.000Z"), "id": 684434238809206784, "text": "@MarkAmesExiled Lucas is a shit stain, and while he may have sold off the franchise, the Disney films must be bumping his wealth. Fuck him.", "in_reply_to_status": 684432678016389122, "in_reply_to_user": 38214152, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 38214152 }}, "user": { "id": 422048799, "name": "Gil Gamesh", "screen_name": "OuchoSparks", "lang": "en", "location": "Chicago", "create_at": date("2011-11-26"), "description": "Free agent. Socialist anarchist. Drummer. Very private intellectual. Life is absurd. Rebel or Die.", "followers_count": 245, "friends_count": 165, "statues_count": 22276 }, "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-05T10:00:13.000Z"), "id": 684434239073456128, "text": "Barberton Oh Temp:21.3°F Wind:1 mph Dir:SW Baro:Falling slowly Rain2day:0.00in Hum:60% UV:0.6 @ 13:00 01/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": 236, "friends_count": 228, "statues_count": 113693 }, "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-01-05T10:00:13.000Z"), "id": 684434239337574400, "text": "Don't make me go to work why did I take an extra shift? Ugh! Oh well money is good!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 339349762, "name": "KDarbs13", "screen_name": "kdarby13", "lang": "en", "location": "Olathe Kansas ", "create_at": date("2011-07-20"), "description": "Hey im Katie and im 22 i work Starbucks and at a bar part time! i love being w/ family and friends i love working out i am in Crossfit its part of my life now❤️", "followers_count": 152, "friends_count": 216, "statues_count": 11868 }, "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-01-05T10:00:13.000Z"), "id": 684434239568216064, "text": "Starbucks: Shift Supervisor (US) https://t.co/lU6EoG2RaV #Hospitality #Veterans #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.8832704,42.3707295"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "Veterans", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22495480, "name": "TMJ-MKE HRTA Jobs", "screen_name": "tmj_mke_hrta", "lang": "en", "location": "Milwaukee, WI", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Milwaukee, WI. Need help? Tweet us at @CareerArc!", "followers_count": 355, "friends_count": 290, "statues_count": 405 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waukegan, IL", "id": "b819c5d90b780b57", "name": "Waukegan", "place_type": "city", "bounding_box": rectangle("-87.96368,42.305624 -87.802772,42.431936") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1779293, "cityName": "Waukegan" } }
+{ "create_at": datetime("2016-01-05T10:00:13.000Z"), "id": 684434239811653632, "text": "After school everybody come to my house ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2878969744, "name": "NAAI❤️", "screen_name": "boythatsNAAI", "lang": "en", "location": "Trap $", "create_at": date("2014-11-15"), "description": "Sc: daddy_naai , #E4tank❤️", "followers_count": 419, "friends_count": 228, "statues_count": 26848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Belle Haven, VA", "id": "734825d1bfe33a28", "name": "Belle Haven", "place_type": "city", "bounding_box": rectangle("-77.081398,38.763609 -77.044219,38.792561") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5105928, "cityName": "Belle Haven" } }
+{ "create_at": datetime("2016-01-05T10:00:13.000Z"), "id": 684434239870218240, "text": "Road construction cleared in #Richmond on 59 SW Fwy Outbound at Hwy 99, stop and go traffic back to Brazos River, delay of 5 mins #traffic", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.69059,29.56113"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Richmond", "traffic" }}, "user": { "id": 249818911, "name": "TTN Houston", "screen_name": "TotalTrafficHOU", "lang": "en", "location": "Houston, TX", "create_at": date("2011-02-09"), "description": "Your #1 stop for all your Houston Traffic needs! See a #Traffic issue? Tweet us or call us on the tip line at 281- 214-0440 #HoustonTexans #HoustonAstros", "followers_count": 1015, "friends_count": 112, "statues_count": 130793 }, "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-01-05T10:00:13.000Z"), "id": 684434239945863168, "text": "@pink_lady56 @benshapiro @POTUS @realDonaldTrump", "in_reply_to_status": 684432577566916608, "in_reply_to_user": 291874773, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 291874773, 17995040, 1536791610, 25073877 }}, "user": { "id": 3310800271, "name": "REW", "screen_name": "robertew2945", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-08-09"), "description": "United States Army. Nuclear Missile Deployment. \nElectronic Consultant. Retired.\nBass Fisherman Guide Service.", "followers_count": 846, "friends_count": 683, "statues_count": 13242 }, "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-05T10:00:13.000Z"), "id": 684434239949934592, "text": "Can you recommend anyone for this #SupplyChain #job? https://t.co/WMa2KDYRzo #Milwaukee, WI #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.9064736,43.0389025"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupplyChain", "job", "Milwaukee", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 189279467, "name": "Milwaukee S-Chain", "screen_name": "tmj_MKE_schn", "lang": "en", "location": "Milwaukee, WI", "create_at": date("2010-09-10"), "description": "Follow this account for geo-targeted Supply Chain job tweets in Milwaukee, WI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 275, "friends_count": 262, "statues_count": 18 }, "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-05T10:00:13.000Z"), "id": 684434240147185664, "text": "@caspian oh word okay. Thanks !", "in_reply_to_status": 684434068675649536, "in_reply_to_user": 32065587, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 32065587 }}, "user": { "id": 160626356, "name": "cold by crossfade", "screen_name": "680South_", "lang": "en", "location": "MI", "create_at": date("2010-06-28"), "description": "null", "followers_count": 542, "friends_count": 531, "statues_count": 70665 }, "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-05T10:00:13.000Z"), "id": 684434240482639872, "text": "@thatsjazzzz @fit_thick_army and airborne and June, lol bet you salute me when you see me though ���� lmao I'm just playing no need to get mad", "in_reply_to_status": 684434005597536256, "in_reply_to_user": 236164655, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 236164655, 4264082836 }}, "user": { "id": 822029652, "name": "•CJ•", "screen_name": "HP_III", "lang": "en", "location": "with Kiana❤️", "create_at": date("2012-09-13"), "description": "Never. Enough. Money•Pitt State #O1GN", "followers_count": 1019, "friends_count": 1048, "statues_count": 22799 }, "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-05T10:00:13.000Z"), "id": 684434240545488896, "text": "I really hate just sitting here but I can't fucking do anything..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 983678312, "name": "Britt Petersen", "screen_name": "QueenGingyy", "lang": "en", "location": "On An Adventure", "create_at": date("2012-12-01"), "description": "Ginger. {Reckless}. RMP. ❤️✌️", "followers_count": 281, "friends_count": 229, "statues_count": 5918 }, "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-01-05T10:00:13.000Z"), "id": 684434240558112768, "text": "@OutnumberedFNC the #AmericanVoter is fed up with #DNC/#GOP who blindly let #Obama be #Dictator so #FireAll536OfThem #MakeAmericaGreatAgain", "in_reply_to_status": -1, "in_reply_to_user": 2448135692, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AmericanVoter", "DNC", "GOP", "Obama", "Dictator", "FireAll536OfThem", "MakeAmericaGreatAgain" }}, "user_mentions": {{ 2448135692 }}, "user": { "id": 787989258, "name": "Tryce Ankrom", "screen_name": "tryceankrom", "lang": "en", "location": "Independence, MO ", "create_at": date("2012-08-28"), "description": "Superintendent, Carpenter, Estimator, Drafter, Former IRS, Ind Voter, Patriot, of founding families, follow ENGLISH ONLY & I DON'T BUY FOLLOWERS! Ever!", "followers_count": 2160, "friends_count": 2160, "statues_count": 23512 }, "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-01-05T10:00:13.000Z"), "id": 684434240663076865, "text": "@jkottke there's a good 30 rock joke about it -- grandkids as skateboarders i think", "in_reply_to_status": 684409999276244992, "in_reply_to_user": 1305941, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1305941 }}, "user": { "id": 15970436, "name": "Fern Diaz", "screen_name": "ferndiaz", "lang": "en", "location": "nyc", "create_at": date("2008-08-24"), "description": "head of community at @hugeinc //// team @point_oh //// i've probably read your book ////", "followers_count": 2062, "friends_count": 2286, "statues_count": 7056 }, "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-05T10:00:13.000Z"), "id": 684434240717500416, "text": "#Hackettstown, NJ #Healthcare #Job: L&D Labor and Delivery RN Registered Nurse at Supplemental Health Care https://t.co/pa1lxNd5Fb #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.8290555,40.8539879"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hackettstown", "Healthcare", "Job", "Jobs" }}, "user": { "id": 21629917, "name": "TMJ- NJN Health Jobs", "screen_name": "tmj_njn_health", "lang": "en", "location": "Northern NJ", "create_at": date("2009-02-22"), "description": "Follow this account for geo-targeted Healthcare job tweets in Northern NJ. Need help? Tweet us at @CareerArc!", "followers_count": 456, "friends_count": 307, "statues_count": 512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hackettstown, NJ", "id": "db6b3052eea65096", "name": "Hackettstown", "place_type": "city", "bounding_box": rectangle("-74.872448,40.806831 -74.782595,40.910458") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34041, "countyName": "Warren", "cityID": 3428710, "cityName": "Hackettstown" } }
+{ "create_at": datetime("2016-01-05T10:00:13.000Z"), "id": 684434241006878720, "text": "Can you recommend anyone for this #Hospitality #job? https://t.co/y62HTMPUvR #WestChester, PA #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.568867,39.9028292"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "WestChester", "Hiring", "CareerArc" }}, "user": { "id": 22489144, "name": "TMJ-PHL HRTA Jobs", "screen_name": "tmj_phl_hrta", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Philadelphia, PA. Need help? Tweet us at @CareerArc!", "followers_count": 453, "friends_count": 295, "statues_count": 469 }, "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" } }
+{ "create_at": datetime("2016-01-05T10:00:13.000Z"), "id": 684434241044623361, "text": "Interested in a #Healthcare #job near #Okemos, MI? This could be a great fit: https://t.co/TgtoLIVG6p #PT #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.4274744,42.722257"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "Okemos", "PT", "Hiring" }}, "user": { "id": 2365967174, "name": "Aureus Medical Jobs", "screen_name": "aureusmedjobs", "lang": "en", "location": "Nationwide", "create_at": date("2014-02-28"), "description": "Follow @aureusmedjobs for #travelnursing, #traveltherapy, imaging, med lab, physicians, NP, and PA #jobs nationwide. Follow our company @aureusmedical.", "followers_count": 372, "friends_count": 1, "statues_count": 10220 }, "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.67829 -84.363083,42.744463") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2660340, "cityName": "Okemos" } }
+{ "create_at": datetime("2016-01-05T10:00:13.000Z"), "id": 684434241065631745, "text": "https://t.co/oBMwGPnkYG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 16995760, "name": "9eyedeel", "screen_name": "9eyedeel", "lang": "en", "location": "Saint-Méard-de-Drône", "create_at": date("2008-10-27"), "description": "negligence is an extreme thing", "followers_count": 1361, "friends_count": 1992, "statues_count": 19036 }, "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-05T10:00:13.000Z"), "id": 684434241153662976, "text": "Mostly sunny this afternoon, high 47 (8 C). Low 33 (1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575255, "name": "Shreveport Weather", "screen_name": "ShreveportWX", "lang": "en", "location": "Shreveport, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Shreveport, LA", "followers_count": 698, "friends_count": 92, "statues_count": 7949 }, "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-05T10:00:13.000Z"), "id": 684434241413713921, "text": "@taylerburch grizfolk, state champs, arkells, Chet faker, swim deep", "in_reply_to_status": 684390787048583168, "in_reply_to_user": 348533270, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 348533270 }}, "user": { "id": 314470761, "name": "Tay™", "screen_name": "TayTayKreiling", "lang": "en", "location": "Colorado/Lake Forest", "create_at": date("2011-06-10"), "description": "I bet a lot of me was lost, t's uncrossed and i's undotted", "followers_count": 494, "friends_count": 299, "statues_count": 13143 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dakota Ridge, CO", "id": "01864fd50a98b69f", "name": "Dakota Ridge", "place_type": "city", "bounding_box": rectangle("-105.172252,39.588911 -105.109576,39.65324") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8059, "countyName": "Jefferson", "cityID": 819150, "cityName": "Dakota Ridge" } }
+{ "create_at": datetime("2016-01-05T10:00:14.000Z"), "id": 684434241510191104, "text": "Also Tinder out here is great. 95% of the girls on it are hot 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": 366433746, "name": "YUNG PIFFY", "screen_name": "Austyn734", "lang": "en", "location": "null", "create_at": date("2011-09-01"), "description": "sip breast milk outta lean cup", "followers_count": 436, "friends_count": 191, "statues_count": 20717 }, "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-05T10:00:14.000Z"), "id": 684434241657126912, "text": "9 noches inolvidables con mi boti�� gracias por todo siempre, eres la… https://t.co/aBIlvaoSMt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.13103972,25.78431767"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 838721786, "name": "negrola✌️", "screen_name": "_ngmrodz", "lang": "en", "location": "Dorado, Pe Erre☀", "create_at": date("2012-09-21"), "description": "una gorda feliz y acostumbrada a las criticas | ⚾️ | UPR |", "followers_count": 362, "friends_count": 756, "statues_count": 22751 }, "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-05T10:00:14.000Z"), "id": 684434241694908416, "text": "Why she thinkin about me all hours of the night �� @Jayy_Lovelyyy https://t.co/YUk0NC2XgJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 145029298 }}, "user": { "id": 175241645, "name": "Mikaela.", "screen_name": "Ummikaela", "lang": "en", "location": "Norfolk, VA", "create_at": date("2010-08-05"), "description": "20. navy. Cali raised. 0524♊️", "followers_count": 349, "friends_count": 256, "statues_count": 19541 }, "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-05T10:00:14.000Z"), "id": 684434241824886786, "text": "When you are Matrix af. �� #matrix #school #friends #goodtimes… https://t.co/pLvF9rrIUm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.64712157,28.80861909"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "matrix", "school", "friends", "goodtimes" }}, "user": { "id": 4640361202, "name": "Josh Lewis", "screen_name": "Jlew_18", "lang": "en", "location": "Florida, USA", "create_at": date("2015-12-28"), "description": "Music Sports Funny Influencers", "followers_count": 13, "friends_count": 62, "statues_count": 7 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Dora, FL", "id": "eb33ee1bfc5b789d", "name": "Mount Dora", "place_type": "city", "bounding_box": rectangle("-81.690281,28.785679 -81.603465,28.851945") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12069, "countyName": "Lake", "cityID": 1247050, "cityName": "Mount Dora" } }
+{ "create_at": datetime("2016-01-05T10:00:14.000Z"), "id": 684434241866711040, "text": "Want to work at ZF TRW? We're #hiring in #Livonia, MI! Click for details: https://t.co/OhbAlrw39q #HR #Veterans #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.3527097,42.36837"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Livonia", "HR", "Veterans", "Job", "Jobs" }}, "user": { "id": 23130397, "name": "TMJ-DTW HR Jobs", "screen_name": "tmj_dtw_hr", "lang": "en", "location": "Detroit, MI", "create_at": date("2009-03-06"), "description": "Follow this account for geo-targeted Human Resources job tweets in Detroit, MI. Need help? Tweet us at @CareerArc!", "followers_count": 387, "friends_count": 294, "statues_count": 19 }, "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-01-05T10:00:14.000Z"), "id": 684434242340655104, "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": 2208234523, "name": "Mulatto❄️", "screen_name": "_SteezyShawn", "lang": "en", "location": "The Bank ", "create_at": date("2013-11-21"), "description": "Dream Epic | #Loading... | 18 | Dtx |", "followers_count": 983, "friends_count": 518, "statues_count": 14707 }, "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-05T10:00:14.000Z"), "id": 684434242667872256, "text": "crazy town references https://t.co/EfD0PrCWWM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 191671359, "name": "irby", "screen_name": "stevesweatpants", "lang": "en", "location": "bedford stuyvestant, dogs.", "create_at": date("2010-09-16"), "description": "co-founder + editor-at-large + social director for @streetdreamsnyc + photographer || #TEAMCOZY stevesweatpantsphoto@gmail.com", "followers_count": 2027, "friends_count": 559, "statues_count": 9943 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:14.000Z"), "id": 684434242667941888, "text": "I'm at Sawtelle / Little Japantown in Los Angeles, CA https://t.co/u6Xo1ird9j", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.45484734,34.03779556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24994477, "name": "Nikki", "screen_name": "hapaniki", "lang": "en", "location": "null", "create_at": date("2009-03-17"), "description": "Did someone say Happy Hour?", "followers_count": 181, "friends_count": 180, "statues_count": 25945 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:14.000Z"), "id": 684434242714112005, "text": "@taylorr_klassen @HannahColeman97 omg this is my fave�� you're awesome����", "in_reply_to_status": 684384961772826624, "in_reply_to_user": 807809443, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 807809443, 165057329 }}, "user": { "id": 887970492, "name": "sa5m", "screen_name": "sami_orchard", "lang": "en", "location": "null", "create_at": date("2012-10-17"), "description": "Romans 12:2", "followers_count": 545, "friends_count": 625, "statues_count": 1636 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sandusky, MI", "id": "c9b86bc3797559a0", "name": "Sandusky", "place_type": "city", "bounding_box": rectangle("-82.85122,43.406486 -82.816173,43.431997") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26151, "countyName": "Sanilac", "cityID": 2671540, "cityName": "Sandusky" } }
+{ "create_at": datetime("2016-01-05T10:00:14.000Z"), "id": 684434242760241152, "text": "Gang! Gang!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3400668586, "name": "Lark", "screen_name": "302Lark", "lang": "en", "location": "Delaware/Niagara Falls ", "create_at": date("2015-08-02"), "description": "Niagara Men's Basketball #NOW", "followers_count": 1052, "friends_count": 570, "statues_count": 6497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lewiston, NY", "id": "1b616336b62a6350", "name": "Lewiston", "place_type": "city", "bounding_box": rectangle("-79.052768,43.131586 -78.964479,43.220267") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36063, "countyName": "Niagara", "cityID": 3642147, "cityName": "Lewiston" } }
+{ "create_at": datetime("2016-01-05T10:00:14.000Z"), "id": 684434242806267905, "text": "Taxiing in Atlanta, once again, but 1st time this year. #worktravel #roadwarrior #frequentflyer… https://t.co/wJGKX4UGdb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.43545035,33.64051041"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "worktravel", "roadwarrior", "frequentflyer" }}, "user": { "id": 16378623, "name": "TonyM", "screen_name": "FunkyCold", "lang": "en", "location": "Kansas City", "create_at": date("2008-09-20"), "description": "Taking it all in!", "followers_count": 245, "friends_count": 584, "statues_count": 3872 }, "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-05T10:00:14.000Z"), "id": 684434243007688704, "text": "@BelieveOnJesus @de_meana Yes even though you asked me not to and insulted me I prayed 4 U Enginne", "in_reply_to_status": 684432698815987712, "in_reply_to_user": 1086906026, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1086906026, 4703627876 }}, "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": 118371, "friends_count": 96750, "statues_count": 42595 }, "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-05T10:00:14.000Z"), "id": 684434243078897664, "text": "lmfaoooooo oh. https://t.co/zLrXb8o8q5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 847957202, "name": "Alyssa✨", "screen_name": "afreakingmazing", "lang": "en", "location": "Austin, TX", "create_at": date("2012-09-26"), "description": "19 & I'm chillen.", "followers_count": 3401, "friends_count": 2808, "statues_count": 67122 }, "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-05T10:00:14.000Z"), "id": 684434243288731649, "text": "// dunn bros // made always with love @ Dunn Bros Coffee - Excelsior, MN https://t.co/LzMh56pAAE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.5655136,44.9038467"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48708105, "name": "Dunn Brothers Coffee", "screen_name": "dbexcelsior", "lang": "en", "location": "11 Water Street, Excelsior MN ", "create_at": date("2009-06-19"), "description": "• open daily at 6 am • coffee • tea • food • wine • beer • live music • meeting room • patio • 952-401-8004 • instagram: @dbexcelsior", "followers_count": 421, "friends_count": 302, "statues_count": 400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Excelsior, MN", "id": "d4d4b40fbc090318", "name": "Excelsior", "place_type": "city", "bounding_box": rectangle("-93.579101,44.89565 -93.552384,44.909349") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2720078, "cityName": "Excelsior" } }
+{ "create_at": datetime("2016-01-05T10:00:14.000Z"), "id": 684434243317972992, "text": "New Book is here!Communication:The Missing Piece to the Marriage Puzzle.Free shipping.Go to https://t.co/tjeDuQidvC @LifeWayMarriage", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 22804757 }}, "user": { "id": 91670121, "name": "Steve/Debbie Wilson", "screen_name": "SteveAndDebbie", "lang": "en", "location": "Cross Roads, TX", "create_at": date("2009-11-21"), "description": "We have a Marriage Ministry called Marriage Matters Now. We travel and teach marriage conferences. For booking info call 318-469-2438.", "followers_count": 663, "friends_count": 100, "statues_count": 556 }, "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-05T10:00:14.000Z"), "id": 684434243330523136, "text": "look at this PUPPY https://t.co/cMhfRAt7Bs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 61793526, "name": "deanna ✨", "screen_name": "jacksmichaeI", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-07-31"), "description": "laters baby // greedy emo", "followers_count": 912, "friends_count": 232, "statues_count": 28503 }, "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-05T10:00:14.000Z"), "id": 684434243368300544, "text": "I'm most definitely not a fan �������������������������� https://t.co/1k8bhl04fK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2169658092, "name": "#️⃣Jan.20‼️", "screen_name": "Dmoe_4Fss", "lang": "en", "location": "Dallas, TX", "create_at": date("2013-11-01"), "description": "Keep it 8 more than 92 with me‼️ #God #OTF #Ballislife #4️⃣ #FleeGang yall know my pops @Dame_Lillard. Racine WI ✈️ Dallas TX ig:dmoe_bucketteam35SC: ll_dmoe4fs", "followers_count": 1492, "friends_count": 1158, "statues_count": 31085 }, "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-05T10:00:14.000Z"), "id": 684434243410210816, "text": "#Rincon, GA #Transportation #Job: Delivery Driver - CDL Class A Required at Lowe's https://t.co/o6bOIPFmkC #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2353905,32.2960289"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Rincon", "Transportation", "Job", "Jobs", "Hiring" }}, "user": { "id": 149734682, "name": "TMJ-GAS Transport.", "screen_name": "tmj_GAS_transp", "lang": "en", "location": "Savannah, GA", "create_at": date("2010-05-29"), "description": "Follow this account for geo-targeted Transportation job tweets in Savannah, GA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 281, "friends_count": 273, "statues_count": 35 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rincon, GA", "id": "ebb4e094eaa3c48e", "name": "Rincon", "place_type": "city", "bounding_box": rectangle("-81.25462,32.237761 -81.191648,32.318171") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13103, "countyName": "Effingham", "cityID": 1365296, "cityName": "Rincon" } }
+{ "create_at": datetime("2016-01-05T10:00:14.000Z"), "id": 684434243573923840, "text": "i don't take any kind of advice from people who wear sperrys", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2950427609, "name": "ヽ༼ ಠ益ಠ ༽ノ", "screen_name": "paloma_narvaez", "lang": "en", "location": "tampa, fla", "create_at": date("2014-12-29"), "description": "Not so serious", "followers_count": 134, "friends_count": 91, "statues_count": 6567 }, "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-05T10:00:14.000Z"), "id": 684434244165304322, "text": "I'm at Triangle Orthopaedic Associates in Raleigh, NC w/ @ultimateoutcast https://t.co/bcSJmq5wzB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.70558262,35.82356353"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 350532675 }}, "user": { "id": 703941709, "name": "◟̽ hannah ◞̽", "screen_name": "chillxlarrie", "lang": "en-gb", "location": "1D #OTRAPittsburgh 8.2.15 -155", "create_at": date("2012-07-18"), "description": "@chillnarrie is my lannahrie ◟̽◞̽", "followers_count": 8289, "friends_count": 4738, "statues_count": 105437 }, "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-05T10:00:14.000Z"), "id": 684434244353953793, "text": "Partly cloudy this afternoon, high 52 (11 C). Low 39 (4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575312, "name": "Lake Charles Weather", "screen_name": "LakeCharlesWX", "lang": "en", "location": "Lake Charles, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Lake Charles, LA", "followers_count": 540, "friends_count": 92, "statues_count": 7933 }, "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-05T10:00:14.000Z"), "id": 684434244718821377, "text": "�� https://t.co/KkBiqsQur6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 299461181, "name": "FEB15♒️", "screen_name": "hentobean_", "lang": "en", "location": "United States", "create_at": date("2011-05-15"), "description": "WhatDoesn'tKillMeMakesMeStronger #SC:hentosotatted #IG:hentoeehoeegnf2️⃣0️⃣on2️⃣8️⃣s", "followers_count": 964, "friends_count": 942, "statues_count": 33495 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bayou La Batre, AL", "id": "ce69549281745bb7", "name": "Bayou La Batre", "place_type": "city", "bounding_box": rectangle("-88.304215,30.375901 -88.225444,30.469013") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile", "cityID": 104684, "cityName": "Bayou La Batre" } }
+{ "create_at": datetime("2016-01-05T10:00:14.000Z"), "id": 684434244811141120, "text": "Good morning, El Niño", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 192038194, "name": "Shina Villines", "screen_name": "shiClassy", "lang": "en", "location": "Brentwood, Los Angeles", "create_at": date("2010-09-17"), "description": "you'll love me or hate me but ... follow me IG shinavillines", "followers_count": 766, "friends_count": 414, "statues_count": 24583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:14.000Z"), "id": 684434245285117952, "text": "All things happen in God's perfect time.\nChardParojinog Showtime2016", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4152704473, "name": "florlita reyes", "screen_name": "florlita_reyes", "lang": "en", "location": "The Woodlands, Texas USA ", "create_at": date("2015-11-06"), "description": "Side by side, miles apart, CHARDNATICS we are one...Supportive and Dedicated to...\n@mrparojinog all the way...I LOVEYOU bunso!!!!", "followers_count": 64, "friends_count": 41, "statues_count": 2405 }, "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-05T10:00:14.000Z"), "id": 684434245444591616, "text": "My biggest Goal is to never fall as low as I did LastYear", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3313409627, "name": "NERA!", "screen_name": "Neraaa4", "lang": "en", "location": "null", "create_at": date("2015-06-08"), "description": "life's what you make it", "followers_count": 247, "friends_count": 167, "statues_count": 10983 }, "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-05T10:00:14.000Z"), "id": 684434245469618176, "text": "@Mike_Horvat that's awesome! I went in '13 and had one of the best times in my life.", "in_reply_to_status": 684429783858417664, "in_reply_to_user": 43743671, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 43743671 }}, "user": { "id": 42819815, "name": "Misty Arnold", "screen_name": "MistyArnold", "lang": "en", "location": "Hanford", "create_at": date("2009-05-26"), "description": "⚾MLB/MiLB ❤️ \n Pollstar && Nails ::IG @misty___lynn", "followers_count": 502, "friends_count": 424, "statues_count": 30290 }, "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-05T10:00:14.000Z"), "id": 684434245616418816, "text": "The weather forecast for tonight is clear with a low of 25°F. #BirdDay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.5333,41.5167"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BirdDay" }}, "user": { "id": 42666285, "name": "Quad Cities Weather", "screen_name": "qcweather", "lang": "en", "location": "Quad Cities", "create_at": date("2009-05-26"), "description": "Is it going to be warm, wet or cold in the Quad Cities today? You can also find us on Facebook.", "followers_count": 741, "friends_count": 0, "statues_count": 11480 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rock Island Arsenal, IL", "id": "8f966c3da836a196", "name": "Rock Island Arsenal", "place_type": "city", "bounding_box": rectangle("-90.570223,41.509992 -90.511303,41.523635") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17161, "countyName": "Rock Island", "cityID": 1765104, "cityName": "Rock Island Arsenal" } }
+{ "create_at": datetime("2016-01-05T10:00:14.000Z"), "id": 684434245624836097, "text": "Disabled vehicle, right lane blocked in #Newark on Rt-22 EB near Rt-27, stopped traffic back to CR-21, delay of 18 mins #traffic", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.19217,40.70797"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Newark", "traffic" }}, "user": { "id": 42640432, "name": "Total Traffic NYC", "screen_name": "TotalTrafficNYC", "lang": "en", "location": "New York City & New Jersey", "create_at": date("2009-05-26"), "description": "Total Traffic Network delivers real-time traffic via in-car and portable navigation, broadcast media, wireless, & Internet. Call Tips Hands-Free to 212-377-7757", "followers_count": 2062, "friends_count": 366, "statues_count": 311970 }, "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-05T10:00:15.000Z"), "id": 684434245725646848, "text": "Pitain j'ai eu un vrai fatigué quand même en tant que binôme pour ce voyage", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 809739811, "name": "Harlem", "screen_name": "Kev_joke", "lang": "fr", "location": "New York, NY", "create_at": date("2012-09-07"), "description": "Papa me reniera jamais , j'suis ni flic ni pd ! #Team237", "followers_count": 144, "friends_count": 53, "statues_count": 14153 }, "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-01-05T10:00:15.000Z"), "id": 684434245729824768, "text": "Wouldn't have wanted to spend 11 straight days at Disney World with… https://t.co/44nTypbObl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.55890096,28.35785666"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1476366728, "name": "Gretchin", "screen_name": "gretchfid", "lang": "en", "location": "Lilburn, GA", "create_at": date("2013-06-01"), "description": "null", "followers_count": 333, "friends_count": 165, "statues_count": 10679 }, "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-01-05T10:00:15.000Z"), "id": 684434246035886082, "text": "My mom literally called me 6 times in a row for cheese.. &made me deliver it to her work. Glad I could help her overcome that lunch tragedy��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "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": 628, "friends_count": 491, "statues_count": 7287 }, "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-05T10:00:15.000Z"), "id": 684434246480605185, "text": "@Newinthecity_ you don't have to necessarily work around my area . lol.", "in_reply_to_status": 684432437519237120, "in_reply_to_user": 261829355, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 261829355 }}, "user": { "id": 30968127, "name": "Danni", "screen_name": "DanniBadd", "lang": "en", "location": "Cleveland , Ohio ", "create_at": date("2009-04-13"), "description": "snapchat : dannikae", "followers_count": 1876, "friends_count": 1549, "statues_count": 85241 }, "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-05T10:00:15.000Z"), "id": 684434246493052930, "text": "Everyone having 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": 2849353561, "name": "kam❣", "screen_name": "kameronlasheaa", "lang": "en", "location": "Htx.", "create_at": date("2014-10-09"), "description": "get with the winning team ...", "followers_count": 944, "friends_count": 696, "statues_count": 7950 }, "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-01-05T10:00:15.000Z"), "id": 684434246610505728, "text": "Wind 8.0 mph S. Barometer 1034.10 mb, Falling. Temperature 36.3 °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": 25, "friends_count": 124, "statues_count": 11626 }, "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-05T10:00:15.000Z"), "id": 684434246736322563, "text": "Congratulations to all the #ConservationDistrict employees that went through #NCCDEA professional development program! #NCASWCD16", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ConservationDistrict", "NCCDEA", "NCASWCD16" }}, "user": { "id": 2882971929, "name": "NC Fnd Soil & Water", "screen_name": "ncsoilwater", "lang": "en", "location": "Raleigh NC", "create_at": date("2014-11-18"), "description": "NC Foundation for Soil and Water Conservation is a 501(c)3 non profit raising funds to improve the environment, educate the citizen, increase district capacity,", "followers_count": 243, "friends_count": 372, "statues_count": 360 }, "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-05T10:00:15.000Z"), "id": 684434246790963200, "text": "throwback to that one time I got yelled at in Hermès in Paris cos I took pictures lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3013753279, "name": "madison smith", "screen_name": "madelizabeth12", "lang": "en", "location": "naperville, il", "create_at": date("2015-02-08"), "description": "que voudras-tu faire?", "followers_count": 90, "friends_count": 181, "statues_count": 3497 }, "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-01-05T10:00:15.000Z"), "id": 684434246916685824, "text": "Mostly sunny this afternoon, high 52 (11 C). Low 43 (6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575370, "name": "Lafayette Weather", "screen_name": "LafayetteWX", "lang": "en", "location": "Lafayette, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Lafayette, LA", "followers_count": 468, "friends_count": 92, "statues_count": 7972 }, "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-05T10:00:15.000Z"), "id": 684434246937612288, "text": "Y'all @getsomeblasian @_Akaayy_ @ronniefolarin https://t.co/awAdelpl1o", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3828172273, 329997649, 3196848588 }}, "user": { "id": 2907702350, "name": "Janelle Siaffa", "screen_name": "JennyGotDaJuice", "lang": "en", "location": "HTX", "create_at": date("2014-11-22"), "description": "Pre-med major. 19. Let your actions manifest your thoughts, your beliefs, & your passion.♌️❤️ SC: NellyNellz_96", "followers_count": 259, "friends_count": 185, "statues_count": 9383 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:15.000Z"), "id": 684434246950334465, "text": "Lies on us blkmoms @vp @potus>@SenBlumenthal @senatorreid @mitchellreports @thefix @SusanPage @berniesanders @HillaryClinton #Sandyhook", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sandyhook" }}, "user_mentions": {{ 325830217, 1536791610, 278124059, 16789970, 89820928, 14412533, 21307076, 216776631, 1339835893 }}, "user": { "id": 15970073, "name": "lovelyladypa", "screen_name": "whome51obama", "lang": "en", "location": "null", "create_at": date("2008-08-24"), "description": "null", "followers_count": 1648, "friends_count": 499, "statues_count": 497401 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Swissvale, PA", "id": "23e4902f541fed6e", "name": "Swissvale", "place_type": "city", "bounding_box": rectangle("-79.899459,40.410552 -79.867691,40.43494") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4275816, "cityName": "Swissvale" } }
+{ "create_at": datetime("2016-01-05T10:00:15.000Z"), "id": 684434247646445568, "text": "@PalmitaLokz I heard they weren't allowed to pull cashiers anymore? :/", "in_reply_to_status": 684433394621640705, "in_reply_to_user": 283174178, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 283174178 }}, "user": { "id": 183317983, "name": "babydoll", "screen_name": "lizxl4mebr4in", "lang": "en", "location": "null", "create_at": date("2010-08-26"), "description": "castigada en el granero", "followers_count": 807, "friends_count": 584, "statues_count": 53301 }, "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-05T10:00:15.000Z"), "id": 684434247659081728, "text": "I've been to Gillette in Foxborough and the Cardinals stadium in Glendale. Levi's is a dump compared to those stadiums that are much older.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 301846724, "name": "Jason", "screen_name": "JayMinz", "lang": "en", "location": "San Francisco", "create_at": date("2011-05-19"), "description": "Giants, Sharks, World Champion Warriors and Niners. New found love of traveling. San Francisco native.", "followers_count": 113, "friends_count": 559, "statues_count": 2375 }, "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-05T10:00:15.000Z"), "id": 684434247705296900, "text": "#pd for #fcsmeti @AlpharettaHigh! https://t.co/PEyDrONLPx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "pd", "fcsmeti" }}, "user_mentions": {{ 604071930 }}, "user": { "id": 43392716, "name": "Benjamin Lynch", "screen_name": "bukharajones", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-05-29"), "description": "Librarian and educational technology instructor in Fulton County.", "followers_count": 83, "friends_count": 212, "statues_count": 328 }, "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-05T10:00:15.000Z"), "id": 684434248791502848, "text": "Raul fucked up my thumb guys", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 320005483, "name": "Abe", "screen_name": "AlmightyAbe", "lang": "en", "location": "PSHS x Richardson x Dallas ", "create_at": date("2011-06-18"), "description": "YOU GETTIN MAD IM GETTING RICH", "followers_count": 519, "friends_count": 313, "statues_count": 60657 }, "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-05T10:00:15.000Z"), "id": 684434248854458368, "text": "I HAVE NEVER BEEN SO TIRED OF SOMEONES EXISTENCE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1569064375, "name": "mars", "screen_name": "maria_brindle", "lang": "en", "location": "i never know", "create_at": date("2013-07-04"), "description": "its me, dyke boy", "followers_count": 273, "friends_count": 185, "statues_count": 9745 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bella Vista, AR", "id": "988bfa67502cf995", "name": "Bella Vista", "place_type": "city", "bounding_box": rectangle("-94.355602,36.415389 -94.176396,36.499732") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton", "cityID": 504840, "cityName": "Bella Vista" } }
+{ "create_at": datetime("2016-01-05T10:00:15.000Z"), "id": 684434249219313664, "text": "The rain is hella killing 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": 3230282990, "name": "Ashley", "screen_name": "_Ashleyfeng", "lang": "en", "location": "null", "create_at": date("2015-05-29"), "description": "null", "followers_count": 190, "friends_count": 151, "statues_count": 230 }, "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-05T10:00:15.000Z"), "id": 684434249412382721, "text": "Exactly���� https://t.co/HMd5uAdA5a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1532075162, "name": "onerrrr", "screen_name": "_janinee__", "lang": "en", "location": "null", "create_at": date("2013-06-19"), "description": "null", "followers_count": 1554, "friends_count": 980, "statues_count": 68937 }, "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-05T10:00:15.000Z"), "id": 684434249588453378, "text": "See our latest #Westfield, MA #job and click to apply: RN - https://t.co/ZmsPpyRLRT #RN #Nursing #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.7592109,42.11807"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Westfield", "job", "RN", "Nursing", "Hiring" }}, "user": { "id": 2688167298, "name": "Noble Hospital Jobs", "screen_name": "JobsatNobleHosp", "lang": "en", "location": "Westfield, MA", "create_at": date("2014-07-28"), "description": "Noble Hospital is a regional community hospital, providing a broad range of expert services to Greater Westfield and surrounding areas.", "followers_count": 17, "friends_count": 0, "statues_count": 95 }, "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-01-05T10:00:15.000Z"), "id": 684434249836007425, "text": "iLMAOOO https://t.co/zLZIJirpFp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 224354592, "name": "Edgar Allan Hoe", "screen_name": "XIXMCMLXXXV", "lang": "en", "location": "Angel Grove", "create_at": date("2010-12-08"), "description": "Welcome...AND RAQUEL TAKE THAT FUCKING JACKET OFF!! #StopGunViolence", "followers_count": 2184, "friends_count": 624, "statues_count": 64618 }, "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-05T10:00:16.000Z"), "id": 684434249911537665, "text": "@trafficMcbo HABLEMOS DEL PRESENTE Y FUTURO DE VZLA fuera estos malandros corruptos ladrones ������������������������ https://t.co/hQMwnXGp6b", "in_reply_to_status": -1, "in_reply_to_user": 95471348, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 95471348 }}, "user": { "id": 308242174, "name": "Yelixsa Sanchez", "screen_name": "Yelixsasanchez", "lang": "es", "location": "Maracaibo-Vzla", "create_at": date("2011-05-30"), "description": "Vzla - EL 6D NOS DA LA OPORTUNIDAD DE RESCATAR A VZLA HAY MUCHO TRABAJO POR HACER UNA NUEVA ASAMBLEA A DARLE DURO Vzla productiva y libre", "followers_count": 156, "friends_count": 215, "statues_count": 1513 }, "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-05T10:00:16.000Z"), "id": 684434249923997696, "text": "Mostly sunny this afternoon, high 45 (7 C). Low 29 (-2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575409, "name": "Fort Smith Weather", "screen_name": "FortSmithWX", "lang": "en", "location": "Fort Smith, AR", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Fort Smith, AR", "followers_count": 442, "friends_count": 92, "statues_count": 8021 }, "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-05T10:00:16.000Z"), "id": 684434249970143232, "text": "I could be there by midnight... You have no idea how tempted I am RN #iheartapu https://t.co/eXfctHpBRV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "iheartapu" }}, "user": { "id": 3151862215, "name": "Marisa Grace", "screen_name": "CousinoMarisa", "lang": "en", "location": "null", "create_at": date("2015-04-12"), "description": "CO→CA // APU", "followers_count": 54, "friends_count": 52, "statues_count": 57 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dillon, CO", "id": "15d2b24f6471e501", "name": "Dillon", "place_type": "city", "bounding_box": rectangle("-106.063106,39.603056 -106.010416,39.649464") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8117, "countyName": "Summit", "cityID": 820440, "cityName": "Dillon" } }
+{ "create_at": datetime("2016-01-05T10:00:16.000Z"), "id": 684434249995300865, "text": "I feel like break has been going on forever. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 459784484, "name": "C•nail ☀️", "screen_name": "CaitlynTheNail", "lang": "en", "location": "Fort Worth, Weatherford Texas ", "create_at": date("2012-01-09"), "description": "i try doing a lot of squats so my ass can some day be as big as my sass", "followers_count": 1026, "friends_count": 419, "statues_count": 29096 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weatherford, TX", "id": "344b25bca5f9cdc5", "name": "Weatherford", "place_type": "city", "bounding_box": rectangle("-97.839791,32.716432 -97.723744,32.789288") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48367, "countyName": "Parker", "cityID": 4876864, "cityName": "Weatherford" } }
+{ "create_at": datetime("2016-01-05T10:00:16.000Z"), "id": 684434250242732033, "text": "@joomlacomb grams wildin", "in_reply_to_status": 684262942649397248, "in_reply_to_user": 2852934384, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2852934384 }}, "user": { "id": 18279444, "name": "helena", "screen_name": "northwesternsky", "lang": "en", "location": "eug/pdx", "create_at": date("2008-12-20"), "description": "everytime i rebuild i am shown an even brighter light", "followers_count": 192, "friends_count": 689, "statues_count": 9769 }, "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-05T10:00:16.000Z"), "id": 684434250360176640, "text": "I want 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": 1276827536, "name": "alexcia", "screen_name": "alexciasanchez", "lang": "en", "location": "null", "create_at": date("2013-03-17"), "description": "Ohs.", "followers_count": 276, "friends_count": 236, "statues_count": 12049 }, "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-05T10:00:16.000Z"), "id": 684434250469367808, "text": "imagine Rize and Saiko as real friends though", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1135768740, "name": "New Year New Oikawa", "screen_name": "0SwanPrincess0", "lang": "en", "location": "null", "create_at": date("2013-01-30"), "description": "cosplayer: http://theprincesswithmagichair.tumblr.com ✨she/her pretty gay ✨ 98% of my life is spent thinking about anime and girls but also Oikawa ✨ENFJ✨", "followers_count": 226, "friends_count": 287, "statues_count": 17087 }, "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-05T10:00:16.000Z"), "id": 684434250523750400, "text": "[Tue Jan 5th, 12:00pm] Temperature: 65.12 F, 18.4 C; Humidity: 38.1%; Light: 19.04%. https://t.co/UI5qWnQ21d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 269430133, "name": "Shoemaker Sandbox", "screen_name": "shoe_sandbox", "lang": "en", "location": "Earth (for now). ", "create_at": date("2011-03-20"), "description": "I am a bot. Developer account for @shoemaker.", "followers_count": 7, "friends_count": 1, "statues_count": 24501 }, "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-05T10:00:16.000Z"), "id": 684434250750279680, "text": "@V_CARD4 @_ughemi I'm down", "in_reply_to_status": 684433395321942016, "in_reply_to_user": 1884901537, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1884901537, 2826684084 }}, "user": { "id": 629064411, "name": "r kelly", "screen_name": "JoyceYaDigg", "lang": "en", "location": "null", "create_at": date("2012-07-06"), "description": "the sky is the limit", "followers_count": 981, "friends_count": 1309, "statues_count": 21607 }, "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-05T10:00:16.000Z"), "id": 684434250767056896, "text": "If @twitter uses a 10k tweet character limit, then I will make a twitter clone that isn't stupid.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 783214 }}, "user": { "id": 1342977524, "name": "Jared", "screen_name": "rylorjs", "lang": "en", "location": "Seattle, WA", "create_at": date("2013-04-10"), "description": "17 year old web dev / backend engineer. My tweets/opinions don't represent my employer. Web Master for @EverfreeNW. Look for me on @forbes in 5 years.", "followers_count": 836, "friends_count": 112, "statues_count": 81915 }, "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-05T10:00:16.000Z"), "id": 684434251069050880, "text": "Cladistics or phylogenetic systematics reveals the history of life via organic evolution and nothing about any god. https://t.co/2fLbnm8h6J", "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": 25065, "friends_count": 1983, "statues_count": 48516 }, "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-01-05T10:00:16.000Z"), "id": 684434251089981440, "text": "Bak to that Gym Flow today ������������ @GoodAngelChris U ain't reeeeadyy #2hrs in Everdaaaaaay ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "2hrs" }}, "user_mentions": {{ 2300990269 }}, "user": { "id": 2919289530, "name": "V!Va La V!Da", "screen_name": "badieGalcookiie", "lang": "en-US", "location": " LA ", "create_at": date("2014-12-04"), "description": "#Independent #Hustler (She's Mad But She's Magic✨) ♈️❤️♒️", "followers_count": 60, "friends_count": 138, "statues_count": 3317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:16.000Z"), "id": 684434251157245952, "text": "If you aren't here to make my life better then I don't want you in it.���� it's all so simple", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 802996351, "name": "golden✨", "screen_name": "niahh_love", "lang": "en", "location": "null", "create_at": date("2012-09-04"), "description": "Blood type: Gold", "followers_count": 540, "friends_count": 314, "statues_count": 14451 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sun Prairie, WI", "id": "0004580506c66a0e", "name": "Sun Prairie", "place_type": "city", "bounding_box": rectangle("-89.2948,43.154941 -89.166248,43.238626") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55025, "countyName": "Dane", "cityID": 5578600, "cityName": "Sun Prairie" } }
+{ "create_at": datetime("2016-01-05T10:00:16.000Z"), "id": 684434251383713793, "text": "I absolutely hate financial aid. (@ Front Range Community College - @frcc in Westminster, CO) https://t.co/BgYjzl8CQ1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.0378855,39.9014731"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23088018 }}, "user": { "id": 860053962, "name": "April Annabell", "screen_name": "AprilAnnabellM", "lang": "en", "location": "Berkley, CO", "create_at": date("2012-10-03"), "description": "Leave all doubt behind. Find your inspiration. And believe in yourself because you are worth it and you can do anything.", "followers_count": 250, "friends_count": 412, "statues_count": 9701 }, "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-05T10:00:16.000Z"), "id": 684434251807195137, "text": "@JennaJ55 @fijiwriter Also, I don’t like a lot of the crunchy sugar and grocery stores add to much of that to mask the taste of boring cake.", "in_reply_to_status": 684433495586910209, "in_reply_to_user": 25417140, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25417140, 14328897 }}, "user": { "id": 1081323786, "name": "jim", "screen_name": "bigjim", "lang": "en", "location": "Dallas, TX", "create_at": date("2013-01-11"), "description": "You know that little thing inside your head that keeps you from saying things you shouldn't? Yeah, I don't have one of those.", "followers_count": 129, "friends_count": 303, "statues_count": 7560 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Government District, Dallas", "id": "32e19f2a700afb9d", "name": "Government District", "place_type": "neighborhood", "bounding_box": rectangle("-96.806496,32.774565 -96.794997,32.782576") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-05T10:00:16.000Z"), "id": 684434251857657856, "text": "@Britt_Acks Thank goodness you wear headphones at your desk!!! ;-)", "in_reply_to_status": 684432978286620676, "in_reply_to_user": 544857836, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 544857836 }}, "user": { "id": 14392973, "name": "Gregg Weiss", "screen_name": "greggweiss", "lang": "en", "location": "Hastings on Hudson, NY", "create_at": date("2008-04-14"), "description": "Everything is with my family in mind. I help provide for them by doing work I love as VP Social Media at @MasterCard US. Content here=personal views.", "followers_count": 11951, "friends_count": 7542, "statues_count": 37376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hastings-on-Hudson, NY", "id": "c2800db81ba22abc", "name": "Hastings-on-Hudson", "place_type": "city", "bounding_box": rectangle("-73.888831,40.97653 -73.857066,41.0039") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3632710, "cityName": "Hastings-on-Hudson" } }
+{ "create_at": datetime("2016-01-05T10:00:16.000Z"), "id": 684434252415500288, "text": "Tickets are now available for both sessions of Winter Brew #LincolnSquare. Space is limited, so claim yours today! https://t.co/wC0AjdfsLs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LincolnSquare" }}, "user": { "id": 78399032, "name": "LincolnSq-Ravenswood", "screen_name": "LincolnSquareCC", "lang": "en", "location": "Chicago, IL", "create_at": date("2009-09-29"), "description": "2 Neigborhoods. 1 Community. Working Together. We are the Lincoln Square Ravenswood Chamber of Commerce (LSRCC) in Chicago tweeting everything local!", "followers_count": 3459, "friends_count": 422, "statues_count": 2839 }, "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-05T10:00:16.000Z"), "id": 684434252570693632, "text": "������ https://t.co/cUxdp4dxL9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 472711739, "name": "Papi Champú", "screen_name": "YeahImJamar", "lang": "en", "location": "null", "create_at": date("2012-01-23"), "description": "Sinatra lifestyle im just being Frank with you", "followers_count": 494, "friends_count": 460, "statues_count": 7595 }, "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-05T10:00:16.000Z"), "id": 684434252658806784, "text": "@GigiAjani so you saying you dont know any other Nigerians ��", "in_reply_to_status": 684433928946585600, "in_reply_to_user": 300327324, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 300327324 }}, "user": { "id": 57072966, "name": "God's Work", "screen_name": "Kingzbillay", "lang": "en", "location": "Heaven, Not in this World", "create_at": date("2009-07-15"), "description": "Nigerian Igbo | #UMES18 | Stay humbled or be humbled.", "followers_count": 2670, "friends_count": 1484, "statues_count": 26317 }, "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-05T10:00:16.000Z"), "id": 684434252692234240, "text": "Due to this weather, I will now be watching Netflix until I have to go back to work ������☔️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3656134285, "name": "Kerianne Pawell", "screen_name": "KeriannePawell", "lang": "en", "location": "Yorba Linda, CA", "create_at": date("2015-09-22"), "description": "just being me", "followers_count": 131, "friends_count": 219, "statues_count": 502 }, "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-05T10:00:16.000Z"), "id": 684434252750958592, "text": "Shout out to terry this girl got a black berry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 501109392, "name": "LordFlvckoGoldchainz", "screen_name": "Trent_Flemons", "lang": "en", "location": "null", "create_at": date("2012-02-23"), "description": "You won't rise to your occasion you'll fall to your level of training #FARWG young free n wildin", "followers_count": 858, "friends_count": 691, "statues_count": 23445 }, "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-05T10:00:16.000Z"), "id": 684434252776083456, "text": "Partly cloudy this afternoon, high 44 (7 C). Low 28 (-2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575444, "name": "Little Rock Weather", "screen_name": "LittleRockWX", "lang": "en", "location": "Little Rock, AR", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Little Rock, AR", "followers_count": 1834, "friends_count": 92, "statues_count": 7895 }, "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-05T10:00:16.000Z"), "id": 684434252776095744, "text": "Check out Kerry Hefner's listing in #Lincolnton #NC https://t.co/z2hMwar5A6 #realestate #realtor https://t.co/hNvNo7Qa4S", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.1281525,35.5382515"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Lincolnton", "NC", "realestate", "realtor" }}, "user": { "id": 2445654229, "name": "RE/MAX Crossroads", "screen_name": "lincolntonremax", "lang": "en", "location": "Lincolnton NC", "create_at": date("2014-04-15"), "description": "null", "followers_count": 71, "friends_count": 43, "statues_count": 6131 }, "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.32187,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37109, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-05T10:00:16.000Z"), "id": 684434252851621888, "text": "Can you recommend anyone for this #Healthcare #job? https://t.co/cwPfQkj8UG #ORjobs #Monterey, CA #Hiring https://t.co/WbKI4JL6Vq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.9131935,36.5787155"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "ORjobs", "Monterey", "Hiring" }}, "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": 55, "friends_count": 27, "statues_count": 50 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monterey, CA", "id": "49af5b43d4963f4c", "name": "Monterey", "place_type": "city", "bounding_box": rectangle("-121.927024,36.573741 -121.830974,36.618826") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 648872, "cityName": "Monterey" } }
+{ "create_at": datetime("2016-01-05T10:00:16.000Z"), "id": 684434253212446725, "text": "Why is everyone being stupid today ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 358542686, "name": "Romana.", "screen_name": "shortcake101_", "lang": "en", "location": "null", "create_at": date("2011-08-19"), "description": "MM❤️", "followers_count": 972, "friends_count": 823, "statues_count": 50350 }, "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-05T10:00:16.000Z"), "id": 684434253292122112, "text": "and they all legit stared at us till we got to our seat and some continued to stare", "in_reply_to_status": 684434072144330753, "in_reply_to_user": 372016097, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 372016097, "name": "susie carmichael", "screen_name": "sheabuttaqueen", "lang": "en", "location": "thickthighnation", "create_at": date("2011-09-11"), "description": "well, my name is susie and gucci think i love him", "followers_count": 919, "friends_count": 650, "statues_count": 82533 }, "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-01-05T10:00:16.000Z"), "id": 684434253342347264, "text": "Want to work at Macy's Retail? We're #hiring in #Atlanta, GA! Click for details: https://t.co/SEcPl9UgnJ #Finance #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.3626569,33.8464688"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Atlanta", "Finance", "Job", "Jobs", "CareerArc" }}, "user": { "id": 23215449, "name": "Atlanta Finance Jobs", "screen_name": "tmj_atl_finance", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-03-07"), "description": "Follow this account for geo-targeted Finance job tweets in Atlanta, GA. Need help? Tweet us at @CareerArc!", "followers_count": 698, "friends_count": 274, "statues_count": 112 }, "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-05T10:00:16.000Z"), "id": 684434253560455169, "text": "@Newegg @UPS @USPS this is how products MY LAPTOP is shipped? Really!!!??? https://t.co/BsuCas0y16", "in_reply_to_status": -1, "in_reply_to_user": 17221819, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17221819, 158129535, 386507775 }}, "user": { "id": 21321735, "name": "Ryan Light", "screen_name": "RealRunRyan", "lang": "en", "location": "NY'er frm Seattle livin in ATL", "create_at": date("2009-02-19"), "description": "Not Running Sucks! |#OCD is a battle I'm willing to fight | Inspire others to live a healthy fit lifestyle | 50xHalf Marathons 4xMarathons Countless 10/5Ks", "followers_count": 58076, "friends_count": 7871, "statues_count": 15093 }, "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-05T10:00:16.000Z"), "id": 684434253598216193, "text": "@shelbster_4 @Lil_danz777 https://t.co/9TEYi0F0Pq", "in_reply_to_status": -1, "in_reply_to_user": 30309886, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 30309886, 421371998 }}, "user": { "id": 280303598, "name": "Hannah Johnson", "screen_name": "hanjohnn", "lang": "en", "location": "null", "create_at": date("2011-04-10"), "description": "Ralph Cox, from wherevers not gonna get me hit.", "followers_count": 370, "friends_count": 498, "statues_count": 8620 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin, MI", "id": "3bb7a2dc038a970d", "name": "Franklin", "place_type": "city", "bounding_box": rectangle("-83.320966,42.507065 -83.28516,42.530161") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2630340, "cityName": "Franklin" } }
+{ "create_at": datetime("2016-01-05T10:00:16.000Z"), "id": 684434253665439746, "text": "If I don't know you, you're unfollowed.", "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": "The Soul God.", "screen_name": "MadamBoatswain", "lang": "en", "location": "SomePlace.", "create_at": date("2014-05-13"), "description": "Senior Social Work Student at North Carolina Agricultural and Technical State University, the #1 HBCU on this planet. AGGIE PRIDE", "followers_count": 936, "friends_count": 681, "statues_count": 73420 }, "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-05T10:00:16.000Z"), "id": 684434253879181312, "text": "When you ghetto AF & bag your belongs in plastic so the rain doesn't mess them up inside your purse ������ #NoShame https://t.co/rVNLvspmUS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NoShame" }}, "user": { "id": 25538545, "name": "Lupe Looove", "screen_name": "lupelooove", "lang": "en", "location": "Los Angeles | 818 ", "create_at": date("2009-03-20"), "description": "•CSUN Alumna •Radio Junkie •Entertainment Journalist •Contributor for http://KarenCivil.com & http://RnBass.com + more ✨ #xoTribe LupeLoooveLA@gmail.com", "followers_count": 1101, "friends_count": 464, "statues_count": 33821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:17.000Z"), "id": 684434254504173568, "text": "Police all into me, this ain't 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": 3875646552, "name": "Yeezy", "screen_name": "Lucascox27", "lang": "en", "location": "DSMA", "create_at": date("2015-10-12"), "description": "Genius", "followers_count": 102, "friends_count": 142, "statues_count": 947 }, "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-05T10:00:17.000Z"), "id": 684434255246680065, "text": "This sounds quite amazing to my ears (music as it were) > https://t.co/0UQNUtpyCc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15273924, "name": "IHeartNoise / Ilya S", "screen_name": "Iheartnoise", "lang": "en", "location": "Boston", "create_at": date("2008-06-29"), "description": "Trying to grapple w/my Wronglish. Small label (@skyjelly @andavolley @toypianoband @lightningpill). Famed rock critic @pitchfork / @_sleeveface_", "followers_count": 2589, "friends_count": 2488, "statues_count": 40002 }, "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-05T10:00:17.000Z"), "id": 684434255263301633, "text": "#PolandIsReadyForRevivalTour 33", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "PolandIsReadyForRevivalTour" }}, "user": { "id": 1691377656, "name": "Justin my Angel", "screen_name": "AngelaDziedzic1", "lang": "pl", "location": "Chicago, IL", "create_at": date("2013-08-22"), "description": "I fell in love with a 15 year old guy who dreams fulfilled now I love 21-year-old man,who helps fulfill the dreams of other!18/11/15 Thank You Justin PotterHead", "followers_count": 325, "friends_count": 808, "statues_count": 2603 }, "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-01-05T10:00:17.000Z"), "id": 684434255292837888, "text": "Chipotle. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209633365, "name": "HIS MAJESTY", "screen_name": "CMXVIII", "lang": "en", "location": "null", "create_at": date("2010-10-29"), "description": "Living For Accomplishments Not Compliments. #BusinessConsultant #FinancialServiceRep #REAgent", "followers_count": 767, "friends_count": 482, "statues_count": 24075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodlawn, MD", "id": "bf64a5dd9b5f1cf2", "name": "Woodlawn", "place_type": "city", "bounding_box": rectangle("-76.915396,38.938828 -76.711092,39.327477") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2486525, "cityName": "Woodlawn" } }
+{ "create_at": datetime("2016-01-05T10:00:17.000Z"), "id": 684434255401779200, "text": "Partly cloudy/wind this afternoon, high 41 (5 C). Low 31 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26195145, "name": "Wichita Weather", "screen_name": "WichitaWeather", "lang": "en", "location": "Wichita, KS", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Wichita, KS", "followers_count": 1849, "friends_count": 88, "statues_count": 7750 }, "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-05T10:00:17.000Z"), "id": 684434255531753473, "text": "@mermaidashley98 @pinknippple https://t.co/COhpRWH3t4", "in_reply_to_status": -1, "in_reply_to_user": 86857044, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 86857044, 2421229362 }}, "user": { "id": 175952730, "name": "gato", "screen_name": "kitzzkat_", "lang": "en", "location": "null", "create_at": date("2010-08-07"), "description": "this is our fate", "followers_count": 676, "friends_count": 633, "statues_count": 16844 }, "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-05T10:00:17.000Z"), "id": 684434255770972160, "text": "Thank you Bob and I can respect the fact that you don't believe without trying to change you. Easy isn't it? https://t.co/PTwysd66eJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1732121486, "name": "Marty", "screen_name": "hopingforachang", "lang": "en", "location": "America", "create_at": date("2013-09-05"), "description": "I am a proud conservative American. I am happy to engage with those willing to discuss ideas openly and honestly. Trolls with agendas: Proceed at your own risk.", "followers_count": 2381, "friends_count": 977, "statues_count": 152021 }, "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-05T10:00:17.000Z"), "id": 684434255825342464, "text": "@Bsowell11 what Doctor uses guns to save lives?Doctors save more lives than they harm.Other than law enforcement, guns don't save many lives", "in_reply_to_status": 684433777779589120, "in_reply_to_user": 389213424, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 389213424 }}, "user": { "id": 535887316, "name": "Austin Heitzinger", "screen_name": "A_Heitz13", "lang": "en", "location": "UCO '17 - ΠΚΑ", "create_at": date("2012-03-24"), "description": "Don't follow me because I have no idea where I'm going. And I'm a man, so I won't ask for directions #BlessUp", "followers_count": 1001, "friends_count": 697, "statues_count": 8360 }, "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-05T10:00:17.000Z"), "id": 684434256559390720, "text": "@EmileEnnisJr YASS https://t.co/vVG7D5JDJj", "in_reply_to_status": -1, "in_reply_to_user": 51530353, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 51530353 }}, "user": { "id": 28017585, "name": "Krista", "screen_name": "lilaznangel16", "lang": "en", "location": "California, USA", "create_at": date("2009-03-31"), "description": "... are you gonna eat that?", "followers_count": 194, "friends_count": 240, "statues_count": 24958 }, "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-05T10:00:17.000Z"), "id": 684434256672718848, "text": "@gavinlindemuth me", "in_reply_to_status": 684433680736104448, "in_reply_to_user": 122741246, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 122741246 }}, "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": 6048, "friends_count": 889, "statues_count": 83747 }, "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-05T10:00:17.000Z"), "id": 684434256899096576, "text": "on our side", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2935420082, "name": "Jared", "screen_name": "jaredcheeseman", "lang": "en", "location": "Bakersfield, CA", "create_at": date("2014-12-18"), "description": "Try it now asshole. (bro helped)", "followers_count": 250, "friends_count": 953, "statues_count": 8702 }, "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-05T10:00:17.000Z"), "id": 684434256957812736, "text": "Temp 37.1°F Rising, Pressure 30.547in Falling, Dew point 23.7°, Wind SE 6mph, Rain today 0.00in https://t.co/gPtnr2dDtV #STLwx #MOwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.35555556,38.47694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "STLwx", "MOwx" }}, "user": { "id": 1191304230, "name": "Oakville Weather", "screen_name": "OakvilleWx", "lang": "en", "location": "Oakville, Missouri, USA", "create_at": date("2013-02-17"), "description": "Current conditions on the hour brought to you by @DaveStLou", "followers_count": 105, "friends_count": 80, "statues_count": 24630 }, "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-01-05T10:00:17.000Z"), "id": 684434256978964480, "text": "@mikayla_lanay lmao seriously & I literally just opened my eyes", "in_reply_to_status": 684433827150823424, "in_reply_to_user": 2442417169, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2442417169 }}, "user": { "id": 1033396430, "name": "bambi.", "screen_name": "_liljass", "lang": "en", "location": "Huntington, WV", "create_at": date("2012-12-24"), "description": "Know Your Worth | MAG", "followers_count": 1865, "friends_count": 1168, "statues_count": 98714 }, "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-05T10:00:17.000Z"), "id": 684434257226285056, "text": "@lahjaee6 well she better get yuh before I hurt yuh ��������", "in_reply_to_status": -1, "in_reply_to_user": 2265525651, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2265525651 }}, "user": { "id": 2866218661, "name": "➰♋️P R O D I G Y♋️➰", "screen_name": "TTrelzzz", "lang": "en", "location": "Donaldsonville, LA™", "create_at": date("2014-10-19"), "description": "\\\\ ѕиαρ¢нαт: тяєℓℓ_вα∂αχχ // \\\\ fяєαк тяєℓℓ fяєαк тяєℓℓ // Goldie was here❣", "followers_count": 243, "friends_count": 191, "statues_count": 2662 }, "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-05T10:00:17.000Z"), "id": 684434257352232960, "text": "Doing big things w babbee :) #Exciting", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Exciting" }}, "user": { "id": 366389268, "name": "Meagan Tkachuk ∞", "screen_name": "MeGs_N_KeGs", "lang": "en", "location": "null", "create_at": date("2011-09-01"), "description": "future Registered Nurse.student at MCPHS University..Cosmetolgist..Pharmacy Technician ..Country Music Lover.Boston Bruins fanatic..taken 12/25 CH ❤", "followers_count": 473, "friends_count": 845, "statues_count": 41352 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Peabody, MA", "id": "facb653464828079", "name": "Peabody", "place_type": "city", "bounding_box": rectangle("-71.034586,42.494855 -70.912696,42.570296") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25009, "countyName": "Essex", "cityID": 2552490, "cityName": "Peabody" } }
+{ "create_at": datetime("2016-01-05T10:00:17.000Z"), "id": 684434257553526785, "text": "Emily had my blanket in her room to cover the window so I went in and took it & I forgot she had work at 530am & now I feel bad lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2148973725, "name": "Jordan Belfort", "screen_name": "jake_valle22", "lang": "en", "location": "NJ/DE", "create_at": date("2013-10-22"), "description": "UCA College National Champion", "followers_count": 450, "friends_count": 554, "statues_count": 10067 }, "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-01-05T10:00:17.000Z"), "id": 684434257985458176, "text": "Partly cloudy this afternoon, high 35 (2 C). Low 29 (-2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26199170, "name": "Omaha Weather", "screen_name": "OmahaWX", "lang": "en", "location": "Omaha, Nebraska", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Omaha, Nebraska", "followers_count": 2589, "friends_count": 88, "statues_count": 7743 }, "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-05T10:00:17.000Z"), "id": 684434258090266631, "text": "Just made the hottest beat", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2931691462, "name": "Jake", "screen_name": "jake4l", "lang": "en", "location": "null", "create_at": date("2014-12-18"), "description": "Tempe/Chicago jokes.", "followers_count": 311, "friends_count": 668, "statues_count": 9319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntley, IL", "id": "40801c91ed89c83c", "name": "Huntley", "place_type": "city", "bounding_box": rectangle("-88.481796,42.120368 -88.388485,42.204813") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17111, "countyName": "McHenry", "cityID": 1736750, "cityName": "Huntley" } }
+{ "create_at": datetime("2016-01-05T10:00:18.000Z"), "id": 684434258421755905, "text": "@NRO @FoundersGirl I think he just had a mental breakdown,to much pressure.", "in_reply_to_status": 684430448160669696, "in_reply_to_user": 19417492, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19417492, 887905129 }}, "user": { "id": 2205261256, "name": "vtul9321", "screen_name": "vtul7321", "lang": "en", "location": "New York ", "create_at": date("2013-12-02"), "description": "Father,grandfather,husband,retired,trying to give back ,NRA. Blue Lives Matter,All Lives Matter. Go Donald. Trump Train ,supports military,vets first", "followers_count": 4060, "friends_count": 4317, "statues_count": 8234 }, "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-01-05T10:00:18.000Z"), "id": 684434258736238592, "text": "Highly considering going back to counseling. This depression is 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": 509071982, "name": "HEYWARD LACKEY! #CHI", "screen_name": "vivalaraza92", "lang": "en", "location": "null", "create_at": date("2012-02-29"), "description": "#GSarmy", "followers_count": 386, "friends_count": 1029, "statues_count": 7168 }, "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-05T10:00:18.000Z"), "id": 684434259000442881, "text": "|| I want to go on dates and meet more people☺️����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 503473715, "name": "kendalena thompson", "screen_name": "kendalenathomps", "lang": "en", "location": "Honolulu, HI", "create_at": date("2012-02-25"), "description": "|| kendalenathompson@gmail.com RIP CAMERON❤️ Back at it |Modeling|Palm springs to Oroville ca☀️✈️ kendalenathompson98@gmail.com", "followers_count": 1404, "friends_count": 536, "statues_count": 4578 }, "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-05T10:00:18.000Z"), "id": 684434259034116096, "text": "A few of the @charchristiannc high school students interested in careers in various sports industries. https://t.co/4Y6uFGCGqz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 346572786 }}, "user": { "id": 222985620, "name": "Karen Goins-Byrnes", "screen_name": "KarenByrnes", "lang": "en", "location": "Charlotte, NC", "create_at": date("2010-12-04"), "description": "Christian. Mom. Volunteer. Extrovert. Planner. Adventurer. Traveler. Organizer. Football Enthusiast. Semi-retired Sports Exec. & Business Owner.", "followers_count": 5114, "friends_count": 208, "statues_count": 775 }, "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-05T10:00:18.000Z"), "id": 684434259420012544, "text": "I'm at Massage Green Troy Mi in Troy, MI https://t.co/xCVfxgZeOV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.13895159,42.56307845"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16976267, "name": "Erwin Coronel", "screen_name": "CreeperCK", "lang": "en", "location": "null", "create_at": date("2008-10-25"), "description": "Martial Arts. Video Games. Comic Books. DETROIT. My wife is @lindsay13garf and she's... pretty cool! Also, I'm the dad of #GabriellaGrace", "followers_count": 337, "friends_count": 869, "statues_count": 12993 }, "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-01-05T10:00:18.000Z"), "id": 684434259478736897, "text": "@2toneDL I do love it, so hopefully!", "in_reply_to_status": 684432390018732032, "in_reply_to_user": 1906451173, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1906451173 }}, "user": { "id": 40960018, "name": "pamela marie", "screen_name": "pwnela", "lang": "en", "location": "Atlanta", "create_at": date("2009-05-18"), "description": "Software Development Manager @MailChimp; @railsgirlsatl cheerleader; infrequent conference speaker; gif consultant; #ahabconf keynoter; pilot's wife.", "followers_count": 1266, "friends_count": 356, "statues_count": 14257 }, "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-05T10:00:18.000Z"), "id": 684434259990450177, "text": "@macdizzly hmmmm, okay Michi lol. ������", "in_reply_to_status": 684434186548166656, "in_reply_to_user": 3417822394, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 3417822394 }}, "user": { "id": 2344358196, "name": "jas ☪", "screen_name": "xo_jasmineeeeee", "lang": "en", "location": "diRty ", "create_at": date("2014-02-14"), "description": "RHS junior | 17", "followers_count": 794, "friends_count": 734, "statues_count": 18284 }, "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-01-05T10:00:18.000Z"), "id": 684434260040650752, "text": "Pickup soccer game? Tommorow morning @GuilloryCade @NickCashio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1592898619, 560013706 }}, "user": { "id": 1602457974, "name": "B. Frazier", "screen_name": "BMFrazier17", "lang": "en", "location": "null", "create_at": date("2013-07-17"), "description": "Out|SideThe|Box ///Strictly Business", "followers_count": 798, "friends_count": 564, "statues_count": 7213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Watson, LA", "id": "00d63c200626178c", "name": "Watson", "place_type": "city", "bounding_box": rectangle("-90.974696,30.559256 -90.902126,30.606555") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22063, "countyName": "Livingston", "cityID": 2280010, "cityName": "Watson" } }
+{ "create_at": datetime("2016-01-05T10:00:18.000Z"), "id": 684434260200046593, "text": "Fuck you rain ���� https://t.co/tPp9tJpCOj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46479143, "name": "Alex Cortez", "screen_name": "ohsnapjunior", "lang": "en", "location": "I Make Youtube Videos (: ", "create_at": date("2009-06-11"), "description": "Follow Me! Im 20 From Los Angeles.", "followers_count": 2889, "friends_count": 1837, "statues_count": 5966 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:18.000Z"), "id": 684434260216786945, "text": "I hope we just go over the syllabus and leave", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 300585205, "name": "TheGirlWiththeTattoo", "screen_name": "BitchNastyDoe", "lang": "en", "location": "Hayward,CA", "create_at": date("2011-05-17"), "description": "I'm not ugly, I'm just eating my pizza and doing my thang #ΚΘΣ", "followers_count": 290, "friends_count": 417, "statues_count": 16444 }, "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-05T10:00:18.000Z"), "id": 684434260569149440, "text": "We're #hiring! Read about our latest #job opening here: Weekday Team - https://t.co/C72RNByTT7 #Jonesboro, AR #Retail", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.704279,35.8422967"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Jonesboro", "Retail" }}, "user": { "id": 59696117, "name": "TMJ-AR Retail Jobs", "screen_name": "tmj_ar_retail", "lang": "en", "location": "Arkansas", "create_at": date("2009-07-23"), "description": "Follow this account for geo-targeted Retail job tweets in Arkansas Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 429, "friends_count": 300, "statues_count": 517 }, "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-01-05T10:00:18.000Z"), "id": 684434260581732352, "text": "Sunny this afternoon, high 40 (4 C). Low 27 (-3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433220, "name": "St. Louis Weather", "screen_name": "StLouisWeather", "lang": "en", "location": "St. Louis Metro Area", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for St. Louis Metro Area", "followers_count": 3677, "friends_count": 88, "statues_count": 7716 }, "place": { "country": "United States", "country_code": "United States", "full_name": "St. Louis, MO", "id": "60e6df5778ff9dac", "name": "St. Louis", "place_type": "city", "bounding_box": rectangle("-90.320515,38.531852 -90.166409,38.774346") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29510, "countyName": "St. Louis", "cityID": 2965000, "cityName": "St. Louis" } }
+{ "create_at": datetime("2016-01-05T10:00:18.000Z"), "id": 684434260808187905, "text": "I just want Sweeto Burrito or Spicy Pie nachos", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2955366290, "name": "Dan Helmen", "screen_name": "dhelmen", "lang": "en", "location": "Fargo, ND//Twin Cities, MN.", "create_at": date("2015-01-01"), "description": "23. Dogs. Music. Resilience. Faith.", "followers_count": 103, "friends_count": 76, "statues_count": 2649 }, "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-05T10:00:18.000Z"), "id": 684434260812431360, "text": "Waiting on Chem 162 in Building 35 https://t.co/wME8RwKZY5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.930509,47.022599"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1493947285, "name": "Sandy Truong", "screen_name": "koolsprite34", "lang": "en", "location": "Washington", "create_at": date("2013-06-08"), "description": "A mommy, blogger, poet, and many other things. \nWest coast best coast!\nFollow me on instagram! asiancutie34", "followers_count": 14, "friends_count": 59, "statues_count": 94 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Puget Sound Community College", "id": "07d9c9655fc85001", "name": "South Puget Sound Community College", "place_type": "poi", "bounding_box": rectangle("-122.9305091,47.0225989 -122.930509,47.022599") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53067, "countyName": "Thurston", "cityID": 5372905, "cityName": "Tumwater" } }
+{ "create_at": datetime("2016-01-05T10:00:18.000Z"), "id": 684434260988674049, "text": "@haydenhbird major worry https://t.co/pr0GQ0w6En", "in_reply_to_status": 684416691300876288, "in_reply_to_user": 306284052, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 306284052 }}, "user": { "id": 268888121, "name": "amadí tídíane thiam", "screen_name": "amadoit", "lang": "en", "location": " 908 | +011 | 617 ", "create_at": date("2011-03-19"), "description": "social media manager @WorldSoccerTalk | editor + staff writer @ChelseaIndex | views mine, my own, precious | http://worldsoccertalk.com/author/amaditidiane", "followers_count": 1369, "friends_count": 97, "statues_count": 42418 }, "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-05T10:00:18.000Z"), "id": 684434261059960832, "text": "@ESmith59 yea but coaches / gms can get extensions and then fired. Look at ruff and Nolan. With that said I don't believe TBN article", "in_reply_to_status": 684430570529636352, "in_reply_to_user": 271514371, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 271514371 }}, "user": { "id": 59675080, "name": "Christopher Ryan", "screen_name": "Polo1616", "lang": "en", "location": "1 of the original 13 colonies", "create_at": date("2009-07-23"), "description": "something clever, blah, blah, blah...", "followers_count": 151, "friends_count": 591, "statues_count": 5424 }, "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-05T10:00:18.000Z"), "id": 684434261080981504, "text": "Be stronger than your excuses... https://t.co/vw0nsm0TmY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 234908237, "name": "Marie", "screen_name": "dmarie43ny", "lang": "en", "location": "NY", "create_at": date("2011-01-06"), "description": "Sassy, Classy Broad @ CubeOne * Healthcare & HR Chick (PHR, SHRM-CP) * The best is yet to come... #cubeoneproblems", "followers_count": 640, "friends_count": 1959, "statues_count": 2066 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawthorne, NY", "id": "8788aa1675cd9146", "name": "Hawthorne", "place_type": "city", "bounding_box": rectangle("-73.811171,41.094566 -73.77392,41.114014") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3632842, "cityName": "Hawthorne" } }
+{ "create_at": datetime("2016-01-05T10:00:18.000Z"), "id": 684434261252820992, "text": "#Hospitality #Job in #Stanford, CA: Busser - Dishwasher at Guckenheimer https://t.co/7jjBBiWEEp #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.1660756,37.424106"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "Job", "Stanford", "Jobs", "Hiring" }}, "user": { "id": 731027906, "name": "Guckenheimer Careers", "screen_name": "GuckenheimerJob", "lang": "en", "location": "null", "create_at": date("2012-08-01"), "description": "We are a high end food service company providing restaurant quality food in corporate settings. We are looking to fill a variety of positions across the US.", "followers_count": 138, "friends_count": 2, "statues_count": 298 }, "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.406455 -122.149162,37.443731") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 673906, "cityName": "Stanford" } }
+{ "create_at": datetime("2016-01-05T10:00:18.000Z"), "id": 684434261315719168, "text": "girl same lol https://t.co/Gf1ohkCXAI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 917635135, "name": "yessi", "screen_name": "yeseniacuelllar", "lang": "en", "location": "null", "create_at": date("2012-10-31"), "description": "Join n da essence of da cool out breed", "followers_count": 761, "friends_count": 393, "statues_count": 21482 }, "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-01-05T10:00:18.000Z"), "id": 684434261798027264, "text": "@ValeriePerez58 YAAAAAY FOR DISNEY / MAMÍ DAY !!! ������", "in_reply_to_status": 684434097041575936, "in_reply_to_user": 1646341878, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 1646341878 }}, "user": { "id": 1210325276, "name": "ayelin", "screen_name": "whtsilean", "lang": "en", "location": "Disneyland ", "create_at": date("2013-02-22"), "description": "null", "followers_count": 393, "friends_count": 281, "statues_count": 35635 }, "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-05T10:00:18.000Z"), "id": 684434261802381313, "text": "@MszHunniiee oh they grew? ��������", "in_reply_to_status": 684434181594726400, "in_reply_to_user": 19770597, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19770597 }}, "user": { "id": 2614264554, "name": "CREEP", "screen_name": "CROMO____", "lang": "en", "location": "JERSEY ", "create_at": date("2014-07-09"), "description": "Everybody runs game just peep it don't let them game you, dont give them what they expect, they all find a reason to hate you", "followers_count": 1367, "friends_count": 1409, "statues_count": 70733 }, "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-05T10:00:18.000Z"), "id": 684434262011973632, "text": "Want to work in #HighPoint, NC? View our latest opening: https://t.co/D47g7Cltph #BusinessMgmt #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.0423151,35.9679365"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HighPoint", "BusinessMgmt", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 27642121, "name": "TMJ-NCT Mgmt. Jobs", "screen_name": "tmj_nct_mgmt", "lang": "en", "location": "Triad Area, NC", "create_at": date("2009-03-30"), "description": "Follow this account for geo-targeted Business/Mgmt. job tweets in Triad Area, NC from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 303, "friends_count": 254, "statues_count": 33 }, "place": { "country": "United States", "country_code": "United States", "full_name": "High Point, NC", "id": "aef8c3da277ca498", "name": "High Point", "place_type": "city", "bounding_box": rectangle("-80.073291,35.911849 -79.917426,36.07826") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3731400, "cityName": "High Point" } }
+{ "create_at": datetime("2016-01-05T10:00:18.000Z"), "id": 684434262041432064, "text": "@BradLitt22 @AdamSchefter F off stiff your a troll fire starter pay attention to your own", "in_reply_to_status": 684410126002819072, "in_reply_to_user": 2852357394, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2852357394, 51263592 }}, "user": { "id": 898216435, "name": "Charles Blackburn II", "screen_name": "RomperStomper0", "lang": "en", "location": "Fairview Park, Ohio", "create_at": date("2012-10-22"), "description": "Healthcare Navigator @ Cuyahoga Health Access Partnership, USMC/USCG Veteran, And proud to call CLEVELAND my home", "followers_count": 75, "friends_count": 829, "statues_count": 4851 }, "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-05T10:00:18.000Z"), "id": 684434262292955138, "text": "This Subaru Impreza came in for our Interior Slate to get a refresh. Few stains here and there.… https://t.co/EHBeydxcLy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.0910568,29.9711494"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 224469934, "name": "Brandon Henry", "screen_name": "bhenrysr", "lang": "en", "location": "NOLA", "create_at": date("2010-12-08"), "description": "null", "followers_count": 42, "friends_count": 221, "statues_count": 356 }, "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-05T10:00:19.000Z"), "id": 684434262532083712, "text": "I would do anything for you to 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": 306736857, "name": "Cacia Anderson", "screen_name": "LovelyGirl1025", "lang": "en", "location": "null", "create_at": date("2011-05-28"), "description": "I have a thing for ferry boats.", "followers_count": 270, "friends_count": 338, "statues_count": 3335 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gardner, KS", "id": "00f1f5014dc531da", "name": "Gardner", "place_type": "city", "bounding_box": rectangle("-94.964408,38.788889 -94.871212,38.855976") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20091, "countyName": "Johnson", "cityID": 2025425, "cityName": "Gardner" } }
+{ "create_at": datetime("2016-01-05T10:00:19.000Z"), "id": 684434262796431360, "text": "Get CPR, AED, and First Aid certified at the SRC! Through these special classes you can get… https://t.co/QdktIzdb6J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.52503867,34.2397923"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 312331858, "name": "CSUN SRC", "screen_name": "CSUN_SRC", "lang": "en", "location": "Northridge, CA", "create_at": date("2011-06-06"), "description": "The Student Recreation Center is a 100,000 square foot facility for exercise & leisure activity that promotes lifelong health and wellness. src.csun.edu", "followers_count": 2126, "friends_count": 1260, "statues_count": 3921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:19.000Z"), "id": 684434262821486592, "text": "@hungrytien like hell I will. I have a retreat on my birthday weekend.", "in_reply_to_status": 684434145334935552, "in_reply_to_user": 75254549, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 75254549 }}, "user": { "id": 65566237, "name": "Alyssa", "screen_name": "ae221", "lang": "en", "location": "Dallas, Texas", "create_at": date("2009-08-13"), "description": "Graphic designer based in DFW.", "followers_count": 164, "friends_count": 333, "statues_count": 15502 }, "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-05T10:00:19.000Z"), "id": 684434262842425345, "text": "AI went to work last night", "in_reply_to_status": 684251904147210240, "in_reply_to_user": 88808904, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 88808904, "name": "Nailuj", "screen_name": "DJELZEE", "lang": "en", "location": "609, NJ", "create_at": date("2009-11-09"), "description": "mi casa su casa my house is a hoe house sc: ellzy1993 #Kean #dormshaker j.monet", "followers_count": 1245, "friends_count": 656, "statues_count": 56164 }, "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-01-05T10:00:19.000Z"), "id": 684434262859354112, "text": "Eu sai pra comprar comida tarra com preguiça de me vestir fui de shorts e TA FRIO PRA CARALHO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 151604442, "name": "Haus of Tacacá", "screen_name": "HollaTaio", "lang": "en", "location": "null", "create_at": date("2010-06-03"), "description": "Made in Brazil and exported to the world. Born nigrinha. You may see my bedroom as a bedroom but actually it's my tomb.", "followers_count": 88, "friends_count": 54, "statues_count": 11572 }, "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-05T10:00:19.000Z"), "id": 684434262863515648, "text": "Wofford doing the same. Smart move by both schools ... https://t.co/0mFVxLlQr6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 443329978, "name": "Pee Dee Sports Daily", "screen_name": "PDSPORTSDAILY", "lang": "en", "location": "Florence, South Carolina", "create_at": date("2011-12-21"), "description": "Live and local sports talk weekday afternoons from 4-6pm. Join Chris, Ben, and Dirty to discuss your favorite teams. Listen live at 100.1 FM.", "followers_count": 997, "friends_count": 2019, "statues_count": 10775 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florence, SC", "id": "d49251f0d0a2652d", "name": "Florence", "place_type": "city", "bounding_box": rectangle("-79.918523,34.110516 -79.642093,34.254807") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45041, "countyName": "Florence", "cityID": 4525810, "cityName": "Florence" } }
+{ "create_at": datetime("2016-01-05T10:00:19.000Z"), "id": 684434262938882048, "text": "A6. I like to check often to optimize & break out where ever possible. #ppcchat", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ppcchat" }}, "user": { "id": 3094579765, "name": "Orlando Valencia", "screen_name": "ValenciaSEM", "lang": "en", "location": "Sunnyvale, CA", "create_at": date("2015-03-17"), "description": "SEM Manager @SimplyHired\n| Outdoorsman | Digital Marketing Consultant | Traveler |", "followers_count": 247, "friends_count": 381, "statues_count": 348 }, "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-05T10:00:19.000Z"), "id": 684434262968369152, "text": "@DataAsRisk o", "in_reply_to_status": 684434208220143616, "in_reply_to_user": 100122237, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 100122237 }}, "user": { "id": 27190644, "name": "つ ◕ _◕ ༽つ Zach", "screen_name": "Zach_Baum", "lang": "en", "location": "New York, NY", "create_at": date("2009-03-27"), "description": "WARNING: This Account is Canon,\n\nChapter 3 of the Hoopz Barkley SaGa", "followers_count": 644, "friends_count": 2010, "statues_count": 20435 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Union Square, Manhattan", "id": "68a6c55e55d5acc3", "name": "Union Square", "place_type": "neighborhood", "bounding_box": rectangle("-73.993613,40.733973 -73.98739,40.737909") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-05T10:00:19.000Z"), "id": 684434263010234369, "text": "https://t.co/PtJmh8ChAv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 404986494, "name": "Corey", "screen_name": "IAintGonnaStunt", "lang": "en", "location": "SC: kendall_7", "create_at": date("2011-11-04"), "description": "Board Sports. Born Bred and Water Fed. Former flag football quarterback.", "followers_count": 836, "friends_count": 268, "statues_count": 29807 }, "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-05T10:00:19.000Z"), "id": 684434263035400192, "text": "Mostly sunny this afternoon, high 40 (4 C). Low 31 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433608, "name": "Kansas City Weather", "screen_name": "KansasCityWX", "lang": "en", "location": "Kansas City Metro Area", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Kansas City Metro Area", "followers_count": 1931, "friends_count": 88, "statues_count": 7836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "ddabe75a8f44fe5a", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.765917,38.827176 -94.385522,39.356662") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-01-05T10:00:19.000Z"), "id": 684434264125906944, "text": "rip to my legs today from leg day today and that conditioning at practice :''''')", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1031301475, "name": "Yaz", "screen_name": "Yaz_zay29", "lang": "en", "location": "CA✈AZ✈OH", "create_at": date("2012-12-23"), "description": "BHS Senior|softball| Walsh University Signee '20", "followers_count": 386, "friends_count": 1311, "statues_count": 3565 }, "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-05T10:00:19.000Z"), "id": 684434264516128768, "text": "@chosen4gr8nesss lol yeah my picture is from last year.", "in_reply_to_status": 684418934997581824, "in_reply_to_user": 3291905311, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3291905311 }}, "user": { "id": 3199006006, "name": "Min. Bill Burroughs", "screen_name": "revbillb_", "lang": "en", "location": "Columbus, OH", "create_at": date("2015-04-23"), "description": "Christ is God. To Him be the glory & honor forever & ever. Acts 6:4, Revelation 3:20", "followers_count": 546, "friends_count": 305, "statues_count": 13937 }, "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-05T10:00:19.000Z"), "id": 684434264948125697, "text": "@ImTheWavyOne brah trust me I know, I had a navy instructor and all he do is lie", "in_reply_to_status": 684433931660308480, "in_reply_to_user": 65972749, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 65972749 }}, "user": { "id": 333060542, "name": "Troy ®", "screen_name": "lilreid_4prez", "lang": "en", "location": "Danville 434 ✈️ Norfolk 7✋7", "create_at": date("2011-07-10"), "description": "#YNC Head of security / #SvG$/ #NICE⛽️/ #ODU/ IG: @lilreid_4president ...FAILURE IS NOT AN OPTION ! ✊", "followers_count": 1085, "friends_count": 967, "statues_count": 37870 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Danville, VA", "id": "a58a872123811d63", "name": "Danville", "place_type": "city", "bounding_box": rectangle("-79.519308,36.540813 -79.317136,36.673579") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51590, "countyName": "Danville", "cityID": 5121344, "cityName": "Danville" } }
+{ "create_at": datetime("2016-01-05T10:00:19.000Z"), "id": 684434265078108160, "text": "@enRICHment__ how do you find out if you're cleared or not ?", "in_reply_to_status": 684430917386006528, "in_reply_to_user": 3235896051, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3235896051 }}, "user": { "id": 1582128474, "name": "Matthew J. Joseph", "screen_name": "urban_ikon", "lang": "en", "location": "Queens, NY", "create_at": date("2013-07-09"), "description": "Men suffer in silence || OakwoodUniversity", "followers_count": 439, "friends_count": 184, "statues_count": 7805 }, "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-05T10:00:19.000Z"), "id": 684434265208008704, "text": "\"Yeah, it's all about loving yourself,\nI never forget where I came from\" https://t.co/x7d1Xcy2id", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 367818055, "name": "javier.", "screen_name": "_JayGarden", "lang": "en", "location": "San Diego, CA", "create_at": date("2011-09-04"), "description": "NASTI #SDCA // King Street // RIP Slim & Eazy Inquiries: jaygarden305@gmail.com", "followers_count": 541, "friends_count": 230, "statues_count": 44627 }, "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-05T10:00:19.000Z"), "id": 684434265556123648, "text": "Between Tearra and Munu they keep me Updated on the Latest Fuckery !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1489591302, "name": "Ash.", "screen_name": "BadAss_Ashhh", "lang": "en", "location": "null", "create_at": date("2013-06-06"), "description": "null", "followers_count": 991, "friends_count": 369, "statues_count": 12319 }, "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-05T10:00:19.000Z"), "id": 684434265585496064, "text": "@hayla_nicole @idknichi you need to sing this song and post it on here ��������", "in_reply_to_status": 680467988273733637, "in_reply_to_user": 1035002508, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1035002508, 1864833517 }}, "user": { "id": 3499318572, "name": "B L A Z E", "screen_name": "bllazzex", "lang": "en", "location": "Hawaii, USA", "create_at": date("2015-09-08"), "description": "stay happy : )", "followers_count": 81, "friends_count": 64, "statues_count": 1650 }, "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-01-05T10:00:19.000Z"), "id": 684434265619087360, "text": "@josieolvera1 @Carlos__996 @anthro96 yes we do Josie", "in_reply_to_status": 684413439205220352, "in_reply_to_user": 2852670608, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2852670608, 742150622, 857518465 }}, "user": { "id": 855809970, "name": "AP", "screen_name": "antonioopadron", "lang": "en", "location": "Texas", "create_at": date("2012-09-30"), "description": "Just a man god invented.", "followers_count": 284, "friends_count": 196, "statues_count": 20952 }, "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-05T10:00:19.000Z"), "id": 684434265640165378, "text": "Im my own man iont hang wit nobody but 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": 63503362, "name": "Uncle Slim", "screen_name": "662_MILLZ", "lang": "en", "location": "FT wayne ", "create_at": date("2009-08-06"), "description": "took alot of chances to get where im at sc:money_millz", "followers_count": 958, "friends_count": 1028, "statues_count": 42284 }, "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-05T10:00:19.000Z"), "id": 684434265807798272, "text": "Sunny this afternoon, high 45 (7 C). Low 31 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433972, "name": "Memphis Weather", "screen_name": "MemphisWX", "lang": "en", "location": "Memphis, TN", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Memphis, TN", "followers_count": 1688, "friends_count": 88, "statues_count": 7748 }, "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-05T10:00:19.000Z"), "id": 684434265984110592, "text": "Been having the dopest dreams ever lately.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 549746161, "name": "Logan Koch", "screen_name": "logankoch44", "lang": "en", "location": "null", "create_at": date("2012-04-09"), "description": "Chill man.", "followers_count": 483, "friends_count": 290, "statues_count": 36180 }, "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-05T10:00:19.000Z"), "id": 684434266600542208, "text": "Bethesda Hospital East #Nursing #Job: RN Float Pool (Internal Registry) Tele-Med-Surg-Onc (#BoyntonBeach, FL) https://t.co/o4B8INtn3Z #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.0697937,26.5047866"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "Job", "BoyntonBeach", "Jobs" }}, "user": { "id": 2615559954, "name": "Bethesda Health Jobs", "screen_name": "BH_Jobs", "lang": "en", "location": "null", "create_at": date("2014-07-10"), "description": "Whether you’re a nurse looking for a more challenging position or a clerical prof. looking for great benefits, you’ll find a rewarding career @ Bethesda Health.", "followers_count": 343, "friends_count": 671, "statues_count": 1762 }, "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-01-05T10:00:20.000Z"), "id": 684434267003211777, "text": "\"Stay Down For The Come Up\" I Don't Be Saying That Shit Too Use My Breath™.��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 450992953, "name": "Rico©.", "screen_name": "lilsmokeD_", "lang": "en", "location": "New Orleans, LA", "create_at": date("2011-12-30"), "description": "null", "followers_count": 1991, "friends_count": 1458, "statues_count": 61326 }, "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-05T10:00:20.000Z"), "id": 684434267045277696, "text": "#911nonEmergency at 6112-6251 Carrier Dr. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4647359,28.4559831"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "911nonEmergency", "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": 275, "friends_count": 1, "statues_count": 67700 }, "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-05T10:00:20.000Z"), "id": 684434267095564289, "text": "Over time you realize that some people aren't worth it anymore.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2275528976, "name": "GLORY_BOY", "screen_name": "kenny_w3st", "lang": "en", "location": "Joliet, IL", "create_at": date("2014-01-03"), "description": "I can accept failure, but I cant accept not trying. ~Michael Jordan", "followers_count": 329, "friends_count": 183, "statues_count": 750 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Romeoville, IL", "id": "1edc79dbea3434b0", "name": "Romeoville", "place_type": "city", "bounding_box": rectangle("-88.163714,41.587343 -88.028539,41.684521") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1765442, "cityName": "Romeoville" } }
+{ "create_at": datetime("2016-01-05T10:00:20.000Z"), "id": 684434267246481408, "text": "This #Labor #job might be a great fit for you: Selector - https://t.co/32vVPP7BrE #Cairo, Georgia #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.2012897,30.8774132"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Labor", "job", "Cairo", "Hiring", "CareerArc" }}, "user": { "id": 117968952, "name": "TMJ-GA Labor Jobs", "screen_name": "tmj_GA_LABOR", "lang": "en", "location": "Georgia", "create_at": date("2010-02-26"), "description": "Follow this account for geo-targeted General Labor job tweets in Georgia Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 335, "friends_count": 285, "statues_count": 99 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cairo, GA", "id": "87eee0c68035bf1e", "name": "Cairo", "place_type": "city", "bounding_box": rectangle("-84.235765,30.845742 -84.169121,30.902098") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13131, "countyName": "Grady", "cityID": 1312400, "cityName": "Cairo" } }
+{ "create_at": datetime("2016-01-05T10:00:20.000Z"), "id": 684434267397459968, "text": "Interested in a #Finance #job near #Indianapolis, IN? This could be a great fit: https://t.co/VkDx4iGgS6 #Hiring https://t.co/5CgYvQTRUQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.1580423,39.7683765"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Finance", "job", "Indianapolis", "Hiring" }}, "user": { "id": 4099603043, "name": "WestPoint Financial", "screen_name": "WestpointF", "lang": "en", "location": "Indianapolis, IN", "create_at": date("2015-11-02"), "description": "Our mission at WestPoint Financial Group is simple. We are dedicated to educating our clients to help them fulfill their financial goals.", "followers_count": 36, "friends_count": 80, "statues_count": 10 }, "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-01-05T10:00:20.000Z"), "id": 684434267456311296, "text": "The EDGE Annual Question was published New Years Eve - 197 essays in response-134,300 words. https://t.co/rsXmxDOso2 https://t.co/fbb1xDLN9w", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15125585, "name": "John Brockman", "screen_name": "edge", "lang": "en", "location": "null", "create_at": date("2008-06-15"), "description": "null", "followers_count": 45319, "friends_count": 105, "statues_count": 209 }, "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-05T10:00:20.000Z"), "id": 684434267569549312, "text": ".@Dart_Adams #truth", "in_reply_to_status": 684431864820547588, "in_reply_to_user": 18521955, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "truth" }}, "user_mentions": {{ 18521955 }}, "user": { "id": 2958066579, "name": "Xperience Creative", "screen_name": "YourXperience", "lang": "en", "location": "Boston, MA", "create_at": date("2015-01-03"), "description": "Entertainment Marketing • Artist Relations • Events The team makes it happen: @only1lisamarie @devinonelove @killerboombox @bedlam_showoff @jassbianchi", "followers_count": 368, "friends_count": 673, "statues_count": 718 }, "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.282413 -71.156939,42.367744") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2545560, "cityName": "Newton" } }
+{ "create_at": datetime("2016-01-05T10:00:20.000Z"), "id": 684434267691089922, "text": "#Labor #Job in #Hattiesburg, MS: Receiver/Stocker at Lowe's https://t.co/4FeUtEPZdk #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.2903392,31.3271189"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Labor", "Job", "Hattiesburg", "Jobs", "Hiring" }}, "user": { "id": 119210988, "name": "TMJ-MS Labor Jobs", "screen_name": "tmj_MS_LABOR", "lang": "en", "location": "Mississippi", "create_at": date("2010-03-02"), "description": "Follow this account for geo-targeted General Labor job tweets in Mississippi Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 296, "friends_count": 277, "statues_count": 24 }, "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-05T10:00:20.000Z"), "id": 684434267896688640, "text": "Would be great if I could find my fucking wallet", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 261805446, "name": "Lord of the Juice", "screen_name": "kodaboi_757", "lang": "en", "location": "Norfolk, VA➡Virginia Beach, VA", "create_at": date("2011-03-06"), "description": "19. Recording Artist. 757. local bitch finesser. 6'3 douchebag. hotline isn't blinging", "followers_count": 1299, "friends_count": 1575, "statues_count": 18076 }, "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-05T10:00:20.000Z"), "id": 684434268022505472, "text": "#ManDown at 490-841 Flower Fields Ln. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3777117,28.3642414"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ManDown", "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": 90, "friends_count": 1, "statues_count": 18150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Southchase, FL", "id": "01b7adb9111f2cd2", "name": "Southchase", "place_type": "city", "bounding_box": rectangle("-81.404746,28.347641 -81.372821,28.40757") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1267270, "cityName": "Southchase" } }
+{ "create_at": datetime("2016-01-05T10:00:20.000Z"), "id": 684434268374855680, "text": "@redroomrantings I found Perry's movies fascinating as a look into his psyche. MADEA'S FAMILY REUNION is the rare one that actually worked", "in_reply_to_status": 684409345858211841, "in_reply_to_user": 78740665, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 78740665 }}, "user": { "id": 382997644, "name": "Brian Schmid", "screen_name": "SchmanthonyP", "lang": "en", "location": "Aurora, CO", "create_at": date("2011-09-30"), "description": "Constantly tweeting. Social justice. Pro wrestling. Pop music. Film. Denver sports. Obsessive Listmaker. #TheCinemasochist. FLEE NOW.", "followers_count": 1006, "friends_count": 833, "statues_count": 237641 }, "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-05T10:00:20.000Z"), "id": 684434268617969664, "text": "Rise of the Click-Thru Novelette. https://t.co/tkfdnEByLH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34339828, "name": "Avery Holton", "screen_name": "averyholton", "lang": "en", "location": "Salt Lake City, UT", "create_at": date("2009-04-22"), "description": "Digital Media Professor @UUtah. @UTJSchool alum. Researching intersections of media, health, and society. Formerly of the @RRExpress.", "followers_count": 1869, "friends_count": 932, "statues_count": 6830 }, "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-05T10:00:20.000Z"), "id": 684434268727066624, "text": "Can you recommend anyone for this #Sales #job? https://t.co/mhFEupWZcc #TruGreen #RapidCity, SD #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-103.2310149,44.0805434"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "job", "TruGreen", "RapidCity", "Hiring" }}, "user": { "id": 2776752523, "name": "TruGreen Jobs", "screen_name": "TruGreenJobs", "lang": "en", "location": "null", "create_at": date("2014-08-28"), "description": "null", "followers_count": 18, "friends_count": 0, "statues_count": 1029 }, "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-05T10:00:20.000Z"), "id": 684434268924329985, "text": "I'm In #Brooklyn This Friday For \nKEVIN AKA @salinfrared \nBday Celebration This Friday \n\nAt Cafe… https://t.co/nbm9uIy6P9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9489441,40.643425"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Brooklyn" }}, "user_mentions": {{ 237866350 }}, "user": { "id": 44799185, "name": "DJ PoLiSH", "screen_name": "DJPoLiSHxL", "lang": "en", "location": "ÜT: 40.673329,-73.766642", "create_at": date("2009-06-04"), "description": "BBM PIN - 7A23238A INSTAGRAM @DJPOLISHXL DJPOLISH@gmail.com http://www.Instagram.com/DJPOLISHXL", "followers_count": 1785, "friends_count": 642, "statues_count": 15596 }, "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-05T10:00:20.000Z"), "id": 684434269008228353, "text": "Special S/O to @coachsenato29 He is a beast! Identifies talent way before anyone even knows about a player! Him and I would be dynamic!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 452436456 }}, "user": { "id": 3392706453, "name": "Mike Carney⚾", "screen_name": "LEMC_INTLSPR24", "lang": "en", "location": "Miami,FL|SanJuan,PR(Lawoffice)", "create_at": date("2015-07-28"), "description": "⚾Scout Atlanta @Braves|@csmbaseball alum|Advisor @SPR_INTL127.| BA Athlete MGMT|New IG @SPR_INTERNATIONAL |Head Coach/Recruiter @cubaprospects ⚾", "followers_count": 1007, "friends_count": 157, "statues_count": 1793 }, "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-05T10:00:20.000Z"), "id": 684434269138104320, "text": "Glad to Give my time volunteering #GladToGive #sweepstakes https://t.co/gSrY6gFcJr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GladToGive", "sweepstakes" }}, "user": { "id": 43279671, "name": "Marilyn S Stephens", "screen_name": "uniongap", "lang": "en", "location": "Kent, Washington", "create_at": date("2009-05-28"), "description": "null", "followers_count": 298, "friends_count": 1276, "statues_count": 13071 }, "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-05T10:00:20.000Z"), "id": 684434269171654657, "text": "Road @ I-70 https://t.co/KPdf4z7ehl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-99.131331,38.87848115"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 38486351, "name": "Francois Coquemont", "screen_name": "KroazDu", "lang": "en", "location": "Breizh+Zion", "create_at": date("2009-05-07"), "description": "Consultant at Adobe Digital Marketing. World traveler. Curious by nature. I run, ski and love my family. Thoughts/opinions my own.\nhttps://t.co/mg3Tgk3Vbq", "followers_count": 389, "friends_count": 362, "statues_count": 5919 }, "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": 20051, "countyName": "Ellis" } }
+{ "create_at": datetime("2016-01-05T10:00:20.000Z"), "id": 684434269247270912, "text": "#BurglaryResidence 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": {{ "BurglaryResidence", "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": 150, "friends_count": 1, "statues_count": 35563 }, "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-05T10:00:20.000Z"), "id": 684434269599485953, "text": "Millard south wassssup", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2502533389, "name": "tea", "screen_name": "tttea123", "lang": "en", "location": "null", "create_at": date("2014-05-17"), "description": "null", "followers_count": 1139, "friends_count": 595, "statues_count": 854 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chalco, NE", "id": "01d7ae2510775a65", "name": "Chalco", "place_type": "city", "bounding_box": rectangle("-96.196696,41.140119 -96.101091,41.19082") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31153, "countyName": "Sarpy", "cityID": 3108640, "cityName": "Chalco" } }
+{ "create_at": datetime("2016-01-05T10:00:20.000Z"), "id": 684434269737844737, "text": "Why do i 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": 4461094752, "name": "patrizia warren", "screen_name": "patriziaawarren", "lang": "en", "location": "null", "create_at": date("2015-12-04"), "description": "pvhs", "followers_count": 97, "friends_count": 108, "statues_count": 241 }, "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-05T10:00:20.000Z"), "id": 684434269796564992, "text": "Why tf is it so cold on legend", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 421231985, "name": "Cader5⃣3⃣", "screen_name": "CadeFries", "lang": "en", "location": "Parker, CO", "create_at": date("2011-11-25"), "description": "Heart over Hype #DolphinGang #FearAmeer", "followers_count": 347, "friends_count": 736, "statues_count": 2360 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Parker, CO", "id": "005d9455753e7700", "name": "Parker", "place_type": "city", "bounding_box": rectangle("-104.813236,39.478431 -104.708485,39.568631") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8035, "countyName": "Douglas", "cityID": 857630, "cityName": "Parker" } }
+{ "create_at": datetime("2016-01-05T10:00:20.000Z"), "id": 684434269805133824, "text": "AJJAJAJAJAJAJAJAJJAJAJAJAJAJAJJAJ https://t.co/Cp2OTuTfJf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 98515672, "name": "SR", "screen_name": "SerRodriguezGAS", "lang": "en", "location": "North Miami Beach, FL", "create_at": date("2009-12-21"), "description": "18 CCS/MIA", "followers_count": 315, "friends_count": 250, "statues_count": 6265 }, "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-01-05T10:00:20.000Z"), "id": 684434269989670912, "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": 1654072015, "name": "hulia☀️", "screen_name": "JuliaHoplight1", "lang": "en", "location": "Ashtabula, OH", "create_at": date("2013-08-07"), "description": "lakeside highschool | RaeShaun♥️", "followers_count": 509, "friends_count": 129, "statues_count": 8105 }, "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-01-05T10:00:20.000Z"), "id": 684434270044041216, "text": "@TracyMcCreery @stlouisbiz Despite the LA weather, his stadium is domed so they can hold non-sporting events. If only we had a dome in STL.", "in_reply_to_status": 684393214954967041, "in_reply_to_user": 462208178, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 462208178, 194235592 }}, "user": { "id": 1924400334, "name": "Shamed Dogan", "screen_name": "Dogan4Rep", "lang": "en", "location": "Ballwin, MO", "create_at": date("2013-10-01"), "description": "Husband, father of 2, Republican State Rep for Missouri's 98th District", "followers_count": 960, "friends_count": 793, "statues_count": 939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballwin, MO", "id": "cc1dc38105ac44d4", "name": "Ballwin", "place_type": "city", "bounding_box": rectangle("-90.59217,38.54757 -90.509959,38.624853") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2903160, "cityName": "Ballwin" } }
+{ "create_at": datetime("2016-01-05T10:00:20.000Z"), "id": 684434270081921024, "text": "My house is so weird. I'm really scared", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 237557202, "name": "$hePoppin ☀️", "screen_name": "BaileyKaite_", "lang": "en", "location": "†††††††", "create_at": date("2011-01-12"), "description": "Mind set and plans made . There is NOTHING more attractive than ambition . #Natural ; iG: baileykaite_", "followers_count": 2078, "friends_count": 1706, "statues_count": 98460 }, "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-01-05T10:00:20.000Z"), "id": 684434270207647744, "text": "Lowe's #CustomerService #Job: Seasonal Associate - Assembler (#Framingham, MA) https://t.co/2qLPmdmdiY #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.4161565,42.279286"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CustomerService", "Job", "Framingham", "Jobs", "Hiring" }}, "user": { "id": 71719032, "name": "TMJ-BOS CstSrv Jobs", "screen_name": "tmj_bos_cstsrv", "lang": "en", "location": "Boston, MA", "create_at": date("2009-09-04"), "description": "Follow this account for geo-targeted Customer Service job tweets in Boston, MA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 328, "friends_count": 287, "statues_count": 206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Framingham, MA", "id": "dbd1411661903bb0", "name": "Framingham", "place_type": "city", "bounding_box": rectangle("-71.496927,42.257263 -71.374193,42.352931") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2524960, "cityName": "Framingham" } }
+{ "create_at": datetime("2016-01-05T10:00:20.000Z"), "id": 684434270585106434, "text": "Rogue GM's, insurgents, anonymous player leaks, schematic differences, ultimatums. Anyone else tired of the endless bull?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18022253, "name": "Michael", "screen_name": "manecci", "lang": "en", "location": "I have a 585 area code. ", "create_at": date("2008-12-10"), "description": "¯\\_(ツ)_/¯ an entertaining and educated Twitter voice - Tim Graham", "followers_count": 1201, "friends_count": 661, "statues_count": 135231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Rochester, NY", "id": "00d65e664ac70544", "name": "East Rochester", "place_type": "city", "bounding_box": rectangle("-77.524981,43.074573 -77.454465,43.12448") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3622865, "cityName": "East Rochester" } }
+{ "create_at": datetime("2016-01-05T10:00:20.000Z"), "id": 684434270585135104, "text": "#CountDown2King @Philreports https://t.co/NuKwMAnuxF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "CountDown2King" }}, "user_mentions": {{ 44677050 }}, "user": { "id": 69384446, "name": "c e sonny scroggins", "screen_name": "biasbustersofks", "lang": "en", "location": "null", "create_at": date("2009-08-27"), "description": "null", "followers_count": 1352, "friends_count": 2127, "statues_count": 72931 }, "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-01-05T10:00:20.000Z"), "id": 684434270660636672, "text": "Yaaaay I'm home!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 96644534, "name": "Ashlynn Wiley", "screen_name": "The_Taco_God", "lang": "en", "location": "null", "create_at": date("2009-12-13"), "description": "Middle-aged soccer mom", "followers_count": 781, "friends_count": 458, "statues_count": 6580 }, "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-01-05T10:00:20.000Z"), "id": 684434270828412928, "text": "@ImmaWildling @Sttbs73 if it can stop one mass killing worth it to me. My family own guns. Not against them. Just want good background check", "in_reply_to_status": 684430318749761536, "in_reply_to_user": 3310936494, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3310936494, 199875709 }}, "user": { "id": 2492497783, "name": "Sheri", "screen_name": "Lilyladybuggy", "lang": "en", "location": "California, USA", "create_at": date("2014-05-12"), "description": "Interests are Human Rights, People, Environment, Animals, Pop Culture,Entertainment News, Music, Movies, Art, Cooking, and Books", "followers_count": 11119, "friends_count": 11433, "statues_count": 7682 }, "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-05T10:00:21.000Z"), "id": 684434270933237760, "text": "If Twitter moves to 10,000 characters, email must move to 140 characters. \n\nIt's only fair.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 652193, "name": "M.G. Siegler", "screen_name": "mgsiegler", "lang": "en", "location": "San Francisco, CA", "create_at": date("2007-01-16"), "description": "GP @GoogleVentures. \nOn the road with @msquinn. \nA man of few words. \nExcept when writing.", "followers_count": 169007, "friends_count": 919, "statues_count": 45492 }, "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-05T10:00:21.000Z"), "id": 684434271134564352, "text": "Sunny this afternoon, high 44 (7 C). Low 28 (-2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119697197, "name": "Huntsville Weather", "screen_name": "HuntsvilleWX", "lang": "en", "location": "null", "create_at": date("2010-03-04"), "description": "null", "followers_count": 670, "friends_count": 65, "statues_count": 6059 }, "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-05T10:00:21.000Z"), "id": 684434271503695873, "text": "@GizzMagana uggs*", "in_reply_to_status": 684434204734521345, "in_reply_to_user": 3285997818, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3285997818 }}, "user": { "id": 3285997818, "name": "gg", "screen_name": "GizzMagana", "lang": "en", "location": "null", "create_at": date("2015-07-20"), "description": "null", "followers_count": 129, "friends_count": 118, "statues_count": 2789 }, "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-05T10:00:21.000Z"), "id": 684434271692439552, "text": "was gonna get subway but Erik hooked it up with a steak quesadilla ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2300480550, "name": "Flaca ♡", "screen_name": "estephanie_08", "lang": "en", "location": "null", "create_at": date("2014-01-19"), "description": "Tal ves extrañe y recruede algunas personas, pero no las quiero de vuelta.", "followers_count": 404, "friends_count": 355, "statues_count": 10006 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edgewater, CO", "id": "974fec603a36a324", "name": "Edgewater", "place_type": "city", "bounding_box": rectangle("-105.074408,39.743486 -105.053178,39.758488") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8059, "countyName": "Jefferson", "cityID": 823135, "cityName": "Edgewater" } }
+{ "create_at": datetime("2016-01-05T10:00:21.000Z"), "id": 684434271717572608, "text": "United Rentals: Driver CDL A - Power (#ForestPark, GA) https://t.co/VZ86SIQfdW #Transportation #Veterans #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.363149,33.646907"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ForestPark", "Transportation", "Veterans", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 395581347, "name": "ATL Veteran Jobs", "screen_name": "tmj_atl_vets", "lang": "en", "location": "Atlanta, GA", "create_at": date("2011-10-21"), "description": "Follow this account for geo-targeted Veteran job tweets in Atlanta, GA. Need help? Tweet us at @CareerArc!", "followers_count": 236, "friends_count": 157, "statues_count": 78 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forest Park, GA", "id": "26bbc696a1774a65", "name": "Forest Park", "place_type": "city", "bounding_box": rectangle("-84.403047,33.58767 -84.307368,33.648043") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13063, "countyName": "Clayton", "cityID": 1330536, "cityName": "Forest Park" } }
+{ "create_at": datetime("2016-01-05T10:00:21.000Z"), "id": 684434271881175041, "text": "Speaking up for yourself is ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1663704012, "name": "Tj Sing", "screen_name": "Tj2015Tj", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2013-08-11"), "description": "#LackOfWorries #LowLife", "followers_count": 2669, "friends_count": 2092, "statues_count": 33060 }, "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-05T10:00:21.000Z"), "id": 684434271977771008, "text": "I said I wanted a girl friend a few months ago... Dodged a bullet", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 120910566, "name": "ClubSoda Papi", "screen_name": "DineroGarcia", "lang": "en", "location": "414", "create_at": date("2010-03-07"), "description": "Come catch the fade.. #OnWisconsin #CozyBoyz #ATribeCalledHurt", "followers_count": 1145, "friends_count": 674, "statues_count": 140028 }, "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-05T10:00:21.000Z"), "id": 684434271981813760, "text": "@MaricharlotteHa Thank You ��Kasaan", "in_reply_to_status": -1, "in_reply_to_user": 68163467, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 68163467 }}, "user": { "id": 22189163, "name": "Dr Carl Metzger", "screen_name": "ClearMindedness", "lang": "en", "location": "Bend, OR", "create_at": date("2009-02-27"), "description": "ClearMindedness phone talking/books-917.675.0781 http://ClearMindedness.com", "followers_count": 38931, "friends_count": 33268, "statues_count": 230384 }, "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-01-05T10:00:21.000Z"), "id": 684434272409628672, "text": "Ophthalmic Technician II (certified) - San Carlos - Palo Alto Medical Foundation: (#SanCarlos, CA) https://t.co/96hWPjXGcV #Healthcare #Job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.2605222,37.5071591"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "hashtags": {{ "SanCarlos", "Healthcare", "Job" }}, "user": { "id": 120915363, "name": "TMJ-CAP Health Jobs", "screen_name": "tmj_CAP_health", "lang": "en", "location": "The Peninsula, CA", "create_at": date("2010-03-07"), "description": "Follow this account for geo-targeted Healthcare job tweets in The Peninsula, CA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 356, "friends_count": 279, "statues_count": 114 }, "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-01-05T10:00:21.000Z"), "id": 684434272409772033, "text": "Will @JonahHill play Paul when they make a movie about the Browns winning the Super Bowl?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 256573865 }}, "user": { "id": 22277998, "name": "Big Red", "screen_name": "BigRedGilbert", "lang": "en", "location": "Stryker, OH", "create_at": date("2009-02-28"), "description": "Huge Cleveland sports fan! Browns, Buckeyes, Blue Jackets, Cavs, and Indians! Browns Season Ticket Holder. NASCAR #2 Car Fan.", "followers_count": 1167, "friends_count": 1011, "statues_count": 22542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bryan, OH", "id": "f70be696d1eb1051", "name": "Bryan", "place_type": "city", "bounding_box": rectangle("-84.593528,41.455478 -84.488535,41.499462") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39171, "countyName": "Williams", "cityID": 3909792, "cityName": "Bryan" } }
+{ "create_at": datetime("2016-01-05T10:00:21.000Z"), "id": 684434272527192064, "text": "I'm in a bad mood if anyone can tell", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2254942120, "name": "lex", "screen_name": "alexis___002", "lang": "en", "location": "null", "create_at": date("2013-12-31"), "description": "|fly high steven ❤️| south '17 |", "followers_count": 383, "friends_count": 179, "statues_count": 27708 }, "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-01-05T10:00:21.000Z"), "id": 684434272904560642, "text": "YouTube is the new Google for Mobile #mobilemarketing @briansolis @Brand_Innovator #CES2016", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "mobilemarketing", "CES2016" }}, "user_mentions": {{ 11489, 584016075 }}, "user": { "id": 25408012, "name": "Lara Balazs", "screen_name": "LaraHBalazs", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-03-19"), "description": "svp, head of north america marketing visa inc. views are my own.", "followers_count": 1469, "friends_count": 692, "statues_count": 511 }, "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-05T10:00:21.000Z"), "id": 684434273034567681, "text": "#sailing on #miamibay with such an awesome team! Sad to be leaving today when I've had such a… https://t.co/3JgQjq5BYp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.1302414,25.7829399"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sailing", "miamibay" }}, "user": { "id": 199687124, "name": "Joey Bevan Official", "screen_name": "JoeyBevan", "lang": "en", "location": "London", "create_at": date("2010-10-07"), "description": "official - #FashionStylist #FashionDesigner #Presenter #PublishedWriter #ArtDirector - Winner of #inspirational youth of the year & #stylist of the year #award", "followers_count": 30832, "friends_count": 4305, "statues_count": 45245 }, "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-05T10:00:21.000Z"), "id": 684434273110245380, "text": "@JustinThomas34 Just like Bobbie Jones sent out the first year.", "in_reply_to_status": 684070291023134721, "in_reply_to_user": 256317852, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 256317852 }}, "user": { "id": 63139313, "name": "BRIAN H KEARNEY", "screen_name": "BrianHKearney", "lang": "en", "location": "Proud S. S. Irish Chicago, IL", "create_at": date("2009-08-05"), "description": "Evans Scholar Indiana University", "followers_count": 190, "friends_count": 802, "statues_count": 3371 }, "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-01-05T10:00:21.000Z"), "id": 684434273290551296, "text": "I'm at United States Census Bureau in Suitland, MD https://t.co/YJFF5eAcuK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.93047523,38.84689404"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 925891705, "name": "Peter Czekaj", "screen_name": "eflatpete", "lang": "en", "location": "null", "create_at": date("2012-11-04"), "description": "Trumpeter, Audio Engineer", "followers_count": 68, "friends_count": 70, "statues_count": 1868 }, "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-01-05T10:00:21.000Z"), "id": 684434273672101888, "text": "Sunny this afternoon, high 45 (7 C). Low 28 (-2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119698223, "name": "Birmingham Weather", "screen_name": "BirminghamWX", "lang": "en", "location": "Birmingham, AL", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Birmingham, AL", "followers_count": 503, "friends_count": 68, "statues_count": 6071 }, "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-05T10:00:21.000Z"), "id": 684434273881817088, "text": "@tandleman Ah, now I'm with you. I had a tour once, loved the hop room especially. Don't think I saw the whole site though.", "in_reply_to_status": 684429718771396609, "in_reply_to_user": 60290077, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 60290077 }}, "user": { "id": 23511292, "name": "MikeMcG Kansas City", "screen_name": "MikeMcGWirral", "lang": "en", "location": "Kansas City, MO", "create_at": date("2009-03-09"), "description": "British brewer since 1996 @ Ubrew,brewpub,micro,regional,etc. UK,US&Euro styles. Ex @BrimstageBeer. Play uke & percussion. Moved to MidWest USA! Kansas City KC", "followers_count": 2419, "friends_count": 2727, "statues_count": 28301 }, "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-05T10:00:21.000Z"), "id": 684434274313834496, "text": "\"Well then 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": 1578532506, "name": "HeyItsNae❤️", "screen_name": "victoriagomez99", "lang": "en", "location": " seguin tx ", "create_at": date("2013-07-08"), "description": "im pretty lame actually", "followers_count": 297, "friends_count": 713, "statues_count": 2558 }, "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-01-05T10:00:21.000Z"), "id": 684434274762797056, "text": "@Taterdawgg you cold", "in_reply_to_status": 684416048758714369, "in_reply_to_user": 2212706978, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2212706978 }}, "user": { "id": 70890213, "name": "Wale", "screen_name": "Wale_3", "lang": "en", "location": "Miami,FL", "create_at": date("2009-09-01"), "description": "I Work & A Gamer GT: Wale 0 Psn: WaIe_2 #TeamLeo #HeatNation http://twitch.tv/wale__", "followers_count": 225, "friends_count": 1259, "statues_count": 3949 }, "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-05T10:00:21.000Z"), "id": 684434275005890560, "text": "@abbybrewm this is probably the most poetic tweet I've ever see", "in_reply_to_status": 684386913768177664, "in_reply_to_user": 3146674329, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3146674329 }}, "user": { "id": 471429436, "name": "ASIMWE", "screen_name": "Oben_YourMind", "lang": "en", "location": "Lake Forest, IL", "create_at": date("2012-01-22"), "description": "Tall, Dark, and Sorta OK Looking on a Good Day. I sound like a pterodactyl #AllForSchwallie", "followers_count": 405, "friends_count": 393, "statues_count": 11265 }, "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-05T10:00:22.000Z"), "id": 684434275270144000, "text": "CASHMERE FUCKING CAT https://t.co/ppIhxtzTvz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 436300727, "name": "Brooke Williams", "screen_name": "BrookeHWilliams", "lang": "en", "location": "seattle", "create_at": date("2011-12-13"), "description": "ig: brookehw_", "followers_count": 822, "friends_count": 297, "statues_count": 49308 }, "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-05T10:00:22.000Z"), "id": 684434275794464769, "text": ".@neiltyson I'm sure they would do something equally weird. As is affection's way.", "in_reply_to_status": 684409730186477570, "in_reply_to_user": 19725644, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19725644 }}, "user": { "id": 14222861, "name": "Kali Sakai", "screen_name": "kalisakai", "lang": "en", "location": "Seattle, WA", "create_at": date("2008-03-25"), "description": "Freelance writer about parenting, the media, geekery & NW topics. Also reluctant homemaker, shark-lover, tech nerd. Writing seen on @ParentMap.", "followers_count": 209, "friends_count": 119, "statues_count": 1302 }, "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-05T10:00:22.000Z"), "id": 684434276134293506, "text": "�������� https://t.co/EQN2pe6Dku", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 412734116, "name": "angie elf on a shelf", "screen_name": "AngelaCorsetti", "lang": "en", "location": "null", "create_at": date("2011-11-14"), "description": "null", "followers_count": 1323, "friends_count": 702, "statues_count": 53747 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shelby, MI", "id": "5c8c81a4fef94364", "name": "Shelby", "place_type": "city", "bounding_box": rectangle("-83.095428,42.623997 -82.973298,42.715777") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26127, "countyName": "Oceana", "cityID": 2672840, "cityName": "Shelby" } }
+{ "create_at": datetime("2016-01-05T10:00:22.000Z"), "id": 684434276322942977, "text": "@daniellekkaiu @alllilorenzo CAN WE PLS GO. PRETTY PRETTY PLS!������", "in_reply_to_status": 684434135755063296, "in_reply_to_user": 2170865252, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2170865252, 2358531602 }}, "user": { "id": 559080234, "name": "Jennifer Ida", "screen_name": "Jenida15", "lang": "en", "location": "null", "create_at": date("2012-04-20"), "description": "CSULB18", "followers_count": 219, "friends_count": 270, "statues_count": 3687 }, "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-05T10:00:22.000Z"), "id": 684434276331438081, "text": "What a start of 2016! Beauty spread for @nylonmag �� #thankyou #beauty #redlips #rednails… https://t.co/qkIgLRuFgb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0064,40.7142"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thankyou", "beauty", "redlips", "rednails" }}, "user_mentions": {{ 14222518 }}, "user": { "id": 92340321, "name": "Loli", "screen_name": "LoliLo13", "lang": "en", "location": "null", "create_at": date("2009-11-24"), "description": "fashion model&blogger from Prague currently having the time of her life in New York City", "followers_count": 346, "friends_count": 228, "statues_count": 3403 }, "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-05T10:00:22.000Z"), "id": 684434276553732096, "text": "PLEASE LEAVE!!!!!! https://t.co/qDh4oqWh2r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 298613909, "name": "...call me Zay!", "screen_name": "_ProfessorX", "lang": "en", "location": "Gwynn Oak, Maryland", "create_at": date("2011-05-14"), "description": "null", "followers_count": 2626, "friends_count": 1678, "statues_count": 224130 }, "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-05T10:00:22.000Z"), "id": 684434276557918208, "text": "https://t.co/1JvWm6tx2V", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 80750498, "name": "ashleigh ⚓", "screen_name": "ashleigharndt", "lang": "en", "location": "elyria", "create_at": date("2009-10-07"), "description": "sass master, dog loving, disney addict, who messes nearly everything in her life up.", "followers_count": 303, "friends_count": 259, "statues_count": 50398 }, "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-01-05T10:00:22.000Z"), "id": 684434276704628736, "text": "Sunny this afternoon, high 52 (11 C). Low 36 (2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119698549, "name": "Mobile Weather", "screen_name": "MobileWX", "lang": "en", "location": "Mobile, Al", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Mobile, Al", "followers_count": 402, "friends_count": 68, "statues_count": 6146 }, "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-05T10:00:22.000Z"), "id": 684434277006741509, "text": "can't handle anybody", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1051975866, "name": "michael", "screen_name": "miichaelharris", "lang": "en", "location": "null", "create_at": date("2012-12-31"), "description": "cute but psycho", "followers_count": 789, "friends_count": 519, "statues_count": 23038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cuyahoga Falls, OH", "id": "7645bade208f5ea9", "name": "Cuyahoga Falls", "place_type": "city", "bounding_box": rectangle("-81.573215,41.117165 -81.449817,41.213328") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3919778, "cityName": "Cuyahoga Falls" } }
+{ "create_at": datetime("2016-01-05T10:00:22.000Z"), "id": 684434277027594240, "text": "Tyco #CustomerService #Job: Operational Excellence Coordinator (#Marinette, WI) https://t.co/RJpH0RQoeV #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6306623,45.0999849"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CustomerService", "Job", "Marinette", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 72055440, "name": "WI Cust. Srv. Jobs", "screen_name": "tmj_WI_cstsrv", "lang": "en", "location": "Wisconsin", "create_at": date("2009-09-06"), "description": "Follow this account for geo-targeted Customer Service job tweets in Wisconsin Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 298, "friends_count": 278, "statues_count": 125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marinette, WI", "id": "e606455fee2abcf5", "name": "Marinette", "place_type": "city", "bounding_box": rectangle("-87.665514,45.044732 -87.575475,45.106914") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55075, "countyName": "Marinette", "cityID": 5549300, "cityName": "Marinette" } }
+{ "create_at": datetime("2016-01-05T10:00:22.000Z"), "id": 684434277153427456, "text": "Checked out @TAMPAHDGROUP & @TBBCo today. Thumbs up to both! #HarleysandBeer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HarleysandBeer" }}, "user_mentions": {{ 59885759, 43378422 }}, "user": { "id": 120917039, "name": "The Reverend", "screen_name": "Vanilla_Grrilla", "lang": "en", "location": "ASGARD", "create_at": date("2010-03-07"), "description": "Ordained Minister. HOG member. Seminoles, Ravens, Blackhawks, White Sox, Hoosiers, Ireland Rugby Fan.", "followers_count": 50, "friends_count": 241, "statues_count": 3352 }, "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-05T10:00:22.000Z"), "id": 684434277392515072, "text": "Breakfast with the primoooo! @ Kelley's Country Cookin https://t.co/O2Ks1UyKUf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.267198,29.55797473"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 111502381, "name": "Tila Gallardo", "screen_name": "TotallyTILA", "lang": "en", "location": "Houston Tx", "create_at": date("2010-02-04"), "description": "NKOTB, Celebrity stalker, family, beer, fun, all day", "followers_count": 110, "friends_count": 113, "statues_count": 1311 }, "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-05T10:00:22.000Z"), "id": 684434277585440769, "text": "Standard Raining in LA Traffic Content https://t.co/J1tPklatQs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16608336, "name": "Kyle Kim", "screen_name": "kyleykim", "lang": "en", "location": "Los Angeles", "create_at": date("2008-10-05"), "description": "Graphics, data journalist for @LATimes. Swedophile. Hot sauce-obsessed. Coding badly.", "followers_count": 4485, "friends_count": 1264, "statues_count": 14262 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:22.000Z"), "id": 684434277883359233, "text": "@prettyronie_ what size are u???", "in_reply_to_status": 684433718736388096, "in_reply_to_user": 15352472, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15352472 }}, "user": { "id": 18692467, "name": "DeadStock Ric'", "screen_name": "deadstockric", "lang": "en", "location": "ÜT: 40.684264,-73.945492", "create_at": date("2009-01-06"), "description": "I sell sneakers and travel for a living. need me? deadstockric@gmail.com", "followers_count": 27354, "friends_count": 23564, "statues_count": 62875 }, "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-05T10:00:22.000Z"), "id": 684434278038421504, "text": "In a singing mood because its lunch time..\n#5onit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.9615114,35.9138127"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "5onit" }}, "user": { "id": 414445747, "name": "Naomi Usti Birdtail", "screen_name": "NaomiRBirdtail", "lang": "en", "location": "Tahlequah, OK", "create_at": date("2011-11-16"), "description": "Im 25 Likes To Laugh In Have Fun Plus Im Straight Blunt NATIVE AMERICAN CHEROKEE/CREEK", "followers_count": 300, "friends_count": 1505, "statues_count": 6848 }, "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-01-05T10:00:22.000Z"), "id": 684434278399238145, "text": "We all grown... That's for children, I'm so sho", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 100937957, "name": "peache$", "screen_name": "kissashh", "lang": "en", "location": "Baton Rouge, Louisiana", "create_at": date("2010-01-01"), "description": "|SU| keep yo heart 3 stacks. tryna be 23 w/ a money tree #dopeforevereva❤️", "followers_count": 2095, "friends_count": 594, "statues_count": 77070 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Opelousas, LA", "id": "dd1231585f98adbd", "name": "Opelousas", "place_type": "city", "bounding_box": rectangle("-92.164674,30.469767 -92.03885,30.587433") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22097, "countyName": "St. Landry", "cityID": 2258045, "cityName": "Opelousas" } }
+{ "create_at": datetime("2016-01-05T10:00:22.000Z"), "id": 684434278550233090, "text": "Wind 2.0 mph NNE. Barometer 30.592 in, Steady. Temperature 24.6F. Rain today 0.00 in. Humidity 37% | 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": 575, "friends_count": 773, "statues_count": 35287 }, "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-05T10:00:22.000Z"), "id": 684434278780768256, "text": "Arizona Winning Super Bowl ...", "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": "Yo Competition", "screen_name": "214LilDesmond", "lang": "en", "location": "DTX ", "create_at": date("2011-11-20"), "description": "The Wolf of 4 Street #SHSU", "followers_count": 1946, "friends_count": 883, "statues_count": 54494 }, "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-05T10:00:22.000Z"), "id": 684434278973702145, "text": "�������� shit is still funny https://t.co/26E29ZWRjD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 210624595, "name": "your new man", "screen_name": "EmadDoItBetter", "lang": "en", "location": "Your girl dm's ", "create_at": date("2010-10-31"), "description": "null", "followers_count": 721, "friends_count": 571, "statues_count": 110537 }, "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-05T10:00:22.000Z"), "id": 684434278986477568, "text": "You are �� correct �� https://t.co/PfwrMnoHq4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 776379271, "name": "•Krit•", "screen_name": "xoxokristen", "lang": "en", "location": "Sc kristen_sellers", "create_at": date("2012-08-23"), "description": "I touch the future... I teach", "followers_count": 809, "friends_count": 647, "statues_count": 5047 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shelby, NC", "id": "5506172b34d4c9a3", "name": "Shelby", "place_type": "city", "bounding_box": rectangle("-81.611574,35.237295 -81.460676,35.347086") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37045, "countyName": "Cleveland", "cityID": 3761200, "cityName": "Shelby" } }
+{ "create_at": datetime("2016-01-05T10:00:22.000Z"), "id": 684434279154094080, "text": "More characters in tweets means idiots like Trump have more space for their racist garbage.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28683272, "name": "Global Money", "screen_name": "goshortandlong", "lang": "en", "location": "The world.", "create_at": date("2009-04-03"), "description": "Doing something to make the world better (no matter how big or small) will give you a sense of inner peace.", "followers_count": 358, "friends_count": 341, "statues_count": 12471 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:23.000Z"), "id": 684434279263244288, "text": "Sometimes I think @wolfblitzer real non partisan journalist but again he proved me wrong #StandWithOurTroops #ObamaIgnoresTheConstitution", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "StandWithOurTroops", "ObamaIgnoresTheConstitution" }}, "user_mentions": {{ 71294756 }}, "user": { "id": 57047795, "name": "LionBlood", "screen_name": "virgojesse", "lang": "en", "location": "Detroit, MI", "create_at": date("2009-07-15"), "description": "Trolling bears fans. Foreign Affairs Insider. Detroit sports fan. News junkie. RT not endorsement.", "followers_count": 174, "friends_count": 726, "statues_count": 1066 }, "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-05T10:00:23.000Z"), "id": 684434279275745280, "text": "Truth https://t.co/sD0ZtXbmJC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2215181174, "name": "Jerod", "screen_name": "thejerodfox", "lang": "en", "location": "the tron ➡️ hastings ne", "create_at": date("2013-11-25"), "description": "I don't run your mind? Let me jog your memory. Former NHS student body president. Proudly from the 316. Altman 300 gang. Snapchat: jfoxrockchalk #SOD", "followers_count": 866, "friends_count": 598, "statues_count": 10349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hastings, NE", "id": "ac9830fe327eb63f", "name": "Hastings", "place_type": "city", "bounding_box": rectangle("-98.467334,40.561183 -98.347736,40.646339") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31001, "countyName": "Adams", "cityID": 3121415, "cityName": "Hastings" } }
+{ "create_at": datetime("2016-01-05T10:00:23.000Z"), "id": 684434279602892801, "text": "Partly cloudy this afternoon, high 34 (1 C). Low 27 (-3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119751854, "name": "Chicago Weather", "screen_name": "Chicago_WX", "lang": "en", "location": "Chicago, IL", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Chicago, IL", "followers_count": 1060, "friends_count": 68, "statues_count": 6428 }, "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-05T10:00:23.000Z"), "id": 684434279879675904, "text": "#PolandIsReadyForRevivalTour 34", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "PolandIsReadyForRevivalTour" }}, "user": { "id": 1691377656, "name": "Justin my Angel", "screen_name": "AngelaDziedzic1", "lang": "pl", "location": "Chicago, IL", "create_at": date("2013-08-22"), "description": "I fell in love with a 15 year old guy who dreams fulfilled now I love 21-year-old man,who helps fulfill the dreams of other!18/11/15 Thank You Justin PotterHead", "followers_count": 325, "friends_count": 808, "statues_count": 2604 }, "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-01-05T10:00:23.000Z"), "id": 684434280651571201, "text": "@_soulcheyy Rich Friend You Coming To Class ��", "in_reply_to_status": -1, "in_reply_to_user": 2626056718, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2626056718 }}, "user": { "id": 1408866512, "name": "$B.$WEET✨", "screen_name": "MoneylaneMari", "lang": "en", "location": "null", "create_at": date("2013-05-06"), "description": "Ja'Mari Sweet C/o'18 Plantation High #GodFirst", "followers_count": 556, "friends_count": 476, "statues_count": 3427 }, "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-05T10:00:23.000Z"), "id": 684434280676630528, "text": "Holy shit, god bless stacey", "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": "Heaviside", "create_at": date("2014-04-21"), "description": "null", "followers_count": 451, "friends_count": 205, "statues_count": 15606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:23.000Z"), "id": 684434280991309825, "text": "@TodoriCodollins snap story ��", "in_reply_to_status": -1, "in_reply_to_user": 161952900, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 161952900 }}, "user": { "id": 461421213, "name": "BB-8", "screen_name": "WZRDBEMO", "lang": "en", "location": "null", "create_at": date("2012-01-11"), "description": "19. Ohio. Keep it a buck and don't ever play yourself.", "followers_count": 519, "friends_count": 424, "statues_count": 16474 }, "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-05T10:00:23.000Z"), "id": 684434281280745472, "text": "Tons of SHARP debris in southbound bike lane and road, Folsom @ Mapleton. Kinda hard to see it. #cyclingupdate @CommunityCycles #noflats", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "cyclingupdate", "noflats" }}, "user_mentions": {{ 124596235 }}, "user": { "id": 23008636, "name": "WandaPelegrinaCaldas", "screen_name": "1daPC", "lang": "en", "location": "Boulder, CO", "create_at": date("2009-03-05"), "description": "Bici-lover. Car-free. Vegan. Island girl. Just add music.", "followers_count": 150, "friends_count": 202, "statues_count": 17861 }, "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-05T10:00:23.000Z"), "id": 684434281393856512, "text": "Can you recommend anyone for this #job? Machining Manager - https://t.co/uAmtHoFeIg #Andover, MN #Engineering #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.2913405,45.2332984"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Andover", "Engineering", "Hiring", "CareerArc" }}, "user": { "id": 23356983, "name": "TMJ-MSP Engin. Jobs", "screen_name": "tmj_msp_eng", "lang": "en", "location": "St. Paul, MN", "create_at": date("2009-03-08"), "description": "Follow this account for geo-targeted Engineering job tweets in St. Paul, MN. Need help? Tweet us at @CareerArc!", "followers_count": 309, "friends_count": 230, "statues_count": 131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Andover, MN", "id": "219ccc645d458eee", "name": "Andover", "place_type": "city", "bounding_box": rectangle("-93.40681,45.21178 -93.265395,45.304142") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27003, "countyName": "Anoka", "cityID": 2701486, "cityName": "Andover" } }
+{ "create_at": datetime("2016-01-05T10:00:23.000Z"), "id": 684434281460924416, "text": "See our latest #Allen, TX #job and click to apply: Host - Military - https://t.co/DFk1Fj9Tai #Hospitality #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.6827565,33.0890798"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Allen", "job", "Hospitality", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 88304749, "name": "TMJ-TX HRTA Jobs", "screen_name": "tmj_tx_hrta", "lang": "en", "location": "Texas", "create_at": date("2009-11-07"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Texas Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 404, "friends_count": 289, "statues_count": 8009 }, "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-05T10:00:23.000Z"), "id": 684434281473654788, "text": "China Beach Love. triciayen #sanfrancisco #engagementphotos @ China Beach https://t.co/vikChrakdC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.49091506,37.78781548"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sanfrancisco", "engagementphotos" }}, "user": { "id": 180410362, "name": "C.M. Howard", "screen_name": "cmhowardphoto", "lang": "en", "location": "San Francisco", "create_at": date("2010-08-19"), "description": "Photographs people. And things. And occasionally places. Makes a good cocktail.", "followers_count": 89, "friends_count": 76, "statues_count": 454 }, "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-05T10:00:23.000Z"), "id": 684434281611956224, "text": "The only issue you can tell Obama cares about is gun control. And that 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": 389253159, "name": "David Wiggins", "screen_name": "davidWiggins25", "lang": "en", "location": "Monroe,LA", "create_at": date("2011-10-11"), "description": "ULM fishing team. NHS alumni. Psalm 28:7. snapchat:davidwiggins", "followers_count": 301, "friends_count": 300, "statues_count": 8472 }, "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-05T10:00:23.000Z"), "id": 684434281628889088, "text": "Me 25/8 https://t.co/4nOp1NChNC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1925008814, "name": "Baby La Flare", "screen_name": "RosesAndRellos", "lang": "en", "location": "478 ", "create_at": date("2013-10-01"), "description": "daddy ain raise no pussy", "followers_count": 1571, "friends_count": 1018, "statues_count": 35530 }, "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-01-05T10:00:23.000Z"), "id": 684434281716830208, "text": "Purchasing - Accountemps: (#WINTERPARK, FL) https://t.co/r4rH4L02wX #Purchasing #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3014284,28.6092911"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WINTERPARK", "Purchasing", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 63051013, "name": "TMJ-USA Purch Jobs", "screen_name": "tmj_usa_purch", "lang": "en", "location": "USA", "create_at": date("2009-08-04"), "description": "Follow this account for geo-targeted Purchasing/Procurement job tweets in USA. Need help? Tweet us at @CareerArc!", "followers_count": 156, "friends_count": 91, "statues_count": 586 }, "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-01-05T10:00:23.000Z"), "id": 684434281813245952, "text": "You automatically get points if your rocking bape, supreme, or palace that stuff not cheap", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 461601170, "name": "Steph Hob", "screen_name": "HobsonStephani", "lang": "en", "location": "null", "create_at": date("2012-01-11"), "description": "the good life//joshua 1:9", "followers_count": 1338, "friends_count": 411, "statues_count": 9655 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wenatchee, WA", "id": "2b67aaa9fac6ba32", "name": "Wenatchee", "place_type": "city", "bounding_box": rectangle("-120.3592,47.396841 -120.295617,47.464871") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53007, "countyName": "Chelan", "cityID": 5377105, "cityName": "Wenatchee" } }
+{ "create_at": datetime("2016-01-05T10:00:23.000Z"), "id": 684434282090106880, "text": "Mostly sunny this afternoon, high 50 (10 C). Low 33 (1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119753180, "name": "Jackson Weather", "screen_name": "JacksonWX", "lang": "en", "location": "Jackson, MS", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Jackson, MS", "followers_count": 295, "friends_count": 68, "statues_count": 6377 }, "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-05T10:00:23.000Z"), "id": 684434282190909440, "text": "@extrasexual @JennaSanLorenzo im ctfu", "in_reply_to_status": 684434124598308866, "in_reply_to_user": 480968284, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2928124344, 538330630 }}, "user": { "id": 480968284, "name": "jennnnnnaaaaa", "screen_name": "vajaejaepolit", "lang": "en", "location": "Fishtown, Pa", "create_at": date("2012-02-01"), "description": "nineteeen | love yours", "followers_count": 951, "friends_count": 890, "statues_count": 17705 }, "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-05T10:00:23.000Z"), "id": 684434282216030209, "text": "@jordan32499 bc Molly da Mazda is going into the shop", "in_reply_to_status": 684434156332408832, "in_reply_to_user": 1571594342, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1571594342 }}, "user": { "id": 428698543, "name": "Kelsy♡", "screen_name": "knbreck", "lang": "en", "location": "FGCU | ΔΔΔ", "create_at": date("2011-12-04"), "description": "I'm just in it for the sugar rush", "followers_count": 489, "friends_count": 395, "statues_count": 17921 }, "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-01-05T10:00:23.000Z"), "id": 684434282237038593, "text": "@jgstewart Yup. Was trying to define a new service in the services.yml. Silly me tried to use a tab where clearly spaces were indicated.", "in_reply_to_status": 684433625123663872, "in_reply_to_user": 17054884, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17054884 }}, "user": { "id": 8996632, "name": "Eric Ranschau", "screen_name": "eranschau", "lang": "en", "location": "Minneapolis, MN", "create_at": date("2007-09-20"), "description": "I'm a web developer, sci-fi junkie, and reef aquarium addict. Feel free to make up the rest. ¯\\_(ツ)_/¯", "followers_count": 491, "friends_count": 719, "statues_count": 12628 }, "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-05T10:00:23.000Z"), "id": 684434282325000192, "text": "My boss just said she thinks Robin Hood Men in Tights is a bad film. I'm not sure I can work here anymore.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20330346, "name": "Ryan Lowy", "screen_name": "AJustConspiracy", "lang": "en", "location": "San Clemente/San Diego", "create_at": date("2009-02-07"), "description": "Internet person I guess.", "followers_count": 304, "friends_count": 529, "statues_count": 12771 }, "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-01-05T10:00:23.000Z"), "id": 684434282434179072, "text": "@vielmetti Because of rights? Who owns the content? Does the content get sold with the spectrum?", "in_reply_to_status": 684434110572556288, "in_reply_to_user": 9272, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 9272 }}, "user": { "id": 8052962, "name": "Josh", "screen_name": "zigziggityzoo", "lang": "en", "location": "Ann Arbor, MI", "create_at": date("2007-08-08"), "description": "I'm a SysAdmin for U of Michigan for both Macs and PCs. Also a political armchair quarterback, photographer, root beer enthusiast, father, and tech geek.", "followers_count": 582, "friends_count": 349, "statues_count": 25430 }, "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-05T10:00:23.000Z"), "id": 684434282622799872, "text": "@TexasObserver SMX is hiring in #McAllen, TX - Stop by today, January 5 starting at 9am at 5700 International Parkway, McAllen, TX 78503", "in_reply_to_status": -1, "in_reply_to_user": 15021918, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "McAllen" }}, "user_mentions": {{ 15021918 }}, "user": { "id": 170284259, "name": "Carlos Calderon", "screen_name": "carlosgcalderon", "lang": "en", "location": "Miami, FL", "create_at": date("2010-07-24"), "description": "SDG Management", "followers_count": 731, "friends_count": 1978, "statues_count": 743 }, "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-05T10:00:23.000Z"), "id": 684434282660675584, "text": "Jan.23rd #laughter&wine#,we gonna bring the house down 501 Central Ave Orange,NJ! @ Southern… https://t.co/VxYipO2Tl9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.2408981,40.7689209"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "laughter" }}, "user": { "id": 115293252, "name": "Q.BIZZLE", "screen_name": "Qrox", "lang": "en", "location": "ON YA MEDULLA OBLONGATA", "create_at": date("2010-02-17"), "description": "Little Fish in a Big Pond aka The young Prince of Team ETHER..Ill dunk on ya grandma @the same dam time.", "followers_count": 447, "friends_count": 592, "statues_count": 7583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, NJ", "id": "559cf1d51980ff42", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-74.255395,40.752105 -74.218098,40.789858") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3451000, "cityName": "Newark" } }
+{ "create_at": datetime("2016-01-05T10:00:23.000Z"), "id": 684434282899570688, "text": "@JessWaltonYR Has Been Playing Jill Abbott For 18 Years Since June 1987! She Has Made This Role Hers! @CBSDaytime @YandR_CBS @YRInsider", "in_reply_to_status": -1, "in_reply_to_user": 35552231, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 35552231, 703436646, 68816598, 112605001 }}, "user": { "id": 1138416104, "name": "Farris Flagg", "screen_name": "FarrisFlagg", "lang": "en", "location": "San Diego", "create_at": date("2013-01-31"), "description": "#Hevon! Bryton James. Mishael Morgan. The Ellen Degeneres Show. The Voice. Scandal. How To Get Away With Murder. #YR #GH Steve Harvey. The Real. Empire.", "followers_count": 871, "friends_count": 2056, "statues_count": 35293 }, "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-05T10:00:23.000Z"), "id": 684434282899611648, "text": "@saaaaavvvyyyy ���� that's the only building i know there �� kudos of living across the street from tvcc for 4 years", "in_reply_to_status": 684433931073011712, "in_reply_to_user": 1942359031, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1942359031 }}, "user": { "id": 378280157, "name": "Milaaaaah", "screen_name": "Milaah27", "lang": "en", "location": "Portland, OR", "create_at": date("2011-09-22"), "description": "Snap/IG: @princessmilah27 PSU Viking. 541 to the 503", "followers_count": 529, "friends_count": 545, "statues_count": 15170 }, "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-05T10:00:23.000Z"), "id": 684434282916388864, "text": "1. Avoid Costly Mistakes 2. Talk w/ Mortgage Expert 3. Apply! (480) 299 - 1846 @tucsonradio #mortgage #fha #valoans https://t.co/USE6YdRuYr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "mortgage", "fha", "valoans" }}, "user_mentions": {{ 220119760 }}, "user": { "id": 2784794838, "name": "James Caskey", "screen_name": "CaskeyJames", "lang": "en", "location": "4500 S. Lakeshore Dr. Tempe ", "create_at": date("2014-09-01"), "description": "Father, Rotarian, Mortgage Broker w/ 20 years of Mtg Lending experience in Conv., FHA, VA, & Alt-A programs.", "followers_count": 668, "friends_count": 1056, "statues_count": 128 }, "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-05T10:00:24.000Z"), "id": 684434283457478656, "text": "See our latest #Gillette, WY #job and click to apply: NICU Level 2 - https://t.co/njELUCnER6 #Nursing #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.5022205,44.2910915"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Gillette", "job", "Nursing", "Hiring", "CareerArc" }}, "user": { "id": 49870147, "name": "Cheyenne Nursing", "screen_name": "tmj_wyc_nursing", "lang": "en", "location": "Cheyenne, WY", "create_at": date("2009-06-22"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Cheyenne, WY from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 331, "friends_count": 258, "statues_count": 44 }, "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-01-05T10:00:24.000Z"), "id": 684434283478519809, "text": "@Demon31G @billmeltzer @BeerMeABeer My trade block: both schenns, couturier, read, streit, simmonds. Obviously not giving away for free.", "in_reply_to_status": 684433691750330369, "in_reply_to_user": 73152491, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 73152491, 25318266, 52939060 }}, "user": { "id": 38267358, "name": "Ben", "screen_name": "Benayunk", "lang": "en", "location": "Philly", "create_at": date("2009-05-06"), "description": "Interested in Philadelphia Sports. Nerdy Science Stuff History", "followers_count": 262, "friends_count": 742, "statues_count": 5674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Exton, PA", "id": "f05705aeb1de96db", "name": "Exton", "place_type": "city", "bounding_box": rectangle("-75.653372,40.014927 -75.607883,40.045387") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42029, "countyName": "Chester", "cityID": 4224440, "cityName": "Exton" } }
+{ "create_at": datetime("2016-01-05T10:00:24.000Z"), "id": 684434283625234433, "text": "Accident cleared. in #NorthSide on I-45 N Fwy Outbound at Rankin, stop and go traffic back to Greens Rd., delay of 3 mins #traffic", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.41885,29.96577"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NorthSide", "traffic" }}, "user": { "id": 249818911, "name": "TTN Houston", "screen_name": "TotalTrafficHOU", "lang": "en", "location": "Houston, TX", "create_at": date("2011-02-09"), "description": "Your #1 stop for all your Houston Traffic needs! See a #Traffic issue? Tweet us or call us on the tip line at 281- 214-0440 #HoustonTexans #HoustonAstros", "followers_count": 1015, "friends_count": 112, "statues_count": 130794 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:24.000Z"), "id": 684434283721691136, "text": "Rainy Day, coffee, & Ableton Good Morning Friends :)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 117621418, "name": "Han Solo Is Dead", "screen_name": "SirRoscoe_", "lang": "en", "location": "City Of Roses, CA ", "create_at": date("2010-02-25"), "description": "@_BeyondNormal_ /˚∆111∆˚/@LAZArecords IMUNE ∆∆∆∆∆∆/@POPGANG_", "followers_count": 1823, "friends_count": 1003, "statues_count": 104239 }, "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-05T10:00:24.000Z"), "id": 684434283776217089, "text": "We're #hiring! Read about our latest #job opening here: Mandarin Bar Bartender - Full... - https://t.co/HR0TQezyhT https://t.co/QGZN3IrM9h", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.1745805,36.1062236"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job" }}, "user": { "id": 3089847157, "name": "Mandarin Las Vegas", "screen_name": "MO_LVJobs", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-03-16"), "description": "Nevada's only Triple Forbes Five-Star luxury destination located on the world-famous #LasVegas Strip. Apply here to open #jobs.", "followers_count": 86, "friends_count": 59, "statues_count": 46 }, "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-05T10:00:24.000Z"), "id": 684434283843448832, "text": ">>>/#TylerDurden/\n>>>/OUT/\n>>>/#ReichNow/", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "TylerDurden", "ReichNow" }}, "user": { "id": 4342451596, "name": "D.F.A. Iŗvën-Sæntýðr", "screen_name": "dfasyn", "lang": "en", "location": "Crossville, TN", "create_at": date("2015-12-01"), "description": "The Mengeleson, Dr. Oliver Lynton-Wolfe", "followers_count": 3046, "friends_count": 4997, "statues_count": 8025 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crossville, TN", "id": "cb839ecd0a29956f", "name": "Crossville", "place_type": "city", "bounding_box": rectangle("-85.096414,35.898172 -84.987422,36.009751") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47035, "countyName": "Cumberland", "cityID": 4718540, "cityName": "Crossville" } }
+{ "create_at": datetime("2016-01-05T10:00:24.000Z"), "id": 684434284153683968, "text": "Please to announce @borgore https://t.co/Qw7TaPJMnr next #Saturday To book a VIP table pls call/text\nErick… https://t.co/VVeuB3nc6r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.1432571,25.7901096"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Saturday" }}, "user_mentions": {{ 66995579 }}, "user": { "id": 885617828, "name": "New World Events LLC", "screen_name": "NewWorldEvents", "lang": "en", "location": "Miami Beach FL", "create_at": date("2012-10-16"), "description": "VIP Concierge Services, Event Planning, and Event Promotions", "followers_count": 89, "friends_count": 38, "statues_count": 1466 }, "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-05T10:00:24.000Z"), "id": 684434284220792834, "text": "@CodyWofford13 @kgibson66_CKA @NateFerrell17 @epicsportsvline I'm just saying @makenziecostner would have tried to bicycle kick it", "in_reply_to_status": 684433905135517696, "in_reply_to_user": 716994469, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 716994469, 414050639, 460321517, 2340816320, 872110201 }}, "user": { "id": 377581137, "name": "Joe Costo", "screen_name": "JoeCosto3", "lang": "en", "location": "null", "create_at": date("2011-09-21"), "description": "If you take life to seriously you'll never make it out alive", "followers_count": 253, "friends_count": 201, "statues_count": 6399 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Newnan, GA", "id": "e00d1c9a87793cec", "name": "East Newnan", "place_type": "city", "bounding_box": rectangle("-84.796944,33.328915 -84.760294,33.362948") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13077, "countyName": "Coweta", "cityID": 1325692, "cityName": "East Newnan" } }
+{ "create_at": datetime("2016-01-05T10:00:24.000Z"), "id": 684434284539736064, "text": "Play tennis, drink wine.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 168327349, "name": "Hillary", "screen_name": "redhairedgirl35", "lang": "en", "location": "null", "create_at": date("2010-07-18"), "description": "Cal Berkeley--go beers, go bears. East✈️West", "followers_count": 230, "friends_count": 142, "statues_count": 6026 }, "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-01-05T10:00:24.000Z"), "id": 684434284594114560, "text": "Partly cloudy this afternoon, high 50 (10 C). Low 41 (5 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119754993, "name": "Waco Weather", "screen_name": "WacoWeather", "lang": "en", "location": "Waco, TX", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Waco, TX", "followers_count": 817, "friends_count": 68, "statues_count": 6358 }, "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.069786,31.639828") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4876000, "cityName": "Waco" } }
+{ "create_at": datetime("2016-01-05T10:00:24.000Z"), "id": 684434284669698049, "text": "@Araya_Hope____ flame for you https://t.co/pDdEqAFlHj", "in_reply_to_status": -1, "in_reply_to_user": 204356919, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 204356919 }}, "user": { "id": 90680228, "name": "uncle bitches", "screen_name": "BenjaminBitches", "lang": "en", "location": "Cincinnat OHIO", "create_at": date("2009-11-17"), "description": "Contact me at EZduzit513@Gmail.com #GKFAM #DeadSociety #TheU", "followers_count": 1302, "friends_count": 2427, "statues_count": 26702 }, "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-05T10:00:24.000Z"), "id": 684434285126811648, "text": "What I'd do to be at the Alphalete gym right now... @BlondeVsWorld @Guzmanfitness #stuckinMinnesota", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.7897234,44.305794"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "stuckinMinnesota" }}, "user_mentions": {{ 33044967, 706503120 }}, "user": { "id": 51132251, "name": "Marquette Bratsch", "screen_name": "marquette10", "lang": "en", "location": "null", "create_at": date("2009-06-26"), "description": "Everyone has a story.", "followers_count": 273, "friends_count": 286, "statues_count": 2518 }, "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.74644,44.33") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46011, "countyName": "Brookings", "cityID": 4607580, "cityName": "Brookings" } }
+{ "create_at": datetime("2016-01-05T10:00:24.000Z"), "id": 684434285437300737, "text": "I feel like if you eat a girl's ass, it's a dub. You lose.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1446206846, "name": "smitty™", "screen_name": "x_JuSmarr_", "lang": "en", "location": "Buffalo, NY", "create_at": date("2013-05-21"), "description": "Love is or it ain't. Thin love ain't love at all. -Toni Morrison ❤\nco'16 \nfollow my ig & add my snap : x_juu ☺", "followers_count": 1297, "friends_count": 1172, "statues_count": 38200 }, "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-05T10:00:24.000Z"), "id": 684434285550415873, "text": "When you see a girl walking in the rain with no jacket, remember what Cardi B said��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "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 | RWTW$ | We dont brag. We flex", "followers_count": 930, "friends_count": 761, "statues_count": 15722 }, "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-05T10:00:24.000Z"), "id": 684434285684768771, "text": "@QuestionBall ����I'm grateful for the Follow, Q Ball! Good luck to you! Visit us for music, books, photos, ideas at https://t.co/weCMM55q3g", "in_reply_to_status": -1, "in_reply_to_user": 3751716923, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3751716923 }}, "user": { "id": 968222028, "name": "Thom Gambino", "screen_name": "ThomGambino", "lang": "en", "location": "New York City", "create_at": date("2012-11-24"), "description": "Saxophonist, flutist, author, composer, arranger, Liberal Democrat. #UniteBlue For full bio and other info, go to:\nhttp://www.thomgambino.com", "followers_count": 7855, "friends_count": 8580, "statues_count": 19409 }, "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-05T10:00:24.000Z"), "id": 684434285831581696, "text": "DSC_8121.JPG https://t.co/MWG4lG8Awk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-123.01197876,37.6991901"), "retweet_count": 0, "lang": "is", "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": 10092 }, "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-05T10:00:24.000Z"), "id": 684434285881769984, "text": "6. Yatim\n7. #Forcillo\n8. Bronze\n9. Anthony Gingras\n10. Puslinch\n\n2016/1/5 11:56 CST #trndnl https://t.co/SHjd3tkINE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.309,56.9547"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Forcillo", "trndnl" }}, "user": { "id": 1270239397, "name": "Trendinalia Canada", "screen_name": "trendinaliaCA", "lang": "es", "location": "Canada", "create_at": date("2013-03-15"), "description": "Information about Twitter's Trending Topics in Canada", "followers_count": 1247, "friends_count": 7, "statues_count": 257265 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Division No. 23, Unorganized, Manitoba", "id": "654fd5617c7361d0", "name": "Division No. 23, Unorganized", "place_type": "city", "bounding_box": rectangle("-102.009382,55.158501 -88.989166,60.000283") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-05T10:00:24.000Z"), "id": 684434285890265092, "text": "@maggiehays_ same here I love you mags! ��", "in_reply_to_status": 684433857937051649, "in_reply_to_user": 4355028519, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4355028519 }}, "user": { "id": 326218211, "name": "Rachel Bartz", "screen_name": "rachbartz", "lang": "en", "location": "the Mitten✋ // Rensselaer ", "create_at": date("2011-06-29"), "description": "i dont mind eating spaghetti for breakfast and waffles for dinner one bit • awareness is key • @jhays523 is my fav SJCXC/TF", "followers_count": 849, "friends_count": 638, "statues_count": 8696 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsburg, IN", "id": "00f424d474454ea0", "name": "Brownsburg", "place_type": "city", "bounding_box": rectangle("-86.439701,39.800082 -86.326794,39.912651") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18063, "countyName": "Hendricks", "cityID": 1808416, "cityName": "Brownsburg" } }
+{ "create_at": datetime("2016-01-05T10:00:24.000Z"), "id": 684434285898690560, "text": "It was 8 degrees this morning the wind made it feel like 0... Just hush lol https://t.co/H0Gqjv9YAq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3331935453, "name": "Brittany", "screen_name": "PoundCakeBee", "lang": "en", "location": "null", "create_at": date("2015-06-17"), "description": "Only for the trill", "followers_count": 32, "friends_count": 37, "statues_count": 704 }, "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-05T10:00:24.000Z"), "id": 684434285990903808, "text": "I think I liked 2015 more....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 971302609, "name": "shania cirba", "screen_name": "_shanaynay_", "lang": "en", "location": "null", "create_at": date("2012-11-25"), "description": "null", "followers_count": 320, "friends_count": 223, "statues_count": 1288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Archbald, PA", "id": "002d7b971d672632", "name": "Archbald", "place_type": "city", "bounding_box": rectangle("-75.592151,41.479945 -75.526675,41.522979") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42069, "countyName": "Lackawanna", "cityID": 4202832, "cityName": "Archbald" } }
+{ "create_at": datetime("2016-01-05T10:00:24.000Z"), "id": 684434286213246976, "text": "Oh shit Mercury retrograde :(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 481730476, "name": "Allison Brainard", "screen_name": "allisonbrain", "lang": "en", "location": "null", "create_at": date("2012-02-02"), "description": "✨", "followers_count": 179, "friends_count": 395, "statues_count": 200 }, "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-05T10:00:24.000Z"), "id": 684434286305411073, "text": "finally someone who knows how to use social media", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14063697, "name": "Nick @ Not Japan :(", "screen_name": "heyooitsnick", "lang": "en", "location": "dreamy lake", "create_at": date("2008-02-29"), "description": "dating @sleepyimouto | regretfully a riajuu. life is unfortunate. | i'm apparently pretty good at rhythm games but i think everyone's lying", "followers_count": 1804, "friends_count": 200, "statues_count": 176847 }, "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-05T10:00:24.000Z"), "id": 684434286351499264, "text": "Lmao, they loyal asl AND nice https://t.co/keXEtdm8Xm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2657024882, "name": "too legited", "screen_name": "joirdan_jackson", "lang": "en", "location": "chicago ✈️ dallas", "create_at": date("2014-07-18"), "description": "|Christ is my Lord and Savior and he died for my sins.✨ | STAYWOKE | photography | 18.", "followers_count": 443, "friends_count": 297, "statues_count": 10690 }, "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-05T10:00:24.000Z"), "id": 684434287005859840, "text": "@RachelLoritz17 story of my life lol", "in_reply_to_status": 684420349161639936, "in_reply_to_user": 67234112, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 67234112 }}, "user": { "id": 1598762456, "name": "Սերժ", "screen_name": "serjdurge", "lang": "en", "location": "Orange, CA", "create_at": date("2013-07-16"), "description": "I've been in town for just now 15 minutes.\n\n snapchat: serjdakessian", "followers_count": 394, "friends_count": 332, "statues_count": 12952 }, "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-05T10:00:24.000Z"), "id": 684434287228133376, "text": "Mostly sunny this afternoon, high 38 (3 C). Low 25 (-4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119755728, "name": "Louisville Weather", "screen_name": "Louisville_WX", "lang": "en", "location": "Louisville, KY", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Louisville, KY", "followers_count": 1311, "friends_count": 68, "statues_count": 6384 }, "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-01-05T10:00:24.000Z"), "id": 684434287463165953, "text": "@falloon19 It's my privilege brotha. #Beardown", "in_reply_to_status": 684391913483407363, "in_reply_to_user": 169000507, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Beardown" }}, "user_mentions": {{ 169000507 }}, "user": { "id": 621524748, "name": "Ray Austin", "screen_name": "rayaustin36", "lang": "en", "location": "Chicago", "create_at": date("2012-06-28"), "description": "Co-Founder & Director of Sports at FANChise|Vol4Life|My kids are my heart|Former NFL Safety|Do stuff on Tv.", "followers_count": 13149, "friends_count": 2714, "statues_count": 1443 }, "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-05T10:00:24.000Z"), "id": 684434287513305089, "text": "Keep posting about me, just makes you look like an idiot.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 574715694, "name": "Haaaaaaaaannnah", "screen_name": "hannahlinker", "lang": "en", "location": "Conway, AR", "create_at": date("2012-05-08"), "description": "Sophie Kay Linker❤️ rip BAD.", "followers_count": 481, "friends_count": 319, "statues_count": 20023 }, "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-05T10:00:25.000Z"), "id": 684434287756619776, "text": "Im getting my tipped pierced", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2205086072, "name": "H Fen", "screen_name": "F1Fenley", "lang": "en", "location": "null", "create_at": date("2013-11-20"), "description": "fishing, or with @Maariiahh_", "followers_count": 890, "friends_count": 225, "statues_count": 21621 }, "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-05T10:00:25.000Z"), "id": 684434287995703296, "text": "I don't trip when I'm at my lowest.. I mean I can only go up from here on", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 820363182, "name": "CLIFF$AVAGE", "screen_name": "ffb_23", "lang": "en", "location": "Houston,Tx", "create_at": date("2012-09-12"), "description": "4'5 #SHSU19 #VSIONS Disregard females, Acquire currency #SluhTeeBoyz", "followers_count": 1027, "friends_count": 992, "statues_count": 14781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:25.000Z"), "id": 684434288050352129, "text": "I met too many niggas moms and it wasn't real https://t.co/yYMJdKqF7Z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3085460280, "name": "Inconsistent Queen", "screen_name": "oddfuckingcaro", "lang": "en", "location": "null", "create_at": date("2015-03-15"), "description": "Next to Mona Lisa, the modern day version with better features", "followers_count": 352, "friends_count": 357, "statues_count": 5877 }, "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-01-05T10:00:25.000Z"), "id": 684434288176136192, "text": "Gustavo Hasbún, eres un mentecato desalmado y mal parido. La cara de weón concuerda perfectamente con cada estupidez q haces y dices.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 143953434, "name": "Sultán", "screen_name": "matitas_weedson", "lang": "es", "location": "null", "create_at": date("2010-05-14"), "description": "Emprendedor, trotamundos mental, psiconauta, #cruzado, defensor de los \nD° Individuales y beodo. ¡Moriré viviendo, pero no viviré muriendo!", "followers_count": 1187, "friends_count": 1198, "statues_count": 58962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Zion, IL", "id": "01201c4051c783ff", "name": "Mount Zion", "place_type": "city", "bounding_box": rectangle("-88.914362,39.762976 -88.838896,39.805346") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17115, "countyName": "Macon", "cityID": 1751206, "cityName": "Mount Zion" } }
+{ "create_at": datetime("2016-01-05T10:00:25.000Z"), "id": 684434288322809856, "text": "RN / REGISTERED NURSE / ER RN / EMERGENCY ROOM NURSE - Aureus Medical Group: (#Richmond, VA) https://t.co/m1sI5jCQXu #Nursing #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.469092,37.542979"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Richmond", "Nursing", "Job", "Jobs" }}, "user": { "id": 26802909, "name": "TMJ-RCH Nursing Jobs", "screen_name": "tmj_rch_nursing", "lang": "en", "location": "Richmond, VA", "create_at": date("2009-03-26"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Richmond, VA. Need help? Tweet us at @CareerArc!", "followers_count": 394, "friends_count": 305, "statues_count": 257 }, "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-05T10:00:25.000Z"), "id": 684434288377335808, "text": "Today is one of those bummer ass days.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 418051510, "name": "RodWayne", "screen_name": "RodWayneSaidIt", "lang": "en", "location": "Arlington Texas", "create_at": date("2011-11-21"), "description": "/iHeart Media/Talk-Radio/Radio-Personality/Motivator/Lifer/Occasional Asshole/...Dedicated to being me! Focused on bettering me! Now lets get it!", "followers_count": 527, "friends_count": 76, "statues_count": 15734 }, "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-05T10:00:25.000Z"), "id": 684434288536891392, "text": "I fucked that all up ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 288523557, "name": "Timbo'", "screen_name": "JuiceDaddyTimbo", "lang": "en", "location": "w/ Mary J.", "create_at": date("2011-04-26"), "description": "#LongLiveTK ♥️", "followers_count": 1471, "friends_count": 915, "statues_count": 96388 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creedmoor, NC", "id": "01a5e2ff6f5705f3", "name": "Creedmoor", "place_type": "city", "bounding_box": rectangle("-78.702421,36.102953 -78.640174,36.130814") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37077, "countyName": "Granville", "cityID": 3715320, "cityName": "Creedmoor" } }
+{ "create_at": datetime("2016-01-05T10:00:25.000Z"), "id": 684434289119916033, "text": "������ https://t.co/7DEy7ixPdi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.1497999,32.3336835"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4127697154, "name": "RESTEASYPOP", "screen_name": "BamOutchea", "lang": "en", "location": "somewhere gassin up ⛽ #SINCITY", "create_at": date("2015-11-06"), "description": "Phenix city 334 , senior c/o16 #GDN #StaySolid", "followers_count": 469, "friends_count": 1288, "statues_count": 573 }, "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": 1113, "countyName": "Russell" } }
+{ "create_at": datetime("2016-01-05T10:00:25.000Z"), "id": 684434289707085824, "text": "If you aren't keeping up with this blog and find politics amusing, you need to start! https://t.co/nFElZvGRKO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3177534191, "name": "Chris Wolniewicz", "screen_name": "ChrisWolniewicz", "lang": "en", "location": "null", "create_at": date("2015-04-17"), "description": "Digital Marketer in Toledo, Avid Reader, Metal Snob, Real Estate Catcher Upper, RE/MAX Preferred Associates", "followers_count": 165, "friends_count": 240, "statues_count": 2068 }, "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-05T10:00:25.000Z"), "id": 684434290080354304, "text": "@dpshphrd23 https://t.co/IQnD3Q9yLw", "in_reply_to_status": -1, "in_reply_to_user": 2335751065, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2335751065 }}, "user": { "id": 391071169, "name": "Spartan-051", "screen_name": "DShep150", "lang": "en", "location": "Delta Halo", "create_at": date("2011-10-14"), "description": "UNSC ODST, ONI Operative \n\nFeet First Into Hell", "followers_count": 541, "friends_count": 476, "statues_count": 58136 }, "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-05T10:00:25.000Z"), "id": 684434290117967872, "text": "Is blockbuster still a thing anywhere? Lol https://t.co/PEC8lAUuql", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 167013342, "name": "Matty Cleveland", "screen_name": "MarronMatt", "lang": "en", "location": "CLE til' I die ", "create_at": date("2010-07-15"), "description": "The Man Behind @SC_Cleveland|Diehard CLE fan|Retired NCAA Track Athlete|Mount Union Alum|John 10:28|Radio Personality|@M_Sheerer is Bae| SCcleveland1@gmail.com", "followers_count": 5977, "friends_count": 557, "statues_count": 28432 }, "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-05T10:00:25.000Z"), "id": 684434290340278272, "text": "Kindred Hospital: Environmental Services Aide - Per Diem - Various Shifts - Kindred... (#SantaAna, CA) https://t.co/kjHeumF4o0 #Labor #Job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.8678338,33.7455731"), "retweet_count": 0, "lang": "da", "is_retweet": false, "hashtags": {{ "SantaAna", "Labor", "Job" }}, "user": { "id": 27316005, "name": "TMJ-CAO Labor Jobs", "screen_name": "tmj_cao_labor", "lang": "en", "location": "Orange County, CA", "create_at": date("2009-03-28"), "description": "Follow this account for geo-targeted General Labor job tweets in Orange County, CA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 327, "friends_count": 292, "statues_count": 44 }, "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-05T10:00:25.000Z"), "id": 684434290428526592, "text": "@JasonLawNews @WCPO goes by forest thomer", "in_reply_to_status": 684433543808839680, "in_reply_to_user": 19683460, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19683460, 17068146 }}, "user": { "id": 477151965, "name": "keith henry", "screen_name": "keithfour", "lang": "en", "location": "null", "create_at": date("2012-01-28"), "description": "Since 1979 #bbn #whodey", "followers_count": 166, "friends_count": 582, "statues_count": 5717 }, "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-05T10:00:25.000Z"), "id": 684434290478718977, "text": "TO WHOM IT CONCERNS: Even tho we have specific info on a thing;at times it's best to allow those in authority to share it. Stay n your lane!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25958426, "name": "Prophetess Buchanan™", "screen_name": "getemworldwide", "lang": "en", "location": "St. Louis, Missouri", "create_at": date("2009-03-22"), "description": "Founder & Visionary of GETEM Worldwide;Author:HealedofAIDS, Prophetess; manifesting Healing by God in your life, moving in gifts of healing,prophecy.", "followers_count": 2754, "friends_count": 878, "statues_count": 25045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellefontaine Neighbors, MO", "id": "2a939ca3e91a24e8", "name": "Bellefontaine Neighbors", "place_type": "city", "bounding_box": rectangle("-90.250586,38.723416 -90.203155,38.772528") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2904222, "cityName": "Bellefontaine Neighbors" } }
+{ "create_at": datetime("2016-01-05T10:00:25.000Z"), "id": 684434290533257216, "text": "Want to work at dressbarn? We're #hiring in #Levittown, NY! Click for details: https://t.co/zTbBCFuSho #Retail #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.5181833,40.7226698"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Levittown", "Retail", "Job", "Jobs", "CareerArc" }}, "user": { "id": 59695127, "name": "TMJ-NY Retail Jobs", "screen_name": "tmj_ny_retail", "lang": "en", "location": "New York", "create_at": date("2009-07-23"), "description": "Follow this account for geo-targeted Retail job tweets in New York Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 466, "friends_count": 308, "statues_count": 1982 }, "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-01-05T10:00:25.000Z"), "id": 684434290554191873, "text": "Can you recommend anyone for this #Retail #job? https://t.co/2OUUUvvpU5 #Springfield, MA #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.5189025,42.1370712"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "Springfield", "Hiring", "CareerArc" }}, "user": { "id": 28708018, "name": "TMJ-CTN Retail Jobs", "screen_name": "tmj_ctn_retail", "lang": "en", "location": "Greater Enfield, CT", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Enfield, CT from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 375, "friends_count": 309, "statues_count": 70 }, "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-05T10:00:25.000Z"), "id": 684434290977853440, "text": "@Nicholauslopez dude no one cares", "in_reply_to_status": 684219963628863488, "in_reply_to_user": 514663956, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 514663956 }}, "user": { "id": 554754026, "name": "Johnny Quinn", "screen_name": "johnnyq_22", "lang": "en", "location": "Texas Christian University 19'", "create_at": date("2012-04-15"), "description": "TCU '19 / DLS '15 / ΦΚΣ", "followers_count": 722, "friends_count": 711, "statues_count": 3218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alamo, CA", "id": "6890a27317bf4a9b", "name": "Alamo", "place_type": "city", "bounding_box": rectangle("-122.067214,37.827626 -121.960052,37.882035") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 600618, "cityName": "Alamo" } }
+{ "create_at": datetime("2016-01-05T10:00:25.000Z"), "id": 684434291024080901, "text": "Who's down for some lunch?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 190114802, "name": "Cody Hickman", "screen_name": "Hickman_Cody", "lang": "en", "location": "Soddy Daisy, Tn", "create_at": date("2010-09-12"), "description": "Call me cody, own a boat love the lake and summer. snapchat codyhickman instagram: cody hickman", "followers_count": 742, "friends_count": 1011, "statues_count": 4035 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Soddy-Daisy, TN", "id": "6e9399e856233b5c", "name": "Soddy-Daisy", "place_type": "city", "bounding_box": rectangle("-85.237112,35.198568 -85.098121,35.305505") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47065, "countyName": "Hamilton", "cityID": 4769560, "cityName": "Soddy-Daisy" } }
+{ "create_at": datetime("2016-01-05T10:00:25.000Z"), "id": 684434291233832960, "text": "Shadows on tha hahbah @ East Boston Ma 02128 https://t.co/JrjBlTHM3s", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.02978744,42.3776649"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 65710665, "name": "Jon O", "screen_name": "jonoj123", "lang": "en", "location": "null", "create_at": date("2009-08-14"), "description": "null", "followers_count": 267, "friends_count": 990, "statues_count": 468 }, "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-05T10:00:25.000Z"), "id": 684434291237859328, "text": "Can you recommend anyone for this #Retail #job? https://t.co/irdAIpvf0w #Buffalo, NY #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.8718193,42.9554677"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "Buffalo", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 28697749, "name": "TMJ-BUF Retail Jobs", "screen_name": "tmj_buf_retail", "lang": "en", "location": "Buffalo, NY", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Buffalo, NY. Need help? Tweet us at @CareerArc!", "followers_count": 368, "friends_count": 309, "statues_count": 168 }, "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-05T10:00:25.000Z"), "id": 684434291401469953, "text": "Lol whu https://t.co/Sj6FLiF9n7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 409151781, "name": "Genesis", "screen_name": "INDlVlDUAL", "lang": "en", "location": "LOS ANGELES | BAY AREA", "create_at": date("2011-11-10"), "description": "Grew up in the 90s. Bay ✈LA. Father. Overthinker. Money Maker. #49ers Faithful. DubNation.", "followers_count": 240, "friends_count": 842, "statues_count": 22319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:25.000Z"), "id": 684434291426738176, "text": "Loving & genuinely being loved is pretty much the shit haha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 420263330, "name": "jana lei macaya", "screen_name": "janaleiii", "lang": "en", "location": "null", "create_at": date("2011-11-24"), "description": "24. Loves to Travel. Aspiring LaSallian, MD. DMCI Agent. BSN (AUP) Latin Honors. Lifestyle Medicine Delegate. Generation Y, 1991", "followers_count": 254, "friends_count": 183, "statues_count": 5160 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Villas, FL", "id": "5f233dab0bebb079", "name": "Villas", "place_type": "city", "bounding_box": rectangle("-81.88672,26.52626 -81.854289,26.576634") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12071, "countyName": "Lee", "cityID": 1274512, "cityName": "Villas" } }
+{ "create_at": datetime("2016-01-05T10:00:25.000Z"), "id": 684434291611299842, "text": "@CarrieMathinson @SHO_Homeland \nDon't pull plug On Quinn Carrie.. it should never have gone that far #SaveQuinn @greensboro_nc @TwitWhizz", "in_reply_to_status": -1, "in_reply_to_user": 3431166794, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SaveQuinn" }}, "user_mentions": {{ 3431166794, 250205792, 28918980, 922847460 }}, "user": { "id": 861548858, "name": "@cindycapo https://t", "screen_name": "cindycapo", "lang": "en", "location": "IL,USA", "create_at": date("2012-10-04"), "description": "Social Media Enthusiast! #Twitter #FaceBook #G+ #EmpireAve, Social Media, Friends, Opportunities, #TeamRT2EAT @RT2EAT #TeamTwitWhizz NO MGWV,Porn,DM, Nude=BLOCK", "followers_count": 26787, "friends_count": 8426, "statues_count": 163110 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grayslake, IL", "id": "26d88ba39cce8290", "name": "Grayslake", "place_type": "city", "bounding_box": rectangle("-88.067992,42.305548 -87.992956,42.388167") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1731121, "cityName": "Grayslake" } }
+{ "create_at": datetime("2016-01-05T10:00:25.000Z"), "id": 684434291682590720, "text": "@Greg_Bloom This would be so stupid, gotta keep the reigns in, the best of the best on here gotta keep it tight!", "in_reply_to_status": 684433589837107200, "in_reply_to_user": 337889478, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 337889478 }}, "user": { "id": 45972356, "name": "Nick Rojas", "screen_name": "ROJO36", "lang": "en", "location": "Barrington, RI (401/607)", "create_at": date("2009-06-09"), "description": "Video GA/Reporter for the PC Friars. Proud Barrington High alum, prouder Ithaca College alum. Aspiring Sports reporter. Yeah, I tweet too much.", "followers_count": 1332, "friends_count": 1071, "statues_count": 50016 }, "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-05T10:00:25.000Z"), "id": 684434291745538048, "text": "@ikaur84 New Year New me.", "in_reply_to_status": 684434185315041281, "in_reply_to_user": 380527922, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 380527922 }}, "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": 55, "friends_count": 56, "statues_count": 1197 }, "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-05T10:00:25.000Z"), "id": 684434291749617664, "text": "@MSF_Sea \nWe were speaking of the people who migrate for economic reasons. you rightly say, there is visa process for them.", "in_reply_to_status": 684432778692276225, "in_reply_to_user": 3163284887, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3163284887 }}, "user": { "id": 53902535, "name": "suematoo", "screen_name": "cindymannings", "lang": "en", "location": "NYC,West Palm Bch,Des Moines", "create_at": date("2009-07-05"), "description": "For first time in my adult life, I am proud of a presidential candidate. It warms my heart to be alive during the (Summer) (Autumn) Winter of Trump. RT= AGREE", "followers_count": 441, "friends_count": 227, "statues_count": 4444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Johnston, IA", "id": "47e37ed19f91e784", "name": "Johnston", "place_type": "city", "bounding_box": rectangle("-93.755921,41.650862 -93.667507,41.732303") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1939765, "cityName": "Johnston" } }
+{ "create_at": datetime("2016-01-05T10:00:25.000Z"), "id": 684434291753893889, "text": "Need food. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 277328754, "name": "Mason Wallace", "screen_name": "_MasonT_", "lang": "en", "location": "Tampa, FL", "create_at": date("2011-04-04"), "description": "Thoughts become things.", "followers_count": 538, "friends_count": 248, "statues_count": 17517 }, "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-05T10:00:26.000Z"), "id": 684434292114468865, "text": "A+ A single man always has choices. Single women do as well. https://t.co/v41zSF8CUg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "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": 830, "friends_count": 2066, "statues_count": 116026 }, "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-05T10:00:26.000Z"), "id": 684434292122976256, "text": "I'm at Trident Center Parking Lot in Los Angeles, CA https://t.co/NSTv2OV3kE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.43136174,34.03863809"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24994477, "name": "Nikki", "screen_name": "hapaniki", "lang": "en", "location": "null", "create_at": date("2009-03-17"), "description": "Did someone say Happy Hour?", "followers_count": 181, "friends_count": 180, "statues_count": 25946 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:26.000Z"), "id": 684434292160593920, "text": "Practicing in the rain is always so 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": 1132652330, "name": "Melody❁", "screen_name": "meloooodddyy", "lang": "en", "location": "null", "create_at": date("2013-01-29"), "description": "Take a shot for me", "followers_count": 728, "friends_count": 706, "statues_count": 13252 }, "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-01-05T10:00:26.000Z"), "id": 684434292886212609, "text": "Product Specialist III - Fidelity Investments: (#Merrimack, NH) https://t.co/OGArst1Hxm #InvestmentBanking #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.4948355,42.8678651"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Merrimack", "InvestmentBanking", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 207352965, "name": "NH Inv. Banking", "screen_name": "tmj_NH_invbank", "lang": "en", "location": "New Hampshire", "create_at": date("2010-10-24"), "description": "Follow this account for geo-targeted Investment Banking job tweets in New Hampshire Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 239, "friends_count": 235, "statues_count": 12 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merrimack, NH", "id": "000a400a1aba18e8", "name": "Merrimack", "place_type": "city", "bounding_box": rectangle("-71.589882,42.790953 -71.453424,42.910922") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33011, "countyName": "Hillsborough", "cityID": 3321780, "cityName": "East Merrimack" } }
+{ "create_at": datetime("2016-01-05T10:00:26.000Z"), "id": 684434293045641216, "text": "boys better stop that Twitter nigga shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54670812, "name": "Braze", "screen_name": "Kaloslows23", "lang": "en", "location": "$$$$$$ , TX", "create_at": date("2009-07-07"), "description": "20. God is Great Weed is good . RIP Unc . #FOE4L $ V O", "followers_count": 2583, "friends_count": 996, "statues_count": 183040 }, "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-05T10:00:26.000Z"), "id": 684434293125320706, "text": "this coconut milk diet got a niggah feeling sick RN ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1531060333, "name": "Näe", "screen_name": "fifthwardnae", "lang": "en", "location": "5th Ward, Houston tx.", "create_at": date("2013-06-19"), "description": "ROLAND is zaddy ❤️ rip to my grandparents.✨ LBLOCKNATION ❕SENIOR CLASS OF 16 snapchat:badgaaldaj", "followers_count": 845, "friends_count": 747, "statues_count": 7617 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:26.000Z"), "id": 684434293255307264, "text": "@lokdwg236 @amgreenwell1 @Boxy_box236 y'all some fools ������", "in_reply_to_status": 684433826496380928, "in_reply_to_user": 3238497542, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3238497542, 2768846697, 3176764689 }}, "user": { "id": 106826703, "name": "♨⭐Split The RELO™♏⭐♨", "screen_name": "Makin_MovesRel", "lang": "en", "location": "in the ✂ flexin & finessin", "create_at": date("2010-01-20"), "description": "SoundCloud SplitTheRelo SC&Vine REL_COOLIN D[M]V past cloud 9 #instagram REL_COOLIN #TeamScorpio #RedskinsNation #TeamLSU #TeamLakers #TeamJayhawks", "followers_count": 620, "friends_count": 712, "statues_count": 20605 }, "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-05T10:00:26.000Z"), "id": 684434293444231168, "text": "@_xPerks DEEEZZZZ NUTZZZZ ������", "in_reply_to_status": 684434171662610432, "in_reply_to_user": 211366359, "favorite_count": 0, "retweet_count": 0, "lang": "eu", "is_retweet": false, "user_mentions": {{ 211366359 }}, "user": { "id": 3032860419, "name": "Ron Martin", "screen_name": "_RonTreyCinco", "lang": "en", "location": "Ohio, USA", "create_at": date("2015-02-12"), "description": "Hocking College Hawks #35 Forward", "followers_count": 241, "friends_count": 276, "statues_count": 2437 }, "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-05T10:00:26.000Z"), "id": 684434293637042176, "text": "@BlakePangman22 how dare you unfollow a fellow @SacramentoKings fanatic! For shame!", "in_reply_to_status": -1, "in_reply_to_user": 3233130019, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3233130019, 667563 }}, "user": { "id": 2694210090, "name": "Trevor", "screen_name": "tdanginc", "lang": "en", "location": "Sac Town", "create_at": date("2014-07-30"), "description": "Family (husband, father of 2), @RAIDERS, @SacramentoKings, @Dodgers, @parkwayofficial, #TWD #GOT and BEARD!", "followers_count": 945, "friends_count": 817, "statues_count": 5464 }, "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.714333") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 624638, "cityName": "Folsom" } }
+{ "create_at": datetime("2016-01-05T10:00:26.000Z"), "id": 684434293938995200, "text": "I wanna write for @AltPress", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14925700 }}, "user": { "id": 25631837, "name": "zoe macall", "screen_name": "ZoeSarrett", "lang": "en", "location": "Arizona", "create_at": date("2009-03-20"), "description": "Music / Dallas Cowboys / Bethesda Games /", "followers_count": 571, "friends_count": 361, "statues_count": 10563 }, "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-05T10:00:26.000Z"), "id": 684434294048083968, "text": "@Ramirian9 haha hmu I got you I'll teach you that shit", "in_reply_to_status": 684423234037526528, "in_reply_to_user": 1365104420, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1365104420 }}, "user": { "id": 1205448091, "name": "Titty Mayne", "screen_name": "Tetonsitoyabish", "lang": "en", "location": "Alhambra ", "create_at": date("2013-02-21"), "description": "You miss a hundred percent of the shots you dont take. (Insert Inspirational Quote here). 77-52 FIFA RECORD", "followers_count": 350, "friends_count": 302, "statues_count": 13759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:26.000Z"), "id": 684434294060617728, "text": "I promise when I find em...I will share the gems with u. Haha https://t.co/hq8xZO4bTS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 341699520, "name": "Stefanie Seifer", "screen_name": "StefSeifer", "lang": "en", "location": "Los Angeles", "create_at": date("2011-07-24"), "description": "Actress / Creator / Human - Co-Founder of http://DirtyandThirty.com Create + Inspire + Play = meaning of life. I ❤️ sour beer + yoga! http://www.xostefanie.com", "followers_count": 15290, "friends_count": 8510, "statues_count": 4545 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:26.000Z"), "id": 684434294127738880, "text": "Check out our #listing in #Lincolnton #NC https://t.co/Dh3Eo33igr #realestate #realtor https://t.co/I7fkwDMwW9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3063845,35.5774643"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "Lincolnton", "NC", "realestate", "realtor" }}, "user": { "id": 2445654229, "name": "RE/MAX Crossroads", "screen_name": "lincolntonremax", "lang": "en", "location": "Lincolnton NC", "create_at": date("2014-04-15"), "description": "null", "followers_count": 71, "friends_count": 43, "statues_count": 6132 }, "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.32187,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37035, "countyName": "Catawba" } }
+{ "create_at": datetime("2016-01-05T10:00:26.000Z"), "id": 684434294517833728, "text": "Imma tell my boy Dylan happy birthday when everyone gets of his dick��", "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": 860, "friends_count": 698, "statues_count": 14087 }, "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-05T10:00:26.000Z"), "id": 684434294719299584, "text": "If Dennis gives me anymore sass he's walking home", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1544804522, "name": "Cole Loren", "screen_name": "Kingcole262", "lang": "en", "location": "Wesley Chapel, FL", "create_at": date("2013-06-24"), "description": "Bring me food", "followers_count": 182, "friends_count": 190, "statues_count": 6739 }, "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("2016-01-05T10:00:26.000Z"), "id": 684434295008575489, "text": "�������� https://t.co/6lE4Ekf2jM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2492188784, "name": "Alejandra Contreras", "screen_name": "ale_33contreras", "lang": "en", "location": "Oakland, CA", "create_at": date("2014-05-12"), "description": "Ahi que darle gusto al gusto. La vida pronto se acaba -Ramon Ayala", "followers_count": 179, "friends_count": 153, "statues_count": 2415 }, "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-05T10:00:26.000Z"), "id": 684434295419580417, "text": "@goshminna happy birthday gorgeous! ����✨��", "in_reply_to_status": -1, "in_reply_to_user": 2738735325, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2738735325 }}, "user": { "id": 2777319581, "name": "champagnepapi", "screen_name": "siennasulema", "lang": "en", "location": "null", "create_at": date("2014-09-19"), "description": "I'm still on my queen shit", "followers_count": 190, "friends_count": 117, "statues_count": 4484 }, "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-05T10:00:26.000Z"), "id": 684434295671357440, "text": "@cassssiiiiie weirdo", "in_reply_to_status": 684434011196911616, "in_reply_to_user": 1239845094, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1239845094 }}, "user": { "id": 367560659, "name": "SPICY C H R I $", "screen_name": "spicyychris", "lang": "en", "location": "where the wind blows", "create_at": date("2011-09-03"), "description": "It's survival of the smoothest and that's the way it goes", "followers_count": 526, "friends_count": 360, "statues_count": 5890 }, "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-05T10:00:26.000Z"), "id": 684434295801253888, "text": "It bugs me when people post pictures of food they made when it looks really spa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 270553063, "name": "Robert Carter", "screen_name": "H3yMrCarter", "lang": "en", "location": "Texas", "create_at": date("2011-03-22"), "description": "... my mom thinks I'm funny\n\nI live tweet Ranger games sometimes", "followers_count": 116, "friends_count": 174, "statues_count": 1987 }, "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-01-05T10:00:26.000Z"), "id": 684434295851741184, "text": "Temp 32.2°F Wind Chill 29.5°F RH 35% Wind 3.0 ESE Gust 9.0 ESE SLP 30.633 in Falling Rain 0.00 in Solar 540 UV 2.5 #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": 102, "friends_count": 65, "statues_count": 32305 }, "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-05T10:00:26.000Z"), "id": 684434295889477632, "text": "janelle monae is such a beautiful artist", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3249519047, "name": "petty flacko", "screen_name": "pettyprincesa", "lang": "en", "location": "null", "create_at": date("2015-05-12"), "description": "woah", "followers_count": 117, "friends_count": 81, "statues_count": 1091 }, "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-05T10:00:26.000Z"), "id": 684434295939702784, "text": "When the starvation is real 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": 414919432, "name": "01/22 ♒️", "screen_name": "_myoni", "lang": "en", "location": "null", "create_at": date("2011-11-17"), "description": "❤", "followers_count": 660, "friends_count": 582, "statues_count": 3503 }, "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-05T10:00:27.000Z"), "id": 684434296019521536, "text": "I just wanna have a few thousand put away so i can randomly book these cheap flights b.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 104195144, "name": "Gingerick Lamar", "screen_name": "odiolasgalletas", "lang": "en", "location": "Brooklyn, NYC", "create_at": date("2010-01-12"), "description": "First off, fuck your bitch and the clique you claim.", "followers_count": 1959, "friends_count": 2081, "statues_count": 180164 }, "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-05T10:00:27.000Z"), "id": 684434296124223488, "text": "I vividly recall Zay saying that when plans something it always follow through. Care to explain ? @zay_daprophet", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2493728474 }}, "user": { "id": 1003345796, "name": "#FreeTheNipple", "screen_name": "BarryR96", "lang": "en", "location": "Cali ", "create_at": date("2012-12-10"), "description": "Regret Nothing.", "followers_count": 535, "friends_count": 451, "statues_count": 18147 }, "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-05T10:00:27.000Z"), "id": 684434296208240640, "text": "@Maree3113 @pabloalboran @franciechats @breannamejia que risaaaa!! Chicago������������", "in_reply_to_status": 684432865937903618, "in_reply_to_user": 481009286, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 481009286, 119776110, 43020926, 127668159 }}, "user": { "id": 344450249, "name": "Gabby Q.", "screen_name": "gabiotita89", "lang": "en", "location": "Illinois, USA", "create_at": date("2011-07-28"), "description": "My musical tastes are ECLECTIC.", "followers_count": 858, "friends_count": 733, "statues_count": 30328 }, "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-05T10:00:27.000Z"), "id": 684434296300388352, "text": "Theworld is sick of your lies! All #business owners are!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "business" }}, "user": { "id": 456726486, "name": "HOLIDAY HEADQUARTERS", "screen_name": "sportparadise", "lang": "en", "location": "null", "create_at": date("2012-01-06"), "description": "#Holiday #Shop #Gifts #Sales #Deals #Bargains #Ebay #Clothing #Fashion #Jewelry #Hunting #Electronics #HomeDecor #Toys #Automotive #teamfollowback", "followers_count": 101898, "friends_count": 94152, "statues_count": 935512 }, "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-05T10:00:27.000Z"), "id": 684434296405405696, "text": "Still light #SnOBX falling at @WrightBrosNPS in KDH #OBX #NCWX https://t.co/Qs2lu803aI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SnOBX", "OBX", "NCWX" }}, "user_mentions": {{ 126689571 }}, "user": { "id": 19184715, "name": "Sam Walker #OBXNews", "screen_name": "SamWalkerOBX", "lang": "en", "location": "Outer Banks, NC", "create_at": date("2009-01-19"), "description": "News Director for Max Radio Of The Carolinas, Staff Writer at http://OuterBanksVoice.com, and Walt Disney World Addict. May the good news be yours.", "followers_count": 4185, "friends_count": 3503, "statues_count": 48886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kill Devil Hills, NC", "id": "282ff353934c2305", "name": "Kill Devil Hills", "place_type": "city", "bounding_box": rectangle("-75.692503,35.98436 -75.643716,36.055349") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37055, "countyName": "Dare", "cityID": 3735720, "cityName": "Kill Devil Hills" } }
+{ "create_at": datetime("2016-01-05T10:00:27.000Z"), "id": 684434296413777920, "text": "Some things never happen because people are too scared to just simply talk ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 766462202, "name": "carli", "screen_name": "clynn_101512", "lang": "en", "location": "Ohio, USA", "create_at": date("2012-08-18"), "description": "my hair is longer than yours", "followers_count": 865, "friends_count": 234, "statues_count": 40141 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glouster, OH", "id": "a36de84618bb06cc", "name": "Glouster", "place_type": "city", "bounding_box": rectangle("-82.106503,39.493281 -82.066435,39.515284") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39009, "countyName": "Athens", "cityID": 3930674, "cityName": "Glouster" } }
+{ "create_at": datetime("2016-01-05T10:00:27.000Z"), "id": 684434296472338432, "text": "#Independence, OH #Nursing #Job: RN Hospice at Cleveland Clinic https://t.co/nS4MsXBIWO #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.6379033,41.3686645"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Independence", "Nursing", "Job", "Jobs", "Hiring" }}, "user": { "id": 2843895727, "name": "ClevelandClinic Jobs", "screen_name": "CleClinicJobs", "lang": "en", "location": "Northeast Ohio", "create_at": date("2014-10-07"), "description": "As one of the nation’s premier hospitals, we offer superior opportunities to those ready for a vital role in the future of medicine providing world class care.", "followers_count": 429, "friends_count": 10, "statues_count": 1452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Independence, OH", "id": "e8e3c182dd6fe3be", "name": "Independence", "place_type": "city", "bounding_box": rectangle("-81.663505,41.350892 -81.599376,41.420653") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3937240, "cityName": "Independence" } }
+{ "create_at": datetime("2016-01-05T10:00:27.000Z"), "id": 684434296489168896, "text": "Hate that gotta walk almost a mile to my classes!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 580576708, "name": "Amber Jones", "screen_name": "Amber44J", "lang": "en", "location": "Neverland ✨", "create_at": date("2012-05-14"), "description": "21. Native American. famILY. Wildland Firefighter. CBC Hawks. Rock n Roll in my soul. \\m/", "followers_count": 801, "friends_count": 312, "statues_count": 52781 }, "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-05T10:00:27.000Z"), "id": 684434296526868480, "text": "@luperangel230 ����������", "in_reply_to_status": 684365740825169921, "in_reply_to_user": 719121432, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 719121432 }}, "user": { "id": 860914908, "name": "Denise Lynett Muñoz", "screen_name": "lynettmunoz", "lang": "en", "location": "McAllen, Tx", "create_at": date("2012-10-03"), "description": "20.❤️ Snapchat/Instagram: Lynettmunoz", "followers_count": 3506, "friends_count": 3680, "statues_count": 42306 }, "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-05T10:00:27.000Z"), "id": 684434296577212416, "text": "Hope you know the feds have Instagram", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2720651959, "name": "Brendon J.", "screen_name": "illmqtic", "lang": "en", "location": "Denver, CO", "create_at": date("2014-08-09"), "description": "water and grease", "followers_count": 218, "friends_count": 164, "statues_count": 1312 }, "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-05T10:00:27.000Z"), "id": 684434296581443584, "text": "Ya Lo supere ya no duele tanto", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 946828538, "name": "Alonso ⚽️", "screen_name": "the_real_gordo_", "lang": "en", "location": "null", "create_at": date("2012-11-13"), "description": "Follow me on IG @alonsoisdope #OcPumas #18", "followers_count": 849, "friends_count": 733, "statues_count": 43524 }, "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-05T10:00:27.000Z"), "id": 684434296636071936, "text": "if you want to be mad, making a murderer on Netflix is for you 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": 948664524, "name": "chlo ✨", "screen_name": "chloeeberghorst", "lang": "en", "location": "null", "create_at": date("2012-11-14"), "description": "fox enthusiast. average picture taking potato.", "followers_count": 349, "friends_count": 131, "statues_count": 8827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jenison, MI", "id": "49e7adaebc60092a", "name": "Jenison", "place_type": "city", "bounding_box": rectangle("-85.921923,42.885184 -85.781818,42.936233") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26139, "countyName": "Ottawa", "cityID": 2641680, "cityName": "Jenison" } }
+{ "create_at": datetime("2016-01-05T10:00:27.000Z"), "id": 684434296644349952, "text": "Have an 401k w/previous employer and its costing u fees We can help\nhttps://t.co/fCwpzK3x4U…/here-s-how-to-make-your-retireme…\n5059033142", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36591754, "name": "Victor A. Montoya", "screen_name": "mav7995", "lang": "en", "location": "Santa Fe", "create_at": date("2009-04-29"), "description": "null", "followers_count": 1065, "friends_count": 1775, "statues_count": 5507 }, "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-05T10:00:27.000Z"), "id": 684434296669474816, "text": "I need a baby shower theme something ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1131309144, "name": "sc:felicialanaee", "screen_name": "Felicialanae_", "lang": "en", "location": "Silsbee, TX.", "create_at": date("2013-01-29"), "description": "null", "followers_count": 3482, "friends_count": 1846, "statues_count": 65959 }, "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-01-05T10:00:27.000Z"), "id": 684434296673669120, "text": "#Healthcare #Job alert: Seasonal Drivers | Fed Ex Smart Post | #Milwaukee, WI https://t.co/EeohY5N5rO #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.92,43.03"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "Job", "Milwaukee", "Jobs", "Hiring" }}, "user": { "id": 21728169, "name": "Milwaukee Health", "screen_name": "tmj_mke_health", "lang": "en", "location": "Milwaukee, WI", "create_at": date("2009-02-23"), "description": "Follow this account for geo-targeted Healthcare job tweets in Milwaukee, WI. Need help? Tweet us at @CareerArc!", "followers_count": 620, "friends_count": 302, "statues_count": 641 }, "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-05T10:00:27.000Z"), "id": 684434297047023616, "text": "Don't go to hooters, you can't focus on food. Just assssssss and booooobs ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1335411536, "name": "Mia", "screen_name": "Mama_mia_232323", "lang": "en", "location": "Lanuola ", "create_at": date("2013-04-07"), "description": "A fire for every experience and an obsession for freedom.", "followers_count": 911, "friends_count": 854, "statues_count": 86324 }, "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-05T10:00:27.000Z"), "id": 684434297235705856, "text": "This #Hospitality #job might be a great fit for you: Housekeeper-Good Samaritan Health Center (Casual, Variable) - https://t.co/dvJTLUoUmV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.683459,45.1805223"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job" }}, "user": { "id": 88318809, "name": "TMJ-WI HRTA Jobs", "screen_name": "tmj_wi_hrta", "lang": "en", "location": "Wisconsin", "create_at": date("2009-11-07"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Wisconsin Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 319, "friends_count": 290, "statues_count": 366 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merrill, WI", "id": "00365e0944c67663", "name": "Merrill", "place_type": "city", "bounding_box": rectangle("-89.758894,45.141203 -89.640881,45.211245") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55069, "countyName": "Lincoln", "cityID": 5551250, "cityName": "Merrill" } }
+{ "create_at": datetime("2016-01-05T10:00:27.000Z"), "id": 684434297801977856, "text": "I would but ..... ��\nI'm lazy �� https://t.co/VOxmLADiFM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2842767289, "name": "saltyerykah", "screen_name": "xo_erykaaah", "lang": "en", "location": "null", "create_at": date("2014-10-06"), "description": "null", "followers_count": 382, "friends_count": 288, "statues_count": 7475 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:27.000Z"), "id": 684434298036981760, "text": "We dont have no specific victims we want all yall", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 385095212, "name": "$$$$$$$", "screen_name": "4up_Keitdron", "lang": "en", "location": "North Charlotte", "create_at": date("2011-10-04"), "description": "4s Up, Hoes Down .... R.I.P TO ALL THE FALLEN SOLDIERS.... IF U WANT IT COME GET IT !!!", "followers_count": 285, "friends_count": 354, "statues_count": 4071 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pineville, NC", "id": "c5b5da4969964536", "name": "Pineville", "place_type": "city", "bounding_box": rectangle("-80.919951,35.053241 -80.85395,35.109799") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3752220, "cityName": "Pineville" } }
+{ "create_at": datetime("2016-01-05T10:00:27.000Z"), "id": 684434298041184261, "text": "@rezaaslan @Real_JMOOB Here's the thing about Gun Humpers- they are all scared shitless and think their guns make them safe.", "in_reply_to_status": 684432841732587523, "in_reply_to_user": 262856460, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 262856460, 324982257 }}, "user": { "id": 1365441361, "name": "Mechadave", "screen_name": "Anumbr1", "lang": "en", "location": "null", "create_at": date("2013-04-19"), "description": "Retired", "followers_count": 60, "friends_count": 78, "statues_count": 5867 }, "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-05T10:00:27.000Z"), "id": 684434298053640192, "text": "Want to work in #NewYork, NY? View our latest opening: https://t.co/nMFnc84gLo #Sales #Job #Jobs #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", "Sales", "Job", "Jobs", "Hiring" }}, "user": { "id": 3004246198, "name": "Colgate Jobs - U.S.", "screen_name": "ColgateJobsUS", "lang": "en", "location": "United States", "create_at": date("2015-01-29"), "description": "null", "followers_count": 1142, "friends_count": 30, "statues_count": 114 }, "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-05T10:00:27.000Z"), "id": 684434298259292160, "text": "#goodproblems is now trending in #Phoenix https://t.co/xxIKx7bKgi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "goodproblems", "Phoenix" }}, "user": { "id": 132338596, "name": "Trendsmap Phoenix", "screen_name": "TrendsPhoenix", "lang": "en", "location": "Phoenix, USA", "create_at": date("2010-04-12"), "description": "Real-time Phoenix Twitter trends", "followers_count": 751, "friends_count": 244, "statues_count": 7602 }, "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-05T10:00:27.000Z"), "id": 684434298410160128, "text": "@mandaaabearr @Molly__Jean @Kaitiii @mattymcd don't worry kids, its just a head cold.", "in_reply_to_status": 684433718493184002, "in_reply_to_user": 22430590, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 22430590, 38173453, 20468965, 14613414 }}, "user": { "id": 288419116, "name": "Dustin Adair ", "screen_name": "TheNightDusto", "lang": "en", "location": "Kansas City, Missouri ", "create_at": date("2011-04-26"), "description": "obnoxious cohost of @RFHshow. Also not a ninja.", "followers_count": 332, "friends_count": 141, "statues_count": 14182 }, "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-05T10:00:27.000Z"), "id": 684434298846507008, "text": "@sarahjoyshockey when we were very young and sometimes shared beds, my little sister would say she was my \"bed buddy.\" Anyway you look sad", "in_reply_to_status": 684432136904953856, "in_reply_to_user": 453252068, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 453252068 }}, "user": { "id": 50359913, "name": "Name cannot be blank", "screen_name": "turtletomorrow", "lang": "en", "location": "Chicago", "create_at": date("2009-06-24"), "description": "retired account, please follow me @donoldduck", "followers_count": 117, "friends_count": 175, "statues_count": 17310 }, "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-05T10:00:27.000Z"), "id": 684434299018416128, "text": "You know you went to a small college when you lose yourhigh school class ring freshman year and 5 yrs later they find it and call you...#wut", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wut" }}, "user": { "id": 334397095, "name": "Abigail Hagerman", "screen_name": "abigailhagerman", "lang": "en", "location": "null", "create_at": date("2011-07-12"), "description": "It all means more than I can tell you. So you must not judge what I know by what I find words for.", "followers_count": 291, "friends_count": 149, "statues_count": 2170 }, "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-05T10:00:27.000Z"), "id": 684434299148496898, "text": "This show is making me 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": 356893372, "name": "Kelsey Nunn", "screen_name": "kelsey_nunn", "lang": "en", "location": "Kentucky, USA", "create_at": date("2011-08-17"), "description": "all I can do is be me. whoever that is.", "followers_count": 463, "friends_count": 719, "statues_count": 4317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlington, KY", "id": "bb3030b300ec775e", "name": "Burlington", "place_type": "city", "bounding_box": rectangle("-84.763511,38.99546 -84.686274,39.04083") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21015, "countyName": "Boone", "cityID": 2111170, "cityName": "Burlington" } }
+{ "create_at": datetime("2016-01-05T10:00:27.000Z"), "id": 684434299152543744, "text": "@OScandrick32 you ain't shxt...", "in_reply_to_status": -1, "in_reply_to_user": 2189323422, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2189323422 }}, "user": { "id": 46562494, "name": "Fuzzy Slippers", "screen_name": "Wizard_Kelly103", "lang": "en", "location": "#LambroghiniMercy", "create_at": date("2009-06-11"), "description": "If you came here lookin for answers, you might as well turn TF around...", "followers_count": 277, "friends_count": 269, "statues_count": 22320 }, "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-05T10:00:27.000Z"), "id": 684434299182051328, "text": "These MeMes be too funny", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2184781852, "name": "Pretty Kee", "screen_name": "Keesoprettyyy", "lang": "en", "location": "null", "create_at": date("2013-11-17"), "description": "Parts of my are in Heaven 10.13.15 Remy you changed my life forever G4ever", "followers_count": 348, "friends_count": 339, "statues_count": 14883 }, "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.022437") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2407125, "cityName": "Bethesda" } }
+{ "create_at": datetime("2016-01-05T10:00:27.000Z"), "id": 684434299194589188, "text": "Very nice", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 923839826, "name": "cate", "screen_name": "thevxrve", "lang": "en", "location": "manhattan, NY", "create_at": date("2012-11-03"), "description": "streetlight people, living just to find emotion Ψ\n#WeMissYouCory", "followers_count": 1256, "friends_count": 280, "statues_count": 89795 }, "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-05T10:00:27.000Z"), "id": 684434299379126272, "text": "Anyways��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1306053426, "name": "Winter ✨", "screen_name": "Winntaaaa", "lang": "en", "location": "Cullowhee, NC", "create_at": date("2013-03-26"), "description": "✨blessings on blessings on blessings✨", "followers_count": 1704, "friends_count": 1124, "statues_count": 101259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clemmons, NC", "id": "290f62c6f654e14f", "name": "Clemmons", "place_type": "city", "bounding_box": rectangle("-80.432988,35.93042 -80.339893,36.078123") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37067, "countyName": "Forsyth", "cityID": 3712960, "cityName": "Clemmons" } }
+{ "create_at": datetime("2016-01-05T10:00:27.000Z"), "id": 684434299420934144, "text": "We're #hiring! Read about our latest #job opening here: Audit Manager/SOX - https://t.co/y3kENcs3bs #SantaClarita, CA #Finance", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.542586,34.3916641"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "SantaClarita", "Finance" }}, "user": { "id": 2237934078, "name": "Princess Cruises", "screen_name": "JobsAtPrincess", "lang": "en", "location": "Santa Clarita, CA ", "create_at": date("2013-12-09"), "description": "Set sail on an exciting new career course with the cruise line that helps you #comebacknew! For non-career-related news and assistance, follow @PrincessCruises.", "followers_count": 278, "friends_count": 249, "statues_count": 155 }, "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-05T10:00:27.000Z"), "id": 684434299555299328, "text": "I just took a personal stand for my freedom by joining NRA. You should too! https://t.co/a3SbLOHa3K", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 140857463, "name": "TJ Waggoner", "screen_name": "tjwaggoner", "lang": "en", "location": "Ashland, KY, USA", "create_at": date("2010-05-06"), "description": "Classical liberal/libertarian Constitutionalist. Football fan.", "followers_count": 239, "friends_count": 300, "statues_count": 4580 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Catlettsburg, KY", "id": "13c103c5de6650c9", "name": "Catlettsburg", "place_type": "city", "bounding_box": rectangle("-82.633458,38.356337 -82.592999,38.443106") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21019, "countyName": "Boyd", "cityID": 2113420, "cityName": "Catlettsburg" } }
+{ "create_at": datetime("2016-01-05T10:00:27.000Z"), "id": 684434299605528577, "text": "@mattmarler36 Welcome to UND Football and The University of North Dakota education, hope you are sucessful in both! Go Hawk's!!!", "in_reply_to_status": 684389135877058560, "in_reply_to_user": 2701132154, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2701132154 }}, "user": { "id": 2264585216, "name": "Dennis Burthwick", "screen_name": "dburthwick", "lang": "en", "location": "Moorhead, MN", "create_at": date("2013-12-27"), "description": "North Dakota backer!", "followers_count": 79, "friends_count": 192, "statues_count": 2866 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moorhead, MN", "id": "ddd9aed661af0c05", "name": "Moorhead", "place_type": "city", "bounding_box": rectangle("-96.79398,46.818321 -96.705732,46.917616") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27027, "countyName": "Clay", "cityID": 2743864, "cityName": "Moorhead" } }
+{ "create_at": datetime("2016-01-05T10:00:27.000Z"), "id": 684434299660009472, "text": "@BreAuna_Monet as your tweet sits above a few of my sneaker drop tweets -_-", "in_reply_to_status": 684169899682340864, "in_reply_to_user": 307070810, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 307070810 }}, "user": { "id": 15670969, "name": "Yves Huy Truong", "screen_name": "yveshuy", "lang": "en", "location": "Los Angeles", "create_at": date("2008-07-30"), "description": "Fashion / Editorial Photographer in Los Angeles", "followers_count": 347, "friends_count": 40, "statues_count": 1194 }, "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-05T10:00:27.000Z"), "id": 684434299722989568, "text": "@leslieeeeeeee_ I do eyebrows mascara and lipstick all the time, and I still think I look fine �� maybe they're not doing it right ����", "in_reply_to_status": 684421970264363008, "in_reply_to_user": 1322826608, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1322826608 }}, "user": { "id": 782968915, "name": "KaloniQueeenn", "screen_name": "Kaloniqueeenn", "lang": "en", "location": "null", "create_at": date("2012-08-26"), "description": "#QTX ⚓️ #HamptonU19 LV/VA ✈️ Yes my middle name is Queen", "followers_count": 947, "friends_count": 878, "statues_count": 901 }, "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-05T10:00:28.000Z"), "id": 684434300268183553, "text": "@MidwstHotMess @TexanMan72 I did not. That's not even funny.", "in_reply_to_status": 684434146345660416, "in_reply_to_user": 1196183576, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1196183576, 1549298550 }}, "user": { "id": 1236856704, "name": "Oklahoma Blondie", "screen_name": "Oklahoma_Blonde", "lang": "en", "location": "MommyLand", "create_at": date("2013-03-02"), "description": "You dont even wanna know ✌ ️ @TexanMan72 is mine ❤️ #RockChalkForever❤️ Cosmetology student ✂️", "followers_count": 895, "friends_count": 611, "statues_count": 10641 }, "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-05T10:00:28.000Z"), "id": 684434300310294528, "text": "@Indians Every Captains game we went to last year, Bobby Bradley hit at least 1 HR. https://t.co/0Y8mQ0X9Ey", "in_reply_to_status": -1, "in_reply_to_user": 52861612, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 52861612 }}, "user": { "id": 577534056, "name": "Jeremy Zura", "screen_name": "jzura74", "lang": "en", "location": "null", "create_at": date("2012-05-11"), "description": "null", "followers_count": 38, "friends_count": 98, "statues_count": 4370 }, "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-01-05T10:00:28.000Z"), "id": 684434300469555200, "text": "If you tryna get paid today hmu frfr I can put bands in ya pocket", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 195908533, "name": "RipClydeAndBigBro", "screen_name": "MrFlyandFlashy", "lang": "en", "location": "Dallas,Tx", "create_at": date("2010-09-27"), "description": "#BstillHoldin #LongLiveClyde Gotta stay ➕ in a world full of ➖#Tamuc", "followers_count": 1259, "friends_count": 1202, "statues_count": 37206 }, "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-05T10:00:28.000Z"), "id": 684434300486422531, "text": "Wind 2.0 mph SSE. Barometer 30.572 in, Falling. Temperature 27.0 °F. Rain today 0.00 in. Humidity 62%", "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": 287, "statues_count": 7279 }, "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-05T10:00:28.000Z"), "id": 684434300918431746, "text": "BLOCKED. https://t.co/q9jTY2pYrt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 277298044, "name": "LightskinJimmyButler", "screen_name": "Terence_Huie", "lang": "en", "location": "Tallahassee, FL", "create_at": date("2011-04-04"), "description": "Accountant by day, sports junkie by night. Sometimes I film basketball. Oh and I tweet a lot. I do it all.", "followers_count": 2042, "friends_count": 740, "statues_count": 188569 }, "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-05T10:00:28.000Z"), "id": 684434300964614144, "text": "#FirstDay back at #school and our #students are hard at work on their #MyOwnBusinessChallenge! https://t.co/8dlVS5g42o", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.79073367,40.73413482"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FirstDay", "school", "students", "MyOwnBusinessChallenge" }}, "user": { "id": 245642727, "name": "Youthful Savings", "screen_name": "YouthfulSavings", "lang": "en", "location": "New York, New York", "create_at": date("2011-01-31"), "description": "#EdTech company dedicated to empowering the next generation with financial education and entrepreneurship training through curriculum, technology and community", "followers_count": 1868, "friends_count": 245, "statues_count": 1556 }, "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-05T10:00:28.000Z"), "id": 684434301073633280, "text": "@THESLUMPGOD ALREADY CRODIE IMA FUCK WIT YAL THIS WEEK", "in_reply_to_status": 684428405618642944, "in_reply_to_user": 488515774, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 488515774 }}, "user": { "id": 150405342, "name": "lookatmyface", "screen_name": "DREWSWACE", "lang": "en", "location": "CROWARDCOUNTY", "create_at": date("2010-05-31"), "description": "i got fronto in my spliff. we break stagez. SWACEY:Blue Ranger OUT NOW!!!MIRAMARBOYZ Projekt Amerika//PoorAmerika. #knickstape #riseup #nyy #gocity", "followers_count": 1420, "friends_count": 691, "statues_count": 113909 }, "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-01-05T10:00:28.000Z"), "id": 684434301312565248, "text": "I hate it when I get queso on my finger and I eat my finger.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 169502073, "name": "Sean and Oates", "screen_name": "SeanInCypress", "lang": "en", "location": "Houston, TX", "create_at": date("2010-07-22"), "description": "Host of Bolivian Idol / Southwest Regional V.P. of Lionel Richie fan club / Pees sitting down from 8pm-8am", "followers_count": 10344, "friends_count": 566, "statues_count": 983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:28.000Z"), "id": 684434301635526657, "text": "\"This training leads to good jobs with growth...\" https://t.co/mq24vgeYh6 #lawjobs https://t.co/ikCZUE3rHG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "lawjobs" }}, "user": { "id": 395379436, "name": "Renton Tech", "screen_name": "RentonTech", "lang": "en", "location": "Renton, WA", "create_at": date("2011-10-21"), "description": "Renton Technical College offers world class training opportunities. Questions? We’ll help you find the answers. Tweets by the communications team.", "followers_count": 804, "friends_count": 428, "statues_count": 831 }, "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-05T10:00:28.000Z"), "id": 684434301635702784, "text": "#universalmardigras is now trending in #Orlando https://t.co/pXrcozSuhF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "universalmardigras", "Orlando" }}, "user": { "id": 132390958, "name": "Trendsmap Orlando", "screen_name": "TrendsOrlando", "lang": "en", "location": "Orlando, USA", "create_at": date("2010-04-12"), "description": "Real-time Orlando Twitter trends", "followers_count": 904, "friends_count": 341, "statues_count": 9023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "19ab9ad13ded6072", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.50773,28.347984 -81.229749,28.614251") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-01-05T10:00:28.000Z"), "id": 684434301673275392, "text": "Interested in a #Retail #job near #FortOglethorpe, GA? This could be a great fit: https://t.co/TloBbh2s26 #rue21 https://t.co/nFwEqbPk8f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.2569,34.9489645"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "FortOglethorpe", "rue21" }}, "user": { "id": 431763008, "name": "rue21", "screen_name": "rue21jobs", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2011-12-08"), "description": "null", "followers_count": 598, "friends_count": 2, "statues_count": 781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Oglethorpe, GA", "id": "7fed75edb0645240", "name": "Fort Oglethorpe", "place_type": "city", "bounding_box": rectangle("-85.272161,34.931657 -85.188694,34.986019") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13047, "countyName": "Catoosa", "cityID": 1330956, "cityName": "Fort Oglethorpe" } }
+{ "create_at": datetime("2016-01-05T10:00:28.000Z"), "id": 684434301702815744, "text": "Can We Get @ChiefKeef & @YoungThug in 2016 so I can finally decide who the best \"Trap\" Rapper/Singer is lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 51404605, 238763290 }}, "user": { "id": 221522909, "name": "GODbody GLDiE", "screen_name": "GLDchainLou", "lang": "en", "location": "Atlanta , USA", "create_at": date("2010-11-30"), "description": "Recording Artist | Graphic Designer | Free Cali | Marijuana Enthusiast | SGR | S4TD 12/20", "followers_count": 852, "friends_count": 785, "statues_count": 85487 }, "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-05T10:00:28.000Z"), "id": 684434301790896128, "text": "@RayWashingtonX she ain't hurt them just yet ������ and I ain't on my savage shit yet either", "in_reply_to_status": 684433990841987072, "in_reply_to_user": 125583991, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 125583991 }}, "user": { "id": 3679149021, "name": "RS.", "screen_name": "rayxscott", "lang": "en", "location": "on top of the pyramid ", "create_at": date("2015-09-16"), "description": "yours truly, the boy. God First! Durham,NC", "followers_count": 392, "friends_count": 344, "statues_count": 1689 }, "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-05T10:00:28.000Z"), "id": 684434301895626752, "text": "@TaraLacerte you too", "in_reply_to_status": 684416525692895232, "in_reply_to_user": 3022104033, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3022104033 }}, "user": { "id": 66562172, "name": "LJ Aviles", "screen_name": "LifeCoachLJ", "lang": "en", "location": "New Jersey, USA", "create_at": date("2009-08-17"), "description": "FREE - How To Get More Followers➡http://LifeCoachLJ.com/FreeTwitter⬅ Entrepreneur & Wrestler help ppl Grow Brand & Business w/ Social Media Marketing", "followers_count": 4039, "friends_count": 4739, "statues_count": 30680 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bridgeton, NJ", "id": "78fa23c1159f23d3", "name": "Bridgeton", "place_type": "city", "bounding_box": rectangle("-75.25581,39.403136 -75.183652,39.490858") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34011, "countyName": "Cumberland", "cityID": 3407600, "cityName": "Bridgeton" } }
+{ "create_at": datetime("2016-01-05T10:00:28.000Z"), "id": 684434302172401664, "text": "@bravebananamilk @viralvaccum @kawaiitildeath @Elena_Rudolph is there a SOAR meeting today??", "in_reply_to_status": -1, "in_reply_to_user": 3100153789, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3100153789, 2575106143, 860291690, 1239752335 }}, "user": { "id": 2969765028, "name": "Martha Martinez", "screen_name": "whosshotbiggie", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2015-01-09"), "description": "I'm with you till the end of the line, pal.", "followers_count": 65, "friends_count": 87, "statues_count": 1030 }, "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-05T10:00:28.000Z"), "id": 684434302390566912, "text": "@teenybaby619 you look cute af in that video. Happy bday btw.", "in_reply_to_status": 684432665160716289, "in_reply_to_user": 459889421, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 459889421 }}, "user": { "id": 387364838, "name": "Jacob Jewell", "screen_name": "wht_boi_fresh", "lang": "en", "location": "The Burg, OR", "create_at": date("2011-10-08"), "description": "R.I.P. Debbie, Wes, Zack, Deven, Dustin, Justin, Jake, and Carlos. Instagram: jacob_jewell541 \nSnapChat: whtboi_fresh", "followers_count": 391, "friends_count": 192, "statues_count": 8232 }, "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-01-05T10:00:28.000Z"), "id": 684434302449414144, "text": "The more you know. @ THE SALSA TRUCK https://t.co/eBZiSpgwlf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6545776,41.88354297"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 457561321, "name": "THE SALSA TRUCK", "screen_name": "THESALSATRUCK", "lang": "en", "location": "West Loop, Chicago, USA", "create_at": date("2012-01-07"), "description": "Dan Salls & THE SALSA TRUCK & THE GARAGE- Chicago's 1st cook on board food truck & lunch counter/commissary! Serving inspired Mexican street food 116 N Aberdeen", "followers_count": 4108, "friends_count": 1216, "statues_count": 3227 }, "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-05T10:00:28.000Z"), "id": 684434302675767296, "text": "#Alameda, CA #ProjectMgmt #Job: Director Solutions & Systems Integration - IoT Professional Services at Wind River https://t.co/RLVHde7DS2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.2824021,37.7712165"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Alameda", "ProjectMgmt", "Job" }}, "user": { "id": 23902391, "name": "TMJ-SFO IT PM Jobs", "screen_name": "tmj_sfo_itpm1", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-03-11"), "description": "Follow this account for geo-targeted IT-PM/BA job tweets in San Francisco, CA. Need help? Tweet us at @CareerArc!", "followers_count": 344, "friends_count": 229, "statues_count": 67 }, "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-01-05T10:00:28.000Z"), "id": 684434302730395648, "text": "@nparoska of course nick :)", "in_reply_to_status": 684434178440577024, "in_reply_to_user": 392327992, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 392327992 }}, "user": { "id": 246518695, "name": "†ay Ryann☀️", "screen_name": "shethinksithink", "lang": "en", "location": "Ohio, USA", "create_at": date("2011-02-02"), "description": "|focus on improving yourself for greatness|I'm still gonna shine, with or with out you|positive vibes only|", "followers_count": 2648, "friends_count": 1442, "statues_count": 92398 }, "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-01-05T10:00:28.000Z"), "id": 684434302893854721, "text": "@reagannT it really does ��", "in_reply_to_status": 684434233750732801, "in_reply_to_user": 741736752, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 741736752 }}, "user": { "id": 1056357265, "name": "brooke .", "screen_name": "brookeerulff", "lang": "en", "location": "Louisiana, USA", "create_at": date("2013-01-02"), "description": "with pain comes strength .❤️", "followers_count": 921, "friends_count": 329, "statues_count": 32461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Napoleonville, LA", "id": "0121e0a2317aef16", "name": "Napoleonville", "place_type": "city", "bounding_box": rectangle("-91.039381,29.89863 -90.987085,29.959606") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22007, "countyName": "Assumption", "cityID": 2253370, "cityName": "Napoleonville" } }
+{ "create_at": datetime("2016-01-05T10:00:28.000Z"), "id": 684434303158104064, "text": "S", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1270339968, "name": "12-3", "screen_name": "Justinchild_", "lang": "en", "location": "null", "create_at": date("2013-03-15"), "description": "null", "followers_count": 521, "friends_count": 352, "statues_count": 30540 }, "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-05T10:00:28.000Z"), "id": 684434303166640128, "text": "It has been 7 years since I first recorded an EP with Josh Hiers Band (with a few single… https://t.co/NpaxSMGgKb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.9574966,28.04356"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19752641, "name": "Aaron Gabriel Ross", "screen_name": "Aaron_Ross", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "Nerd. a Lover and a Fighter. Wanna be Theologian. Christ Follower. PhD Student. Instructor of Theology at Southeastern University.", "followers_count": 413, "friends_count": 229, "statues_count": 7212 }, "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-01-05T10:00:28.000Z"), "id": 684434303179096065, "text": "Good Morning #RaiderNation can you believe a vagina worth 58 Thousand for 18 years, child support has gotten out of hand", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RaiderNation" }}, "user": { "id": 21964913, "name": "D.WOODSIDE", "screen_name": "DeeDub67", "lang": "en", "location": "ÜT: 33.956501,-118.339637", "create_at": date("2009-02-25"), "description": "RAIDERS AND LAKERS FAN FOR LIFE ALWAYS IN #BEASTMODE DIEHARD RAIDER FAN SINCE 1969 DIEHARD LAKERS FAN SINCE 1969 SOCIAL MEDIA ENTHUSIASTS", "followers_count": 4704, "friends_count": 5046, "statues_count": 34378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:28.000Z"), "id": 684434303338467328, "text": "@hebrew102 \n��", "in_reply_to_status": 684429756205547520, "in_reply_to_user": 3968455155, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3968455155 }}, "user": { "id": 951089035, "name": "·", "screen_name": "kitarjo", "lang": "en", "location": "null", "create_at": date("2012-11-15"), "description": "Ahavah & Chok´ma", "followers_count": 264, "friends_count": 197, "statues_count": 18631 }, "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-05T10:00:28.000Z"), "id": 684434303573471232, "text": "@joeyNedbal @LissTav guys I can't attend today, I have a swim meet to go too", "in_reply_to_status": 684433912706281473, "in_reply_to_user": 323963743, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 323963743, 348863787 }}, "user": { "id": 1035497994, "name": "Jenn", "screen_name": "JennDaryl_", "lang": "en", "location": "Scotchtown, NY", "create_at": date("2012-12-25"), "description": "Goonies never say die || Sc: JMarie_1216", "followers_count": 543, "friends_count": 770, "statues_count": 7621 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scotchtown, NY", "id": "d6e3868f1e64c0f7", "name": "Scotchtown", "place_type": "city", "bounding_box": rectangle("-74.405271,41.4573 -74.344193,41.518179") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36071, "countyName": "Orange", "cityID": 3665882, "cityName": "Scotchtown" } }
+{ "create_at": datetime("2016-01-05T10:00:28.000Z"), "id": 684434303682351104, "text": "We're #hiring! Read about our latest #job opening here: Accounts Receivable Clerk - https://t.co/OI3Fel9LmI #Harvey, LA #Accounting", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.0772944,29.9035387"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Harvey", "Accounting" }}, "user": { "id": 318698053, "name": "Louisiana Acct Jobs", "screen_name": "tmj_LA_ACCT", "lang": "en", "location": "Louisiana Non-Metro", "create_at": date("2011-06-16"), "description": "Follow this account for geo-targeted Accounting job tweets in Louisiana Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 301, "friends_count": 285, "statues_count": 35 }, "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-05T10:00:28.000Z"), "id": 684434303690739712, "text": "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": 379984675, "name": "one/six", "screen_name": "Marky_Mark15", "lang": "en", "location": "Dallas, TX", "create_at": date("2011-09-25"), "description": "I rather be hated for who I am, then loved for who I am not. Dallas,TX. Dazed and Confused. #CIROCBOYS #SMOKESQUAD", "followers_count": 426, "friends_count": 425, "statues_count": 25733 }, "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-05T10:00:28.000Z"), "id": 684434304240205824, "text": "Can you recommend anyone for this #job? Service Technician/Diesel Mechanic- Vermeer Midwest - https://t.co/vi1RSx0EiW #Marne, Michigan", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.8278002,43.0355531"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Marne" }}, "user": { "id": 3321760873, "name": "Vermeer Jobs", "screen_name": "vermeerjobs", "lang": "en", "location": "Iowa & Across the Globe", "create_at": date("2015-08-20"), "description": "More than 3,000 team members designing, #manufacturing & supporting #industrial & #agricultural equipment to make a worldwide impact. Dig in! #job #jobs #hiring", "followers_count": 61, "friends_count": 74, "statues_count": 77 }, "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": 26139, "countyName": "Ottawa" } }
+{ "create_at": datetime("2016-01-05T10:00:29.000Z"), "id": 684434304689020928, "text": "#RollTide", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "RollTide" }}, "user": { "id": 844150285, "name": "Hales †", "screen_name": "Hay_dryden8", "lang": "en", "location": "Texas dammit ", "create_at": date("2012-09-24"), "description": "† HEi | Joshua 1:9 | Roll Tide Roll❤️", "followers_count": 247, "friends_count": 237, "statues_count": 9960 }, "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-05T10:00:29.000Z"), "id": 684434304911278080, "text": "@Khaliyaaa go learn something ������", "in_reply_to_status": 684434170517434368, "in_reply_to_user": 29892086, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 29892086 }}, "user": { "id": 143684146, "name": "Izzy", "screen_name": "IsaiahHoward_", "lang": "en", "location": "Mo City, Tx ✈️ Prairie View,Tx", "create_at": date("2010-05-13"), "description": "#PVAMU19", "followers_count": 2550, "friends_count": 2180, "statues_count": 38001 }, "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-05T10:00:29.000Z"), "id": 684434305116803072, "text": "happy birthday gorgeous, hope you have a good one!❤️�� @anisha__reddy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2996557441 }}, "user": { "id": 2232406256, "name": "richi mamí", "screen_name": "RichiKonda", "lang": "en", "location": "FTX", "create_at": date("2013-12-05"), "description": "✭lonestar✭", "followers_count": 636, "friends_count": 485, "statues_count": 4681 }, "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-01-05T10:00:29.000Z"), "id": 684434305418805248, "text": "Real Friends always hits the spot", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 313053656, "name": "Felix", "screen_name": "felixrendoniii", "lang": "en", "location": "Mt. Olympus", "create_at": date("2011-06-07"), "description": "Run Shit or Die.", "followers_count": 322, "friends_count": 278, "statues_count": 10027 }, "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-05T10:00:29.000Z"), "id": 684434305842462720, "text": "Always grab the booty https://t.co/fhCtKKUlLk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46725036, "name": "Kenzie", "screen_name": "kenziesundell", "lang": "en", "location": "Purdue University ", "create_at": date("2009-06-12"), "description": "Boilermaker '17 | IT | Shoemaker | DSG", "followers_count": 359, "friends_count": 295, "statues_count": 6863 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Battle Ground, IN", "id": "019ccbf211844597", "name": "Battle Ground", "place_type": "city", "bounding_box": rectangle("-86.892417,40.495261 -86.829017,40.52206") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18157, "countyName": "Tippecanoe", "cityID": 1803718, "cityName": "Battle Ground" } }
+{ "create_at": datetime("2016-01-05T10:00:29.000Z"), "id": 684434306329006080, "text": "We're #hiring! Click to apply: Banquet Sales Manager - https://t.co/YuWhba3SuW #Hospitality #Durham, NC #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.9450648,35.9054031"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Hospitality", "Durham", "Job", "Jobs", "CareerArc" }}, "user": { "id": 22452620, "name": "TMJ-RDU HRTA Jobs", "screen_name": "tmj_rdu_hrta", "lang": "en", "location": "RDU, NC", "create_at": date("2009-03-01"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Raleigh/Durham, NC. Need help? Tweet us at @CareerArc!", "followers_count": 416, "friends_count": 292, "statues_count": 432 }, "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-05T10:00:29.000Z"), "id": 684434306387738624, "text": "Headline reads as if that's a long time... \n\nI swear journalists today are just feeble https://t.co/NGglkNBEgN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1517261017, "name": "1eyewitness", "screen_name": "1eyewitness", "lang": "en", "location": "Beverly Hills, CA", "create_at": date("2013-06-14"), "description": "Believer. Husband. Father of 2. Corp Attorney. I once was blind, now I see. Liberal for 25 years until reality caught up.Moderate libertarian.Anti-authoritarian", "followers_count": 194, "friends_count": 705, "statues_count": 2850 }, "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-05T10:00:29.000Z"), "id": 684434306480123904, "text": "this interview with @KurtSchlichter on militia stand off was interesting despite fact he keeps getting cut off. https://t.co/blAvicnnsS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18089606 }}, "user": { "id": 326255267, "name": "Andrew Kaczynski", "screen_name": "BuzzFeedAndrew", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2011-06-29"), "description": "Boss at @BuzzFeed Politics' KFILE. I also like cats. Email me at Andrew@BuzzFeed.com. Once challenged to a duel and called an idiot by Rand Paul.", "followers_count": 131596, "friends_count": 2632, "statues_count": 99946 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Intercourse, PA", "id": "0182301b6c7757c0", "name": "Intercourse", "place_type": "city", "bounding_box": rectangle("-76.118058,40.027049 -76.088475,40.039636") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42071, "countyName": "Lancaster", "cityID": 4237016, "cityName": "Intercourse" } }
+{ "create_at": datetime("2016-01-05T10:00:29.000Z"), "id": 684434306853421056, "text": "I don't wanna make a long post about it or dwell. it always still hurts but it's okay, you know? I hope she's happy wherever 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": 261983348, "name": "Luis Edward", "screen_name": "eddykinz", "lang": "en-gb", "location": "Homestead, Florida", "create_at": date("2011-03-06"), "description": "bruh", "followers_count": 122, "friends_count": 178, "statues_count": 4713 }, "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-01-05T10:00:29.000Z"), "id": 684434306924609537, "text": "You never know how STRONG you are! #Quotes #Inspiration https://t.co/KVkYdY5zkQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Quotes", "Inspiration" }}, "user": { "id": 1123950516, "name": "Judy Goldberg", "screen_name": "goldberg_judy", "lang": "en", "location": "San Diego, California, USA", "create_at": date("2013-01-26"), "description": "#Realtor at Windermere Real Estate of SoCal. Your #SanDiego North County Coastal #RealEstate Specialist. Contact me today at 760-274-5910", "followers_count": 2020, "friends_count": 1597, "statues_count": 804 }, "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-01-05T10:00:29.000Z"), "id": 684434306979119104, "text": "Want to work at The Vitamin Shoppe? We're #hiring in #Monroe, NC! Click for details: https://t.co/1U8s2Tcjnv #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.56376,35.00912"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Monroe", "Job", "Jobs" }}, "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": 488, "friends_count": 96, "statues_count": 3128 }, "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-01-05T10:00:29.000Z"), "id": 684434307130081280, "text": "Cobia Crudo, ��, basil, jalapeño. Oyster Stew, collards, meyer ��. Beef Tartare, parsnip, foie gras. #eatlocal #nola https://t.co/zcOBpSo5gi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "eatlocal", "nola" }}, "user": { "id": 142389212, "name": "Andrew Marin", "screen_name": "ndrewmarin", "lang": "xx-lc", "location": "New Orleans", "create_at": date("2010-05-10"), "description": "retired chef de partie and bartender | gonzaga | tulane | GQ | insider | snapchat: andrewmarin | NY→Manila→DC→Firenze", "followers_count": 2214, "friends_count": 20, "statues_count": 23596 }, "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-05T10:00:29.000Z"), "id": 684434307327356928, "text": "Ima get this tattoo by myself. Dont want no one to even go with me anymore", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4295271376, "name": "‼️", "screen_name": "Fucknoobabyy", "lang": "en", "location": "West Bubble Fuck", "create_at": date("2015-11-27"), "description": "||R.I.P. Teddy|| #GxldenGods", "followers_count": 593, "friends_count": 502, "statues_count": 11415 }, "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-01-05T10:00:29.000Z"), "id": 684434307545313280, "text": "All of my money goes to Amazon", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 333595253, "name": "Haleigh Camerlin", "screen_name": "hcamerlin", "lang": "en", "location": "Wichita Falls, Texas", "create_at": date("2011-07-11"), "description": "21•TRH•Nursing•MSU", "followers_count": 677, "friends_count": 227, "statues_count": 33595 }, "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-05T10:00:29.000Z"), "id": 684434307910365184, "text": "my house is so boring.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1654072015, "name": "hulia☀️", "screen_name": "JuliaHoplight1", "lang": "en", "location": "Ashtabula, OH", "create_at": date("2013-08-07"), "description": "lakeside highschool | RaeShaun♥️", "followers_count": 509, "friends_count": 129, "statues_count": 8106 }, "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-01-05T10:00:29.000Z"), "id": 684434308245942272, "text": "@JAnnexo you gunna tutor me when I'm hating life cuz I suck at math", "in_reply_to_status": 684432734262050816, "in_reply_to_user": 44504135, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 44504135 }}, "user": { "id": 392327992, "name": "Nick Paroska", "screen_name": "nparoska", "lang": "en", "location": "null", "create_at": date("2011-10-16"), "description": "Jiggy", "followers_count": 589, "friends_count": 309, "statues_count": 53457 }, "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-01-05T10:00:30.000Z"), "id": 684434308602281984, "text": "@Jerry_AM1 funny I almost did this the other day, except I put it on a different pump and ended up driving around to the right one before", "in_reply_to_status": 684433847321137153, "in_reply_to_user": 43998222, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 43998222 }}, "user": { "id": 324786848, "name": "Coty (8-8)", "screen_name": "Coty_Johnson052", "lang": "en", "location": "Louisiana, USA", "create_at": date("2011-06-26"), "description": "dont quote me boy cause I aint said shit.. ΚΣ, Falcons, Longhorns, Celtics, C4L", "followers_count": 1067, "friends_count": 911, "statues_count": 69580 }, "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-05T10:00:30.000Z"), "id": 684434308640055297, "text": "I haven't even left the airport and I already wanna stay here forever", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 366433746, "name": "YUNG PIFFY", "screen_name": "Austyn734", "lang": "en", "location": "null", "create_at": date("2011-09-01"), "description": "sip breast milk outta lean cup", "followers_count": 436, "friends_count": 191, "statues_count": 20718 }, "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-05T10:00:30.000Z"), "id": 684434308736483329, "text": "@meganxwegan I want to go home.", "in_reply_to_status": 684434025365127168, "in_reply_to_user": 945092454, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 945092454 }}, "user": { "id": 2922686268, "name": "Armando", "screen_name": "armxndosanchez", "lang": "en", "location": "Sugar Land, TX", "create_at": date("2014-12-08"), "description": "w/ Jose ❤️", "followers_count": 715, "friends_count": 275, "statues_count": 10909 }, "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-05T10:00:30.000Z"), "id": 684434308770082816, "text": "@jaketapper People with psychiatric disabilities are far more likely to be victims than perpetrators of violent crime (Appleby, et al., 20\"", "in_reply_to_status": -1, "in_reply_to_user": 14529929, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14529929 }}, "user": { "id": 1961983356, "name": "Dustin DeMoss", "screen_name": "DustinDeMoss", "lang": "en", "location": "Eastern Oklahoma", "create_at": date("2013-10-14"), "description": "Veteran, writer and mental health warrior being portrayed in the film @LightWounds. #Boomer and #ThunderUp", "followers_count": 1339, "friends_count": 1676, "statues_count": 4696 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sallisaw, OK", "id": "5e628a31691fe0e6", "name": "Sallisaw", "place_type": "city", "bounding_box": rectangle("-94.831836,35.444969 -94.756267,35.491265") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40135, "countyName": "Sequoyah", "cityID": 4065000, "cityName": "Sallisaw" } }
+{ "create_at": datetime("2016-01-05T10:00:30.000Z"), "id": 684434309185404933, "text": "I just come in and mmob", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26920219, "name": "Sharnicka", "screen_name": "PrettiDymndz", "lang": "en", "location": "Louisville, KY", "create_at": date("2009-03-26"), "description": "Taking on many gifts while serving God #WHForever #TeamWhitney #TeamAaliyah #TFB |CO-CEO #TPMU & #YBB | #TMW #TLA #TBN #TLMZ #TMM Sargent #TeamSoldier #TFK #TTO", "followers_count": 22821, "friends_count": 24932, "statues_count": 116954 }, "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-01-05T10:00:30.000Z"), "id": 684434309416001536, "text": "Want to work at Zales? We're #hiring in #Rockaway, NJ! Click for details: https://t.co/du7CdM8wDd #BusinessMgmt #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.5143232,40.9012101"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Rockaway", "BusinessMgmt", "Job", "Jobs", "CareerArc" }}, "user": { "id": 24311203, "name": "TMJ-NJN Mgmt. Jobs", "screen_name": "tmj_njn_mgmt", "lang": "en", "location": "Northern NJ", "create_at": date("2009-03-13"), "description": "Follow this account for geo-targeted Business/Mgmt. job tweets in Northern NJ. Need help? Tweet us at @CareerArc!", "followers_count": 386, "friends_count": 256, "statues_count": 248 }, "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-01-05T10:00:30.000Z"), "id": 684434309499850753, "text": "@bhurt42 In all honesty, I don’t have time to do a “socratic dialogue” with you.", "in_reply_to_status": 684434011201089537, "in_reply_to_user": 28632027, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28632027 }}, "user": { "id": 15070140, "name": "w ⊩ jon sterling", "screen_name": "jonsterling", "lang": "en", "location": "Portland, OR", "create_at": date("2008-06-09"), "description": "Type theorist, Brouwerian intuitionist, reluctant programmer, amateur philologist. Cohost of @typetheorypcast.", "followers_count": 2016, "friends_count": 265, "statues_count": 49915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sellwood Moreland, Portland", "id": "16261d425072636b", "name": "Sellwood Moreland", "place_type": "neighborhood", "bounding_box": rectangle("-122.668238,45.457787 -122.636531,45.501491") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-05T10:00:30.000Z"), "id": 684434309629906944, "text": "Can't wait to see youuuuu�� love love love you! https://t.co/sIyNoU8YVB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 98229521, "name": "mads", "screen_name": "MaddiMays", "lang": "en", "location": "sandy eggo CA", "create_at": date("2009-12-20"), "description": "stripper booty and a rack like wow", "followers_count": 400, "friends_count": 243, "statues_count": 33259 }, "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-05T10:00:30.000Z"), "id": 684434309755748352, "text": "@EmilyMarte_ @JakeWeb_ https://t.co/Ntjeigku8p", "in_reply_to_status": 684403981829865476, "in_reply_to_user": 286268103, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 286268103, 339529040 }}, "user": { "id": 3397754292, "name": "curtis 1 williams", "screen_name": "wcurtis38", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-08-30"), "description": "null", "followers_count": 213, "friends_count": 1010, "statues_count": 543 }, "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-05T10:00:30.000Z"), "id": 684434309990584320, "text": "We're #hiring! Click to apply: Family Practice Physician, Valentine, NE - https://t.co/WM4kTf5RC5 #FamilyPractice #Valentine, NE #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-100.5509669,42.8727833"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "FamilyPractice", "Valentine", "Job", "Jobs" }}, "user": { "id": 72460509, "name": "NE Physician Jobs", "screen_name": "NE_physician", "lang": "en", "location": "Nebraska", "create_at": date("2009-09-07"), "description": "Follow this account for geo-targeted Physician job tweets in Nebraska Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 133, "friends_count": 129, "statues_count": 66 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valentine, NE", "id": "6af7bbbaa9a54b29", "name": "Valentine", "place_type": "city", "bounding_box": rectangle("-100.581886,42.860775 -100.534402,42.88934") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31031, "countyName": "Cherry", "cityID": 3149950, "cityName": "Valentine" } }
+{ "create_at": datetime("2016-01-05T10:00:30.000Z"), "id": 684434310259032065, "text": "@ritchiewitha @melik_richard not surprised at all", "in_reply_to_status": 684434158391693312, "in_reply_to_user": 512748224, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 512748224, 83483815 }}, "user": { "id": 384129756, "name": "no dayton, no.", "screen_name": "deeton_", "lang": "en", "location": "Tempe, AZ", "create_at": date("2011-10-02"), "description": "pancake connoisseur | the lighting is unbearable", "followers_count": 151, "friends_count": 48, "statues_count": 4327 }, "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-05T10:00:30.000Z"), "id": 684434310317928448, "text": "rt like bitch where �� https://t.co/OTD7WIvcvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2257219201, "name": "mami.", "screen_name": "mulattox3", "lang": "en", "location": "null", "create_at": date("2013-12-21"), "description": "null", "followers_count": 126, "friends_count": 128, "statues_count": 5197 }, "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-05T10:00:30.000Z"), "id": 684434310368202752, "text": "I get way too excited about food 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": 325059317, "name": "Daezha", "screen_name": "daezha_alyria", "lang": "en", "location": "Georgia, USA", "create_at": date("2011-06-27"), "description": "lover of all things @taylorswift13. Psalms 46:5. Got more issues than Vogue. Auburn University.", "followers_count": 1171, "friends_count": 863, "statues_count": 23867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thomasville, GA", "id": "69cec9bd4dd162c7", "name": "Thomasville", "place_type": "city", "bounding_box": rectangle("-84.016198,30.806439 -83.902855,30.91895") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13275, "countyName": "Thomas", "cityID": 1376224, "cityName": "Thomasville" } }
+{ "create_at": datetime("2016-01-05T10:00:30.000Z"), "id": 684434310527598593, "text": "Graphic Playmats Available Now! https://t.co/1J5y3ssT3C #NewCardGame #CoolNewStuff", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.6477771,40.7422154"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NewCardGame", "CoolNewStuff" }}, "user": { "id": 2675192989, "name": "BattleforSularia", "screen_name": "SulariaBCG", "lang": "en", "location": "Lincoln, Ne", "create_at": date("2014-07-23"), "description": "null", "followers_count": 150, "friends_count": 241, "statues_count": 407 }, "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-05T10:00:30.000Z"), "id": 684434310535856128, "text": "#Healthcare #Job in #Folsom, CA: Pharmacy Technician at Raley's https://t.co/ljcsAnXl4v #pharmacy #Veterans #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.1760583,38.6779591"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "Job", "Folsom", "pharmacy", "Veterans", "Jobs", "Hiring" }}, "user": { "id": 2786536903, "name": "Raley's Jobs", "screen_name": "raleysjobs", "lang": "en", "location": "null", "create_at": date("2014-09-02"), "description": "Feed your passion at Raley’s; join our team! Job opportunities you can grow your career with. Get news, recipes and more @raleys.", "followers_count": 74, "friends_count": 1, "statues_count": 1541 }, "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.714333") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 624638, "cityName": "Folsom" } }
+{ "create_at": datetime("2016-01-05T10:00:30.000Z"), "id": 684434310980440064, "text": "@kuch_pani @t11jba @fireflies640 I from Burbank california USA #WeStandByParth @LaghateParthOfc @Friendly_bini https://t.co/qfRpihHkyd", "in_reply_to_status": 684406293314928640, "in_reply_to_user": 3173016544, "favorite_count": 0, "coordinate": point("-118.3433184,34.1843812"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WeStandByParth" }}, "user_mentions": {{ 3173016544, 2625043349, 3090360210, 2916947141, 1175962135 }}, "user": { "id": 2379835759, "name": "Bubly#WeStandByParth", "screen_name": "bubly_refath", "lang": "en", "location": "bur bank @ CA_USA", "create_at": date("2014-03-08"), "description": "proud of Parth MANAN PaNi Hamesha", "followers_count": 33, "friends_count": 144, "statues_count": 698 }, "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-05T10:00:30.000Z"), "id": 684434310988865536, "text": "Me olvide de pensar por solo sentir", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 257980140, "name": "Edna Zavala➰", "screen_name": "edcanttu", "lang": "en", "location": "San Antonio, Texas.", "create_at": date("2011-02-26"), "description": "Carpe diem.", "followers_count": 288, "friends_count": 376, "statues_count": 8880 }, "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-05T10:00:30.000Z"), "id": 684434311110475777, "text": "Gooti", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2175801300, "name": "Gooti Records", "screen_name": "GootiStudios", "lang": "en", "location": "Mexico City ", "create_at": date("2013-11-05"), "description": "Gooti Records International", "followers_count": 3319, "friends_count": 50, "statues_count": 2568 }, "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-05T10:00:30.000Z"), "id": 684434311370649600, "text": "@ReformedBroker In the future, even urban folk in U.S. will have \"technicals\" -- pick-up trucks with military-style machine guns on the back", "in_reply_to_status": 684424889814626304, "in_reply_to_user": 22522178, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 22522178 }}, "user": { "id": 18532923, "name": "Auld Lang Syne Katt", "screen_name": "BuckyKatt", "lang": "en", "location": "NYC and NJ", "create_at": date("2009-01-01"), "description": "Writer, blogger, runner, music lover, med-sci geek, volunteer. Blogging about music at http://t.co/DrdeAhQ32A & health at http://t.co/DHNOU7l6Ht", "followers_count": 2014, "friends_count": 2151, "statues_count": 32289 }, "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-05T10:00:30.000Z"), "id": 684434311513260032, "text": "The one with the kids in CT was beast to me �� https://t.co/coFx1PnMmQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2318466868, "name": "THE SINFUL MESSIAH", "screen_name": "youngpopi_", "lang": "en", "location": "null", "create_at": date("2014-02-01"), "description": "no snakes in my grass cause I mowed it nigga #LIG #GIG", "followers_count": 255, "friends_count": 288, "statues_count": 12321 }, "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-05T10:00:30.000Z"), "id": 684434311538425856, "text": "Ok so I really need to eat... Like frfr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 65758778, "name": "Niina Coleman", "screen_name": "NyinaBonita", "lang": "en", "location": "Chicago, IL.", "create_at": date("2009-08-14"), "description": "24.FlyGirl. Belizean, and Lakota Sioux #Sagittarius IG:@NyinaBonita ✈Traveling Journalist✈. Welcome to My World. Business Inquiries: NyinaBonita@yah", "followers_count": 661, "friends_count": 1266, "statues_count": 13833 }, "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-05T10:00:30.000Z"), "id": 684434311609630720, "text": "We're #hiring! Click to apply: Drug Safety Specialist - https://t.co/V7Y5dlceAv #Transportation #FosterCity, CA #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.2710788,37.5585465"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Transportation", "FosterCity", "Job", "Jobs" }}, "user": { "id": 23471565, "name": "TMJ-SFO Transp. Jobs", "screen_name": "tmj_sfo_transp", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Transportation job tweets in San Francisco, CA. Need help? Tweet us at @CareerArc!", "followers_count": 429, "friends_count": 323, "statues_count": 136 }, "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-01-05T10:00:30.000Z"), "id": 684434311752343552, "text": "Shaun's Top 15 Songs of 2015 https://t.co/lnsvorJU9s", "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": 271167530, "name": "The Guys From", "screen_name": "guysfromblank", "lang": "en", "location": "Calgary, Alberta", "create_at": date("2011-03-23"), "description": "Home of sarcastic insight into film, tv, sports, news and gaming. Plus news on The Guys From {BLANK} comedy", "followers_count": 249, "friends_count": 19, "statues_count": 505 }, "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-05T10:00:30.000Z"), "id": 684434312041635840, "text": "Taco Cabana sounds good now that I said it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2990720747, "name": "D.D.♚", "screen_name": "DGH5_23", "lang": "en", "location": "my house ", "create_at": date("2015-01-21"), "description": "forever yours @JosephDickson21♥️", "followers_count": 266, "friends_count": 252, "statues_count": 7964 }, "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-05T10:00:30.000Z"), "id": 684434312226287616, "text": "Best of 2015: Highlights from a Traveling Nerd https://t.co/dDPxL3m3iB", "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": 1611275376, "name": "TheUnapologeticNerd", "screen_name": "KristinHuggins", "lang": "en", "location": "null", "create_at": date("2013-07-21"), "description": "Vocal Professor | Opera Singer | #BookBlogger | Cross-stitching Fiend | Lover of all things Nerdy & Quirky | #Bookreviews @ http://www.theunapologeticnerd.com", "followers_count": 169, "friends_count": 403, "statues_count": 277 }, "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-05T10:00:30.000Z"), "id": 684434312440246272, "text": "My mind is on a DUNKIN ICE COFFEE (at @DunkinDonuts in Turner, ME) https://t.co/8gMWxvwH70", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.242756,44.19603316"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 8771022 }}, "user": { "id": 84772484, "name": "BOUCH", "screen_name": "NIGHTRANGER1072", "lang": "en", "location": "Mexico, ME", "create_at": date("2009-10-23"), "description": "I'm a father of 2 wonderful boys and a stepdaughter. Have a wonderful wife. I am a SHR fan (Stewart Hass Racing). Go SMOKE, Go Harvick, Go Outlaw, Go Danica.", "followers_count": 66, "friends_count": 230, "statues_count": 3976 }, "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", "cityID": 2302060, "cityName": "Auburn" } }
+{ "create_at": datetime("2016-01-05T10:00:30.000Z"), "id": 684434312574283776, "text": "I noticed if you activate sound on vine vids and then hit new tweets the time line will update but you will still hear the vid 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": 1386114692, "name": "Jesse McElroy", "screen_name": "JeMcelroy", "lang": "en", "location": "las Vegas,Nv.", "create_at": date("2013-04-27"), "description": "Interest: sports, fitness, , tech, green movement, automotive tech, sales blinds, shutters, solar screens, roller shades, cellular or honeycomb shades, custom", "followers_count": 5230, "friends_count": 5482, "statues_count": 6183 }, "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-05T10:00:31.000Z"), "id": 684434312826077184, "text": "drop the baggage https://t.co/IvKIXxGHE4", "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, "user": { "id": 33767794, "name": "Elle.Davis", "screen_name": "ElleDavisSpeaks", "lang": "en", "location": "Chicago", "create_at": date("2009-04-20"), "description": "Servant | Mompreneur | Speaker | Coach | Consultant | Widow | Youth Development Specialist | Author | Personal & Professional Development", "followers_count": 428, "friends_count": 1113, "statues_count": 689 }, "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-05T10:00:31.000Z"), "id": 684434313023062016, "text": "Es que definitivamente, solo hablan pa cagala.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 74389169, "name": "Charles", "screen_name": "Charlesnoser", "lang": "en", "location": "Houston, TX", "create_at": date("2009-09-15"), "description": "20. La desinformación es el arma más letal. Irse de su país y dejar su cultura sabe a mierda. Snapchat: charlesosorio", "followers_count": 588, "friends_count": 501, "statues_count": 41948 }, "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-01-05T10:00:31.000Z"), "id": 684434313379622912, "text": "Paul and Kevins scenes were too good too. The Bookie is a great actor. #yr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "yr" }}, "user": { "id": 366636001, "name": "Luke Pegues", "screen_name": "Lukeduke75", "lang": "en", "location": "null", "create_at": date("2011-09-02"), "description": "TV fanatic, Y&R junkie, JR Ewing Lover, Cookie Lyons #1 Supporter", "followers_count": 375, "friends_count": 355, "statues_count": 14979 }, "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-05T10:00:31.000Z"), "id": 684434313568325633, "text": "@valeballerina neh", "in_reply_to_status": 684432861252825088, "in_reply_to_user": 2996537383, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2996537383 }}, "user": { "id": 1170106946, "name": "CHUMP!", "screen_name": "TheChump_", "lang": "en", "location": "Dirty. Filthy. Paradise.", "create_at": date("2013-02-11"), "description": "Aaron Moreno: HTX • Proud owner and founder of CHUMP! Co. • Concert photographer (link below) • Mac and Cheese Enthusiast.", "followers_count": 423, "friends_count": 476, "statues_count": 45277 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:31.000Z"), "id": 684434313719365633, "text": "@LuisGyG @ingridmotta qué llevada mamá jeje ��", "in_reply_to_status": 684428679976357888, "in_reply_to_user": 824881, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 824881, 29774381 }}, "user": { "id": 14812899, "name": "Edmundo Stierle", "screen_name": "edmundostierle", "lang": "en", "location": "México DF", "create_at": date("2008-05-17"), "description": "América, Real Madrid, Cowboys, F1, y fan from hell de todos los deportes.", "followers_count": 5289, "friends_count": 4653, "statues_count": 32511 }, "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-05T10:00:31.000Z"), "id": 684434313853550592, "text": "SONIC Drive-In: Carhop/Skating Carhop (Server) (#Cartersville, GA) https://t.co/OxPHZuhJT4 #Hospitality #SONIC #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.7893015,34.1935988"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Cartersville", "Hospitality", "SONIC", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 88113750, "name": "TMJ-GA HRTA Jobs", "screen_name": "tmj_ga_hrta", "lang": "en", "location": "Georgia", "create_at": date("2009-11-06"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Georgia Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 400, "friends_count": 288, "statues_count": 1369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cartersville, GA", "id": "652dcd4f43adb5b5", "name": "Cartersville", "place_type": "city", "bounding_box": rectangle("-84.865819,34.120641 -84.755084,34.222881") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13015, "countyName": "Bartow", "cityID": 1313688, "cityName": "Cartersville" } }
+{ "create_at": datetime("2016-01-05T10:00:31.000Z"), "id": 684434314067578880, "text": "It's times like these I debate dropping out of school and become a tattoo artist who lives with her parents and works out of there basement.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1908933912, "name": "CupKate", "screen_name": "KGargiula", "lang": "en", "location": "Somewhere new ", "create_at": date("2013-09-26"), "description": "You're going to break my heart, but I'm definitely going to let you.", "followers_count": 148, "friends_count": 145, "statues_count": 2216 }, "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-05T10:00:31.000Z"), "id": 684434314294067200, "text": "If anyone is in DC Econ and needs a book hit 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": 41958573, "name": "Liv", "screen_name": "liv_timmerman", "lang": "en", "location": "574 ", "create_at": date("2009-05-22"), "description": "It's a beautiful day to save lives.", "followers_count": 1361, "friends_count": 728, "statues_count": 17238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warsaw, IN", "id": "9a4adc08e83b58df", "name": "Warsaw", "place_type": "city", "bounding_box": rectangle("-85.944986,41.190359 -85.765358,41.304979") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18085, "countyName": "Kosciusko", "cityID": 1880306, "cityName": "Warsaw" } }
+{ "create_at": datetime("2016-01-05T10:00:31.000Z"), "id": 684434314440880128, "text": "kinda figured I would eventually of been left out", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1043636359, "name": "rie", "screen_name": "CaruanaMarie", "lang": "en", "location": "null", "create_at": date("2012-12-28"), "description": "Marys", "followers_count": 440, "friends_count": 581, "statues_count": 2289 }, "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-01-05T10:00:31.000Z"), "id": 684434314637918208, "text": "@AndreaTantaros hating ur potus will make republicans hyperbole as u r none to do!! Conservatives put themselves in a box bc their hateful", "in_reply_to_status": -1, "in_reply_to_user": 36452050, "favorite_count": 0, "coordinate": point("-75.42873144,39.86074219"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 36452050 }}, "user": { "id": 2760391485, "name": "this is not phyllis", "screen_name": "MormilePhyllis", "lang": "en", "location": "null", "create_at": date("2014-09-03"), "description": "#unitedblue#equalityforall#respectyourpotus. SAY HELLO TO A SENIOR#boycottfoxnews HILLARY2016✨. I BLOCK conservatives.", "followers_count": 121, "friends_count": 330, "statues_count": 2915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Village Green, PA", "id": "002170dd611822f0", "name": "Village Green", "place_type": "city", "bounding_box": rectangle("-75.447416,39.851031 -75.39337,39.874069") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42045, "countyName": "Delaware", "cityID": 4280218, "cityName": "Village Green-Green Ridge" } }
+{ "create_at": datetime("2016-01-05T10:00:31.000Z"), "id": 684434314767945728, "text": "Argosy Casino Hotel & Spa: Count Room Cashier - Full Time (#Riverside, MO) https://t.co/sxiE4oEbE4 #Hospitality #cashier #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.6243649,39.162493"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Riverside", "Hospitality", "cashier", "Job", "Jobs", "Hiring" }}, "user": { "id": 1725772980, "name": "Argosy Casino KC", "screen_name": "ArgosyKC_Jobs", "lang": "en", "location": "null", "create_at": date("2013-09-03"), "description": "A Penn National Gaming property, Argosy Casino Hotel & Spa offers gaming excitement, luxury hotel rooms, fine dining and a rejuvenating spa.", "followers_count": 184, "friends_count": 137, "statues_count": 77 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, MO", "id": "f8e9521355ca84c2", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-94.659895,39.154401 -94.601419,39.189295") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29165, "countyName": "Platte", "cityID": 2962156, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-01-05T10:00:31.000Z"), "id": 684434315267018752, "text": "Can you recommend anyone for this #job? https://t.co/YLTBSMP3LK #Houston, TX #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.5511789,29.9660157"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Houston", "Hiring" }}, "user": { "id": 716093454, "name": "MethodistWillowbrook", "screen_name": "HMWBJobs", "lang": "en", "location": "Houston, TX", "create_at": date("2012-07-25"), "description": "Jobs Available at Houston Methodist Willowbrook Hospital.", "followers_count": 107, "friends_count": 8, "statues_count": 252 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:31.000Z"), "id": 684434315476783104, "text": "#PolandIsReadyForRevivalTour 35", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "PolandIsReadyForRevivalTour" }}, "user": { "id": 1691377656, "name": "Justin my Angel", "screen_name": "AngelaDziedzic1", "lang": "pl", "location": "Chicago, IL", "create_at": date("2013-08-22"), "description": "I fell in love with a 15 year old guy who dreams fulfilled now I love 21-year-old man,who helps fulfill the dreams of other!18/11/15 Thank You Justin PotterHead", "followers_count": 325, "friends_count": 808, "statues_count": 2605 }, "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-01-05T10:00:31.000Z"), "id": 684434315883630593, "text": "Today sucks", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 418051510, "name": "RodWayne", "screen_name": "RodWayneSaidIt", "lang": "en", "location": "Arlington Texas", "create_at": date("2011-11-21"), "description": "/iHeart Media/Talk-Radio/Radio-Personality/Motivator/Lifer/Occasional Asshole/...Dedicated to being me! Focused on bettering me! Now lets get it!", "followers_count": 527, "friends_count": 76, "statues_count": 15735 }, "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-05T10:00:31.000Z"), "id": 684434316093341696, "text": "I might gotta jugg @sono_boy_tre bike", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 552765582 }}, "user": { "id": 69970596, "name": "GHOST", "screen_name": "raymonta", "lang": "en", "location": "null", "create_at": date("2009-08-29"), "description": "ODU 19'", "followers_count": 570, "friends_count": 321, "statues_count": 33325 }, "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-05T10:00:31.000Z"), "id": 684434316600844288, "text": "@aachillary Wahahaha!!! ♥", "in_reply_to_status": 684369733123112962, "in_reply_to_user": 1351003356, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 1351003356 }}, "user": { "id": 77179183, "name": "Patricia Castillo", "screen_name": "PaaatriciaAnn", "lang": "en", "location": "Paranaque City", "create_at": date("2009-09-25"), "description": "Corporate Slave | IG: paaannnc | Snapchat: paannnnc", "followers_count": 311, "friends_count": 254, "statues_count": 34835 }, "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-05T10:00:31.000Z"), "id": 684434316705697792, "text": "@abbybrewm *seen", "in_reply_to_status": 684434275005890560, "in_reply_to_user": 471429436, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3146674329 }}, "user": { "id": 471429436, "name": "ASIMWE", "screen_name": "Oben_YourMind", "lang": "en", "location": "Lake Forest, IL", "create_at": date("2012-01-22"), "description": "Tall, Dark, and Sorta OK Looking on a Good Day. I sound like a pterodactyl #AllForSchwallie", "followers_count": 405, "friends_count": 393, "statues_count": 11266 }, "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-05T10:00:31.000Z"), "id": 684434316777156608, "text": "#gamer My email is mediaboxes966@gmail.com\n Android boxes,all channels 99 bux https://t.co/GbCid08lqm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "gamer" }}, "user": { "id": 20930543, "name": "Joe Somerville", "screen_name": "Atlanta_Dining", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-02-15"), "description": "I get butts in seats, guests in your hotel beds, olive eaters for your martinis, make the phone ring & get your biz that extra 20%.", "followers_count": 36169, "friends_count": 36675, "statues_count": 27544 }, "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-01-05T10:00:31.000Z"), "id": 684434316852465664, "text": "This #SkilledTrade #job might be a great fit for you: Plumbing Pro (Full Time) - https://t.co/nV6uWDg855 #Tallahassee, FL #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.2807329,30.4382559"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SkilledTrade", "job", "Tallahassee", "Hiring" }}, "user": { "id": 117932161, "name": "TMJ-FL Skl. Trd. Job", "screen_name": "tmj_FL_skltrd", "lang": "en", "location": "Florida", "create_at": date("2010-02-26"), "description": "Follow this account for geo-targeted Skilled Trade job tweets in Florida Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 368, "friends_count": 279, "statues_count": 98 }, "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-05T10:00:32.000Z"), "id": 684434317020274688, "text": "We're #hiring! Read about our latest #job opening here: Jewelry Consultant - https://t.co/5wmAz9ZSSj #Branson, MO #BusinessMgmt #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.2185144,36.6436731"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Branson", "BusinessMgmt", "CareerArc" }}, "user": { "id": 22927611, "name": "TMJ-MO Mgmt. Jobs", "screen_name": "tmj_mo_mgmt", "lang": "en", "location": "Missouri", "create_at": date("2009-03-05"), "description": "Follow this account for geo-targeted Business/Mgmt. job tweets in Missouri Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 326, "friends_count": 243, "statues_count": 146 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Branson, MO", "id": "00543feb079f1e9e", "name": "Branson", "place_type": "city", "bounding_box": rectangle("-93.331522,36.594421 -93.200981,36.715258") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29213, "countyName": "Taney", "cityID": 2907966, "cityName": "Branson" } }
+{ "create_at": datetime("2016-01-05T10:00:32.000Z"), "id": 684434317120897025, "text": "#Iowa Love it #ImWithHer @HillaryforIA https://t.co/dGy3sDTCQE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Iowa", "ImWithHer" }}, "user_mentions": {{ 3056142064 }}, "user": { "id": 1161879818, "name": "Matt Paul", "screen_name": "MattPaulIA", "lang": "en", "location": "Des Moines, IA", "create_at": date("2013-02-08"), "description": "Views are my own.", "followers_count": 1725, "friends_count": 641, "statues_count": 891 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Osage, IA", "id": "2bb77c8a5545ac3a", "name": "Osage", "place_type": "city", "bounding_box": rectangle("-92.827151,43.267784 -92.791243,43.295103") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19131, "countyName": "Mitchell", "cityID": 1959745, "cityName": "Osage" } }
+{ "create_at": datetime("2016-01-05T10:00:32.000Z"), "id": 684434317133647872, "text": "@MikelSevere @937JohnGaskins @937TheTicket yes it is. Plenty of NE people attended KU but grew up as NU fans. #conflicted", "in_reply_to_status": 684429833963614208, "in_reply_to_user": 38724725, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "conflicted" }}, "user_mentions": {{ 38724725, 274760347, 339275903 }}, "user": { "id": 351133262, "name": "Nick Handley", "screen_name": "NickESPN590", "lang": "en", "location": "Omaha", "create_at": date("2011-08-08"), "description": "Co-Host for The Drive AM 590 ESPN Radio. Microphone loud mouth, reptile hunter and leader of the Anti-Clown movement", "followers_count": 1828, "friends_count": 709, "statues_count": 11403 }, "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-05T10:00:32.000Z"), "id": 684434317301264384, "text": "Want to work in #Kokomo, IN? View our latest opening: https://t.co/sKaQdOOyFY #Healthcare #Job #Jobs #Hiring https://t.co/Sp0M6cNPDr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.1336033,40.486427"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Kokomo", "Healthcare", "Job", "Jobs", "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": 788, "friends_count": 1, "statues_count": 76632 }, "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-01-05T10:00:32.000Z"), "id": 684434317351596033, "text": "LPN, INFUSION, ATLANTA NORTH AREAS - Northside Hospital: (#Atlanta, GA) https://t.co/FgcocZGfmK #Nursing #LPN #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.352759,33.909187"), "retweet_count": 0, "lang": "da", "is_retweet": false, "hashtags": {{ "Atlanta", "Nursing", "LPN", "Job", "Jobs", "Hiring" }}, "user": { "id": 3161440380, "name": "Northside Careers", "screen_name": "NorthsideCareer", "lang": "en", "location": "null", "create_at": date("2015-04-17"), "description": "Dedicated to providing clinical excellence balanced with compassionate care. Learn how you can make a difference and continue growing your career.", "followers_count": 16, "friends_count": 1, "statues_count": 452 }, "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-01-05T10:00:32.000Z"), "id": 684434317540331520, "text": "CVS Health #Healthcare #Job: Pharmacy Technician (#CoralSprings, FL) https://t.co/gKa2JTtZol #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.2706044,26.271192"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "Job", "CoralSprings", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 120668564, "name": "TMJ-FLF Health Jobs", "screen_name": "tmj_FLF_health", "lang": "en", "location": "Ft. Lauderdale, FL", "create_at": date("2010-03-06"), "description": "Follow this account for geo-targeted Healthcare job tweets in Ft. Lauderdale, FL. Need help? Tweet us at @CareerArc!", "followers_count": 360, "friends_count": 287, "statues_count": 216 }, "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-01-05T10:00:32.000Z"), "id": 684434317712441344, "text": "@KingNikolai_ I didn't make the cut this week �� I've been out of town but I'm back & im better.", "in_reply_to_status": 684426896520671233, "in_reply_to_user": 2548001614, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2548001614 }}, "user": { "id": 289492733, "name": "Queen Victoria", "screen_name": "QueeennV_", "lang": "en", "location": "Louisiana", "create_at": date("2011-04-28"), "description": "#LSU ✨", "followers_count": 792, "friends_count": 527, "statues_count": 62569 }, "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-05T10:00:32.000Z"), "id": 684434318005960704, "text": "@MaddyVillacruz @Terrell_McKay damn 0 to 100 real quick ����", "in_reply_to_status": 684434056780488704, "in_reply_to_user": 444391050, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 444391050, 1363066884 }}, "user": { "id": 335177205, "name": "DJ", "screen_name": "dj_palomera", "lang": "en", "location": "null", "create_at": date("2011-07-14"), "description": "Julia Marie Williams is the baddest bitch around", "followers_count": 355, "friends_count": 159, "statues_count": 6935 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho San Diego, CA", "id": "85ffb048a1b30f4f", "name": "Rancho San Diego", "place_type": "city", "bounding_box": rectangle("-116.961891,32.733726 -116.875,32.787924") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 659550, "cityName": "Rancho San Diego" } }
+{ "create_at": datetime("2016-01-05T10:00:32.000Z"), "id": 684434318027059200, "text": "@razeraDTD @sofiaolb razi, se vc quiser mesmo, chama no whatsapp. Acho que é uma ideia plausível pelo tempo que tem ate la", "in_reply_to_status": 684361249208025088, "in_reply_to_user": 40087863, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user_mentions": {{ 40087863, 3546730516 }}, "user": { "id": 83743631, "name": "sambarilove", "screen_name": "danilocarlin_", "lang": "en", "location": "Fort Wayne, IN", "create_at": date("2009-10-19"), "description": "Forgive them, even if they are not sorry.", "followers_count": 404, "friends_count": 256, "statues_count": 43232 }, "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-05T10:00:32.000Z"), "id": 684434318303739904, "text": "@cronicasdelocio @elnortecom ella es la que le dio las llaves de la ciudad a Cristo??", "in_reply_to_status": 684411047755694081, "in_reply_to_user": 48212098, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 48212098, 41925129 }}, "user": { "id": 188954638, "name": "Roberto Correo®", "screen_name": "RCorreo88", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2010-09-09"), "description": "Pieza Poblana Perfecta ( PIPOPE ), viviendo en las vegas. Libra. 1988.", "followers_count": 137, "friends_count": 178, "statues_count": 12542 }, "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-05T10:00:32.000Z"), "id": 684434318974844929, "text": "This #Healthcare #job might be a great fit for you: Qualified Mental Health Associate - Emergency Department - https://t.co/uORvRViKsi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.6392606,45.4462305"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job" }}, "user": { "id": 374610492, "name": "Proviscalling", "screen_name": "Proviscalling", "lang": "en", "location": "AK, CA, MT, OR, WA", "create_at": date("2011-09-16"), "description": "3rd largest non-profit health care system in the U.S. with more than 76,000 employees in AK, CA, MT, OR & WA. Visit http://t.co/8JbJjClRDN to apply!", "followers_count": 353, "friends_count": 101, "statues_count": 13999 }, "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-01-05T10:00:32.000Z"), "id": 684434319142617089, "text": "@Jamey_Halps yep and I'm getting hard core judged for not flossing.", "in_reply_to_status": 684434075273179136, "in_reply_to_user": 2266634124, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2266634124 }}, "user": { "id": 881188831, "name": "Sam ❂", "screen_name": "sammivari", "lang": "en", "location": "Colorado, USA", "create_at": date("2012-10-14"), "description": "XII.XIX.MMXV", "followers_count": 493, "friends_count": 369, "statues_count": 2993 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Parker, CO", "id": "005d9455753e7700", "name": "Parker", "place_type": "city", "bounding_box": rectangle("-104.813236,39.478431 -104.708485,39.568631") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8035, "countyName": "Douglas", "cityID": 857630, "cityName": "Parker" } }
+{ "create_at": datetime("2016-01-05T10:00:32.000Z"), "id": 684434319163658240, "text": "Is Final Fantasy the best rpg series out their?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 151640501, "name": "Surge Gamer", "screen_name": "SergeFoxRoo", "lang": "en", "location": "Tampa, FL", "create_at": date("2010-06-03"), "description": "Nintendo Fan, gamer,shapeshifter, and cartoon junkie", "followers_count": 437, "friends_count": 1348, "statues_count": 22117 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Magdalene, FL", "id": "51264211e147d635", "name": "Lake Magdalene", "place_type": "city", "bounding_box": rectangle("-82.506967,28.054599 -82.452059,28.120751") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1238350, "cityName": "Lake Magdalene" } }
+{ "create_at": datetime("2016-01-05T10:00:32.000Z"), "id": 684434319297806336, "text": "https://t.co/F66tc7aS6o", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3063845,35.5774643"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2445654229, "name": "RE/MAX Crossroads", "screen_name": "lincolntonremax", "lang": "en", "location": "Lincolnton NC", "create_at": date("2014-04-15"), "description": "null", "followers_count": 71, "friends_count": 43, "statues_count": 6133 }, "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.32187,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37035, "countyName": "Catawba" } }
+{ "create_at": datetime("2016-01-05T10:00:32.000Z"), "id": 684434319297916928, "text": "We're #hiring! Read about our latest #job opening here: Retail Cosmetics Sales - Counter Manager Lancome,... - https://t.co/FHa8JEtCSi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.9839499,40.1442721"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "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": 671, "friends_count": 0, "statues_count": 8402 }, "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-05T10:00:32.000Z"), "id": 684434319444717569, "text": "@KasimReed Thanks for flying across the country to show your commitment to making ATL a smarter city. #ATTDevSummit https://t.co/osH4WE7W3f", "in_reply_to_status": -1, "in_reply_to_user": 15256841, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ATTDevSummit" }}, "user_mentions": {{ 15256841 }}, "user": { "id": 927839635, "name": "Joanie Twersky", "screen_name": "JoanieTwersky", "lang": "en", "location": "null", "create_at": date("2012-11-05"), "description": "Techie, foodie & world traveler, connecting people & ideas. Tweet about tech events & the innovation community in ATL. Work@AT&T Foundry but opinions are mine.", "followers_count": 181, "friends_count": 216, "statues_count": 133 }, "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-05T10:00:32.000Z"), "id": 684434319482306560, "text": "@NHLBlackhawks Russian meal", "in_reply_to_status": 683828367209172992, "in_reply_to_user": 14498484, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14498484 }}, "user": { "id": 15608437, "name": "Sherrie Curry", "screen_name": "SherrieWPRS", "lang": "en", "location": "Kansas", "create_at": date("2008-07-25"), "description": "Paranormal Investigator w/WPRS a Proud TAPS Family member. I'm a Realtor w/Better Homes and Gardens Real Estate Alliance & I'm a Wichita Thunder Hockey fan.", "followers_count": 337, "friends_count": 201, "statues_count": 6451 }, "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-05T10:00:32.000Z"), "id": 684434319620702208, "text": "@BabyCat73 @RickiNycgal @Maher48Hours @48Hrs call your sponsor Hun", "in_reply_to_status": 684178377293590528, "in_reply_to_user": 842196487, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 842196487, 1415736788, 2296205832, 35721323 }}, "user": { "id": 2310813366, "name": "Cindi Benson", "screen_name": "cindogg88", "lang": "en", "location": "Los Angeles ", "create_at": date("2014-01-25"), "description": "Lover of literature, music,movies, theater, true crime buff, journalism, travel, hiking, good food and wine. I love football. I have faith in a loving God.", "followers_count": 1231, "friends_count": 1189, "statues_count": 12725 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:32.000Z"), "id": 684434319704633344, "text": "SONIC Drive-In: General Manager (#Monroe, LA) https://t.co/B6fR6jflVQ #Hospitality #SONIC #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.0722021,32.5748129"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Monroe", "Hospitality", "SONIC", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 132057683, "name": "TMJ-LAM HRTA Jobs", "screen_name": "tmj_LAM_HRTA", "lang": "en", "location": "Monroe, LA", "create_at": date("2010-04-11"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Monroe, LA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 294, "friends_count": 272, "statues_count": 128 }, "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-05T10:00:32.000Z"), "id": 684434319729790976, "text": "Lit like bic https://t.co/zgsnIjXL8c", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1835020873, "name": "syd.", "screen_name": "__xosydney", "lang": "en", "location": "Atwater, CA", "create_at": date("2013-09-08"), "description": "17 | I always had $hit", "followers_count": 729, "friends_count": 151, "statues_count": 3283 }, "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-05T10:00:32.000Z"), "id": 684434319868170240, "text": "This rain ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160403022, "name": "Joshua Munguia", "screen_name": "josh_gilligan", "lang": "en", "location": "null", "create_at": date("2010-06-27"), "description": "21 years young.\nLiving the EMT life and I live in LA and im working my way up to Firefighter.", "followers_count": 167, "friends_count": 142, "statues_count": 8710 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:32.000Z"), "id": 684434319880749056, "text": "Love's Travel Stops & Country St... #Automotive #Job: Diesel Mechanic - Tire Care (#Canaan, NY) https://t.co/tGaBpPG5yr #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.4470553,42.4120287"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Automotive", "Job", "Canaan", "Jobs", "Hiring" }}, "user": { "id": 180525195, "name": "New York Automotive", "screen_name": "tmj_NY_auto", "lang": "en", "location": "New York", "create_at": date("2010-08-19"), "description": "Follow this account for geo-targeted Automotive job tweets in New York Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 175, "friends_count": 126, "statues_count": 80 }, "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": 36021, "countyName": "Columbia" } }
+{ "create_at": datetime("2016-01-05T10:00:32.000Z"), "id": 684434320174497792, "text": "@RealTrePlanes naw he funny asf ������������������", "in_reply_to_status": 684434194873888769, "in_reply_to_user": 203817914, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 203817914 }}, "user": { "id": 2567398420, "name": "MP", "screen_name": "immyles_away", "lang": "en", "location": "in kent ", "create_at": date("2014-05-27"), "description": "student Follow tumblr: mylespoole @:immyles_away", "followers_count": 829, "friends_count": 649, "statues_count": 6016 }, "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-01-05T10:00:32.000Z"), "id": 684434320228876288, "text": "#Cinnaminson, NJ #Sales #Job: Retail Sales Associate at CORT Business Services Corp. https://t.co/EcsJbngHCr #sales #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.991489,39.9999"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Cinnaminson", "Sales", "Job", "sales", "Jobs", "Hiring" }}, "user": { "id": 1425504374, "name": "CORT Careers", "screen_name": "CORTCareers", "lang": "en", "location": "Nationwide", "create_at": date("2013-05-13"), "description": "Join the CORT team where we help make a house a home, an office a great place to work, and an event a memorable celebration. @CORTFurniture", "followers_count": 674, "friends_count": 492, "statues_count": 1193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cinnaminson, NJ", "id": "01b0b0510afc9b5a", "name": "Cinnaminson", "place_type": "city", "bounding_box": rectangle("-75.029462,39.96743 -74.958395,40.032623") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-05T10:00:32.000Z"), "id": 684434320241614848, "text": "@HP_Ky @MereNON__ Workin Finishin Up This Tape Yu ?", "in_reply_to_status": 684434086677458944, "in_reply_to_user": 236537959, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 236537959, 253718140 }}, "user": { "id": 253718140, "name": "N.O.N", "screen_name": "MereNON__", "lang": "en", "location": "null", "create_at": date("2011-02-17"), "description": "For Bookings, Verses, Or Hooks: |chamereyoung@gmail.com|", "followers_count": 1285, "friends_count": 492, "statues_count": 41614 }, "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-05T10:00:32.000Z"), "id": 684434320963035136, "text": "@FranziaMom hell yes!!!", "in_reply_to_status": 684434116897599488, "in_reply_to_user": 24324469, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24324469 }}, "user": { "id": 1209926700, "name": "joe p", "screen_name": "vertachi", "lang": "en", "location": "buffalo, ny", "create_at": date("2013-02-22"), "description": "fo*k", "followers_count": 1016, "friends_count": 298, "statues_count": 3409 }, "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-05T10:00:32.000Z"), "id": 684434321093050368, "text": "@dawnmarie4182 @StephenKing So your solution to the problem is? At least he's trying to do something to stop people from being murdered.", "in_reply_to_status": 684433951268716544, "in_reply_to_user": 3436765366, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3436765366, 2233154425 }}, "user": { "id": 32446433, "name": "#KeepPounding", "screen_name": "AbsolutBex", "lang": "en", "location": "North Carolina ", "create_at": date("2009-04-17"), "description": "Taurus, Atheist, Liberal, INFJ", "followers_count": 770, "friends_count": 358, "statues_count": 155655 }, "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-05T10:00:32.000Z"), "id": 684434321147478017, "text": "See our latest #StPaul, MN #job and click to apply: Temporary - ESI Clerk/eDiscovery Technician - https://t.co/x2FVZaJvnw #RHLegalJobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.0941242,45.0325431"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "StPaul", "job", "RHLegalJobs" }}, "user": { "id": 791292943, "name": "RH Legal Jobs", "screen_name": "RHLegalJobs", "lang": "en", "location": "null", "create_at": date("2012-08-30"), "description": "See all our job openings for Robert Half Legal, providing highly skilled legal professionals on a project and full-time basis.", "followers_count": 85, "friends_count": 7, "statues_count": 2945 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Canada, MN", "id": "66cac6a4900d1831", "name": "Little Canada", "place_type": "city", "bounding_box": rectangle("-93.106288,45.006245 -93.055806,45.043202") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27123, "countyName": "Ramsey", "cityID": 2737502, "cityName": "Little Canada" } }
+{ "create_at": datetime("2016-01-05T10:00:33.000Z"), "id": 684434321222963201, "text": "We're #hiring! Click to apply: Auto Tire/Lube Technician - FT $300 BONUS! - https://t.co/9jefzhcFBe #Automotive #mechanic #NewUlm, MN #Job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.461114,44.312225"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Automotive", "mechanic", "NewUlm", "Job" }}, "user": { "id": 2963298858, "name": "Royal Tire Jobs", "screen_name": "RoyalTireJobs", "lang": "en", "location": "St. Cloud, MN", "create_at": date("2015-01-07"), "description": "Since 1948, Royal Tire has been providing customers with quality transportation care, products, and services.", "followers_count": 72, "friends_count": 279, "statues_count": 31 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Ulm, MN", "id": "40aef2dae5f2e9cd", "name": "New Ulm", "place_type": "city", "bounding_box": rectangle("-94.507217,44.280315 -94.414288,44.344475") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27015, "countyName": "Brown", "cityID": 2746042, "cityName": "New Ulm" } }
+{ "create_at": datetime("2016-01-05T10:00:33.000Z"), "id": 684434321269198848, "text": "Love @Buzzsumo. �� https://t.co/0Q6UUSk0x3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 919759388 }}, "user": { "id": 51852208, "name": "Janell McILwain", "screen_name": "JanellMcIlwain", "lang": "en", "location": "#DMV, Southern Maryland", "create_at": date("2009-06-28"), "description": "#SocialMedia Enthusiast helping Faith-Based & Small Biz Build Relationships through Social Media | Graphic Designer | Lover of the WORD | Referral MD Agent", "followers_count": 2237, "friends_count": 1737, "statues_count": 5254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waldorf, MD", "id": "4ddec4770a850303", "name": "Waldorf", "place_type": "city", "bounding_box": rectangle("-76.985159,38.608878 -76.841929,38.660379") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24017, "countyName": "Charles", "cityID": 2481175, "cityName": "Waldorf" } }
+{ "create_at": datetime("2016-01-05T10:00:33.000Z"), "id": 684434321357209601, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 483279023, "name": "Zack's Girl❤", "screen_name": "ForeverBriL", "lang": "en", "location": "California, USA", "create_at": date("2012-02-04"), "description": "Get to know me, before you judge me, that'd be nice! \nZack (12/29/15) ❤\n#Milso", "followers_count": 9352, "friends_count": 2683, "statues_count": 171856 }, "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-01-05T10:00:33.000Z"), "id": 684434321474596867, "text": "This #Hospitality #job might be a great fit for you: Line Cook - https://t.co/TPxrefzXK5 #Tucson, AZ #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.9776482,32.2843865"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Tucson", "Hiring", "CareerArc" }}, "user": { "id": 22511844, "name": "TMJ-TUC HRTA Jobs", "screen_name": "tmj_tuc_hrta", "lang": "en", "location": "Tucson, AZ", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Tucson, AZ. Need help? Tweet us at @CareerArc!", "followers_count": 347, "friends_count": 291, "statues_count": 166 }, "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-05T10:00:33.000Z"), "id": 684434321638293504, "text": "@gabiliel Ahhhhh já ouvi falar", "in_reply_to_status": 684431881971064832, "in_reply_to_user": 181347631, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user_mentions": {{ 181347631 }}, "user": { "id": 151604442, "name": "Haus of Tacacá", "screen_name": "HollaTaio", "lang": "en", "location": "null", "create_at": date("2010-06-03"), "description": "Made in Brazil and exported to the world. Born nigrinha. You may see my bedroom as a bedroom but actually it's my tomb.", "followers_count": 88, "friends_count": 54, "statues_count": 11573 }, "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-05T10:00:33.000Z"), "id": 684434321805979649, "text": "If you're a #Nursing professional in #SanAntonio, TX, check out this #job: https://t.co/ej7PwZYotl #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.4914012,29.6205416"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "SanAntonio", "job", "Hiring" }}, "user": { "id": 259944587, "name": "BaptistHealthCareers", "screen_name": "BHSCareers", "lang": "en", "location": "San Antonio, TX", "create_at": date("2011-03-02"), "description": "Since 1903, Baptist Health System has been involved in the community, caring for the health and spiritual needs of people in San Antonio and beyond. Join us!", "followers_count": 257, "friends_count": 160, "statues_count": 2085 }, "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-05T10:00:33.000Z"), "id": 684434321826971648, "text": ".@JohnChen check it out, do not left beyond...https://t.co/e2AnWxXeTA https://t.co/PLi1sZ3Xbi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2226739236 }}, "user": { "id": 582370581, "name": "Mo J", "screen_name": "mosheschnapp", "lang": "en", "location": "Beverly Hills, CA", "create_at": date("2012-05-16"), "description": "null", "followers_count": 60, "friends_count": 207, "statues_count": 1070 }, "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-05T10:00:33.000Z"), "id": 684434321986334720, "text": "Want to work at Harveys? We're #hiring in #BAXLEY, GA! Click for details: https://t.co/YCe1SuJ8LK #Retail #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.3068927,31.7640769"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "BAXLEY", "Retail", "Job", "Jobs" }}, "user": { "id": 2706847724, "name": "Harveys Jobs", "screen_name": "HarveysJobs", "lang": "en", "location": "null", "create_at": date("2014-08-04"), "description": "null", "followers_count": 1, "friends_count": 0, "statues_count": 3112 }, "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": 13001, "countyName": "Appling", "cityID": 1306016, "cityName": "Baxley" } }
+{ "create_at": datetime("2016-01-05T10:00:33.000Z"), "id": 684434322074419200, "text": "Not trying to go to the gym today.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 554221983, "name": "Paige Mccann", "screen_name": "paige_mccann66", "lang": "en", "location": "under a rock", "create_at": date("2012-04-14"), "description": "cold hearted & ruthless", "followers_count": 398, "friends_count": 318, "statues_count": 29230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roscoe, IL", "id": "016f3f384a13c6aa", "name": "Roscoe", "place_type": "city", "bounding_box": rectangle("-89.059905,42.383638 -88.940615,42.49081") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17201, "countyName": "Winnebago", "cityID": 1765611, "cityName": "Roscoe" } }
+{ "create_at": datetime("2016-01-05T10:00:33.000Z"), "id": 684434322372218880, "text": "Want to work in #WestCapeMay, NJ? View our latest opening: https://t.co/qlfmUbBmxh #Retail #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.9418404,38.9387236"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WestCapeMay", "Retail", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 59785077, "name": "TMJ-NJ Retail Jobs", "screen_name": "tmj_nj_retail", "lang": "en", "location": "New Jersey", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in New Jersey Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 409, "friends_count": 301, "statues_count": 534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Cape May, NJ", "id": "00678d66276a97b8", "name": "West Cape May", "place_type": "city", "bounding_box": rectangle("-74.9704,38.932712 -74.921113,38.966489") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34009, "countyName": "Cape May", "cityID": 3478530, "cityName": "West Cape May" } }
+{ "create_at": datetime("2016-01-05T10:00:33.000Z"), "id": 684434322376364032, "text": "Covelli Enterprises: Panera Bread Shift Leaders/ Hourly Associates New... (#Chillicothe, OH) https://t.co/ia71oH7ZrU #Hospitality #parttime", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.5930338,39.2039915"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Chillicothe", "Hospitality", "parttime" }}, "user": { "id": 2352066656, "name": "Covelli Enterprises", "screen_name": "CovelliJobs", "lang": "en", "location": "null", "create_at": date("2014-02-19"), "description": "At Covelli Enterprises, we pride ourselves on our unrivaled atmosphere, outstanding customer service, and talented people. Apply to our open #jobs here!", "followers_count": 62, "friends_count": 39, "statues_count": 281 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsboro, OH", "id": "b411bb59c0ada8f6", "name": "Hillsboro", "place_type": "city", "bounding_box": rectangle("-83.641444,39.176052 -83.589688,39.239334") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39071, "countyName": "Highland", "cityID": 3935560, "cityName": "Hillsboro" } }
+{ "create_at": datetime("2016-01-05T10:00:33.000Z"), "id": 684434322577821696, "text": "@cindizinha amiga coma", "in_reply_to_status": 684428612645289984, "in_reply_to_user": 27728367, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user_mentions": {{ 27728367 }}, "user": { "id": 33095396, "name": "Cheio de assunto", "screen_name": "viniciusrlima", "lang": "en", "location": "Grand Rapids, MI", "create_at": date("2009-04-18"), "description": "Cada dia mais rabugento", "followers_count": 6891, "friends_count": 1645, "statues_count": 101652 }, "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-05T10:00:33.000Z"), "id": 684434322649157632, "text": "Prob could eat almonds and cashews and be happy for the rest of my life tbh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 491677391, "name": "lay", "screen_name": "sliimmshAD", "lang": "en", "location": "null", "create_at": date("2012-02-13"), "description": "long islandddddd", "followers_count": 338, "friends_count": 337, "statues_count": 5398 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin Square, NY", "id": "4bfbb13bd35cf5e0", "name": "Franklin Square", "place_type": "city", "bounding_box": rectangle("-73.692411,40.682829 -73.661034,40.717994") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3627309, "cityName": "Franklin Square" } }
+{ "create_at": datetime("2016-01-05T10:00:33.000Z"), "id": 684434322682589186, "text": "So true and what you want fades, then the thing that you had fades and you lose both ���� https://t.co/d1kk19NeuF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29249865, "name": "Ralinda", "screen_name": "hickey15", "lang": "en", "location": "null", "create_at": date("2009-04-06"), "description": "health and exercise baby", "followers_count": 152, "friends_count": 628, "statues_count": 878 }, "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-05T10:00:33.000Z"), "id": 684434322741313536, "text": "Anxiety is the devils cocaine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2969549970, "name": "Scene-y Weenie", "screen_name": "onemanxcult", "lang": "en", "location": "Clearfield, UT", "create_at": date("2015-01-09"), "description": "John 3:16|your local sad girl|xaela|straight edge|", "followers_count": 234, "friends_count": 553, "statues_count": 3176 }, "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-01-05T10:00:33.000Z"), "id": 684434322867142657, "text": "The difference between a flower and weed is a judgement.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 214621000, "name": "jeanette", "screen_name": "jean_ette23", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-11-11"), "description": "Así o más claro?", "followers_count": 357, "friends_count": 280, "statues_count": 14130 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:33.000Z"), "id": 684434322980519937, "text": "@J3sslcaMarie Isn’t that every politician.", "in_reply_to_status": 684433955911811072, "in_reply_to_user": 310613389, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 310613389 }}, "user": { "id": 18423953, "name": "Michael", "screen_name": "Mleatherb", "lang": "en", "location": "Sunshine State", "create_at": date("2008-12-27"), "description": "QA Engineer \n\nGamer | Geek | Dad", "followers_count": 914, "friends_count": 853, "statues_count": 22014 }, "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-01-05T10:00:33.000Z"), "id": 684434323177512960, "text": "Nurse Practitioner, Urgent Care - MedExpress: (#Cabot, AR) https://t.co/Yr1vqpN3iw #Healthcare #nurse #ArkansasJobs #Veterans #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.0165336,34.974532"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Cabot", "Healthcare", "nurse", "ArkansasJobs", "Veterans", "Job", "Jobs" }}, "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": 174, "friends_count": 32, "statues_count": 856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cabot, AR", "id": "7c96add4b12f4768", "name": "Cabot", "place_type": "city", "bounding_box": rectangle("-92.088415,34.928053 -91.96429,35.037344") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5085, "countyName": "Lonoke", "cityID": 510300, "cityName": "Cabot" } }
+{ "create_at": datetime("2016-01-05T10:00:33.000Z"), "id": 684434323269881861, "text": "A look North. [Cam 2] on Tuesday, January 5, 2016 @ 1:00:27 PM #CarolinaWx #ClaytonNC https://t.co/aAeUQDLqHJ", "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": 264, "friends_count": 35, "statues_count": 714986 }, "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.32187,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37101, "countyName": "Johnston" } }
+{ "create_at": datetime("2016-01-05T10:00:33.000Z"), "id": 684434323521597440, "text": "@llanky_ exactly what you worried about tyga for he's living already!", "in_reply_to_status": 684434182571974656, "in_reply_to_user": 2278218721, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2278218721 }}, "user": { "id": 2889405232, "name": "Mr.Wu", "screen_name": "KidLuWu", "lang": "en", "location": "Orlando, FL ", "create_at": date("2014-11-23"), "description": "#Blessed", "followers_count": 901, "friends_count": 328, "statues_count": 17039 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hunters Creek, FL", "id": "0a7eb306405f5129", "name": "Hunters Creek", "place_type": "city", "bounding_box": rectangle("-81.447294,28.347846 -81.404031,28.375674") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1232967, "cityName": "Hunters Creek" } }
+{ "create_at": datetime("2016-01-05T10:00:33.000Z"), "id": 684434323861159937, "text": "I hate when ppl assume random shit", "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 asf C/O 2015\r// 9teen♌\n@_Fraydoe followed July 29, 2015 10:30 pm\n@AllenSarinana_ followed August 1, 2015 4:40pm @Em3_Music followed October 31, 2015 9:00pm", "followers_count": 677, "friends_count": 1002, "statues_count": 9194 }, "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-05T10:00:33.000Z"), "id": 684434323970375680, "text": "Home for the next few days �� @ Blue Ridge Mountains, Ga https://t.co/jU5RJobIKI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.27435576,34.9621289"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 272108552, "name": "Millie", "screen_name": "sillymillie_123", "lang": "en", "location": "•Atlanta•Athens•", "create_at": date("2011-03-25"), "description": "Compassion. Tolerance. Ambition. Wit. For bookings: ectagency@gmail.com instagram: sillymillie123 https://soundcloud.com/millie-rose-evans", "followers_count": 4446, "friends_count": 991, "statues_count": 56180 }, "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": 13111, "countyName": "Fannin" } }
+{ "create_at": datetime("2016-01-05T10:00:33.000Z"), "id": 684434324129579009, "text": "Store Seasonal Employee - Lowe's: (#OklahomaCity, OK) https://t.co/e6o8DJsv64 #Retail #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.5164276,35.4675602"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "OklahomaCity", "Retail", "Job", "Jobs", "Hiring" }}, "user": { "id": 28479478, "name": "TMJ-OKC Retail Jobs", "screen_name": "tmj_okc_retail", "lang": "en", "location": "Oklahoma City, OK", "create_at": date("2009-04-02"), "description": "Follow this account for geo-targeted Retail job tweets in Oklahoma City, OK. Need help? Tweet us at @CareerArc!", "followers_count": 360, "friends_count": 309, "statues_count": 340 }, "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-05T10:00:33.000Z"), "id": 684434324335116288, "text": "why you gotta be one bish for", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2708313854, "name": "LV", "screen_name": "LeahVidad", "lang": "en", "location": "null", "create_at": date("2014-08-04"), "description": "a girls twitter is full of hints", "followers_count": 124, "friends_count": 100, "statues_count": 724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lanai City, HI", "id": "7a1f46f0851bcd6b", "name": "Lanai City", "place_type": "city", "bounding_box": rectangle("-156.931017,20.814204 -156.905218,20.836602") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15009, "countyName": "Maui", "cityID": 1543700, "cityName": "Lanai City" } }
+{ "create_at": datetime("2016-01-05T10:00:33.000Z"), "id": 684434324607897600, "text": "Tunnel vision to graduation and life after undergrad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 210603987, "name": "S.", "screen_name": "StephInTheeCity", "lang": "en", "location": "U.S - U.K", "create_at": date("2010-10-31"), "description": "Where West Philly meets Westminster | college girl", "followers_count": 1073, "friends_count": 68, "statues_count": 40039 }, "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-05T10:00:33.000Z"), "id": 684434324637155328, "text": "Airport Location Manager - Puerto Rico: (#Carolina) https://t.co/ZyMKWwkbaH #Empleo #Empleos #Contratar #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-65.9573872,18.3807819"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "Carolina", "Empleo", "Empleos", "Contratar", "CareerArc" }}, "user": { "id": 43258850, "name": "TMJ-PTR Jobs", "screen_name": "tmj_ptr_jobs", "lang": "en", "location": "Puerto Rico", "create_at": date("2009-05-28"), "description": "Follow this account for geo-targeted Other job tweets in Puerto Rico. Need help? Tweet us at @CareerArc!", "followers_count": 900, "friends_count": 562, "statues_count": 10300 }, "place": { "country": "Puerto Rico", "country_code": "Puerto Rico", "full_name": "Carolina, Puerto Rico", "id": "862fdecf3a49a2c9", "name": "Carolina", "place_type": "city", "bounding_box": rectangle("-66.043086,18.353142 -65.918779,18.458693") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72031, "countyName": "Carolina", "cityID": 7214290, "cityName": "Carolina" } }
+{ "create_at": datetime("2016-01-05T10:00:33.000Z"), "id": 684434324666597376, "text": "Happy birthday �� @gabriellapulice", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 262911971 }}, "user": { "id": 499551145, "name": "Anthony Beaubrun", "screen_name": "Ant__Mazi", "lang": "en", "location": "United States", "create_at": date("2012-02-21"), "description": "UC Football!", "followers_count": 693, "friends_count": 618, "statues_count": 3030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hempstead, NY", "id": "1ed0698bc8262763", "name": "West Hempstead", "place_type": "city", "bounding_box": rectangle("-73.668154,40.68238 -73.631865,40.71274") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3680225, "cityName": "West Hempstead" } }
+{ "create_at": datetime("2016-01-05T10:00:33.000Z"), "id": 684434324704239616, "text": ".@bigal953 @owillis @NickKristof the word \"regulated\" is right there in the ammendment! #StopGunViolence https://t.co/erMX82DYmf", "in_reply_to_status": 684433186621943808, "in_reply_to_user": 22221695, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "StopGunViolence" }}, "user_mentions": {{ 22221695, 3497941, 17004618 }}, "user": { "id": 328231381, "name": "stu stein", "screen_name": "thestustein", "lang": "en", "location": "New York", "create_at": date("2011-07-02"), "description": "Father, Husband, Teacher, Obamacrat. Please help me attain my life long goal of getting a follow back from @barackobama", "followers_count": 1228, "friends_count": 882, "statues_count": 14582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Bellmore, NY", "id": "ea82a33637cabb81", "name": "North Bellmore", "place_type": "city", "bounding_box": rectangle("-73.560023,40.67418 -73.52109,40.706377") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3651517, "cityName": "North Bellmore" } }
+{ "create_at": datetime("2016-01-05T10:00:33.000Z"), "id": 684434324813262848, "text": "Many #ksleg leaders suggest KS cannot afford to expand #KanCare. KS can invest in vital workforce, economy by expanding health coverage.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ksleg", "KanCare" }}, "user": { "id": 370310279, "name": "Pattie Mansur", "screen_name": "MansurPl", "lang": "en", "location": "Merriam, Kansas", "create_at": date("2011-09-08"), "description": "Oversee communications, health policy at REACH Healthcare Foundation, dedicated to achieving positive health in KC region. Also passionate about public ed.", "followers_count": 498, "friends_count": 367, "statues_count": 3358 }, "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-05T10:00:33.000Z"), "id": 684434324909854721, "text": "@Brie_Rogers24 @ChrisArcher42 you gon make him catch a case", "in_reply_to_status": 684434085465448450, "in_reply_to_user": 1618043856, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1618043856, 627571568 }}, "user": { "id": 226791156, "name": "christian", "screen_name": "WhoChristian", "lang": "en", "location": "null", "create_at": date("2010-12-14"), "description": "Add a bio to your profile", "followers_count": 1502, "friends_count": 350, "statues_count": 104851 }, "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-05T10:00:33.000Z"), "id": 684434325316612097, "text": "Were you. In a car accident. Or did your pants come. Like that-plumlee https://t.co/eRQUTR2Vmh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 195995175, "name": "im jalyn", "screen_name": "Jdizzy78", "lang": "en", "location": "null", "create_at": date("2010-09-27"), "description": "half the time i don't know what im talking about\ninsta:txtllyrad\nsc:jboogies677", "followers_count": 56, "friends_count": 102, "statues_count": 345 }, "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-05T10:00:33.000Z"), "id": 684434325354319872, "text": "Fuck a makeups real tawlk https://t.co/hxeDEk36Gt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2209833191, "name": "BURNTmacaronie", "screen_name": "Toni2Turnt", "lang": "en", "location": "Houston, TX where da cows poop", "create_at": date("2013-12-05"), "description": "Fuck you and that weak shit!", "followers_count": 416, "friends_count": 390, "statues_count": 42534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:34.000Z"), "id": 684434325417283584, "text": "New Signature: Engagement Manager/Project Manager (Washington, DC METRO) (#Washington, DC) https://t.co/SSXd9VLQh9 #ProjectMgmt #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.0363658,38.8951118"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Washington", "ProjectMgmt", "Job", "Jobs" }}, "user": { "id": 4499586313, "name": "New Signature Jobs", "screen_name": "NewSigCareers", "lang": "en", "location": "null", "create_at": date("2015-12-08"), "description": "Follow this page for open #jobs at @newsignature Now hiring #IT, #computer #science, #web #software #job", "followers_count": 13, "friends_count": 86, "statues_count": 23 }, "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-05T10:00:34.000Z"), "id": 684434325450977282, "text": "That shower was everything I imagined 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": 342308756, "name": "Tae", "screen_name": "tweetsby_tae", "lang": "en", "location": "3️⃣3️⃣0️⃣", "create_at": date("2011-07-25"), "description": "• IG - tae_nicolee • KSU'18 • people will stare at you, make it worth their while •", "followers_count": 1049, "friends_count": 661, "statues_count": 70196 }, "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-01-05T10:00:34.000Z"), "id": 684434325916495872, "text": "These niggas ain't got no heart", "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": "⛽️™®", "screen_name": "_bravojohnny", "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": 2024, "friends_count": 1771, "statues_count": 14399 }, "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-05T10:00:34.000Z"), "id": 684434326025535489, "text": "I'm not even fully up yet", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1388423942, "name": "hevhev", "screen_name": "GMH__", "lang": "en", "location": "con chichi ", "create_at": date("2013-04-28"), "description": "sweet , single & sassy", "followers_count": 1140, "friends_count": 771, "statues_count": 29573 }, "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-05T10:00:34.000Z"), "id": 684434326038147073, "text": "@_briyanaann okay okay ��", "in_reply_to_status": 684434241376141316, "in_reply_to_user": 462909970, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 462909970 }}, "user": { "id": 624142277, "name": ".", "screen_name": "EyeAmGuapo", "lang": "en", "location": "null", "create_at": date("2012-07-01"), "description": "null", "followers_count": 1332, "friends_count": 876, "statues_count": 68310 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Bay Shore, NY", "id": "f87e333473584780", "name": "North Bay Shore", "place_type": "city", "bounding_box": rectangle("-73.281541,40.74276 -73.228098,40.777154") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3651495, "cityName": "North Bay Shore" } }
+{ "create_at": datetime("2016-01-05T10:00:34.000Z"), "id": 684434326067396608, "text": "@yvettejasmine_ bathrooms next", "in_reply_to_status": 684431503388852225, "in_reply_to_user": 162118094, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 162118094 }}, "user": { "id": 183498419, "name": "Jock Cousteau", "screen_name": "DonaldHermann", "lang": "en", "location": "San Leandro, California", "create_at": date("2010-08-26"), "description": "ϟ 9¾ DVC Grad⚾️ sc: DonaldSL ig: fuckdonald7", "followers_count": 601, "friends_count": 674, "statues_count": 54610 }, "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-05T10:00:34.000Z"), "id": 684434326189019136, "text": "Lol @ any one that thinks I'm mad bc they don't fwm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4502620394, "name": "liz", "screen_name": "michelleTheTwin", "lang": "en", "location": "i was in MD & GA now Louisiana", "create_at": date("2015-12-08"), "description": "#McNeeseStateUniversity", "followers_count": 233, "friends_count": 172, "statues_count": 872 }, "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-05T10:00:34.000Z"), "id": 684434326562320384, "text": "the fact Hailey and Justin are a thing 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": 2559521488, "name": "taylor rieter", "screen_name": "taylorrieterrr", "lang": "en", "location": "Lakewood, OH", "create_at": date("2014-05-22"), "description": "ohio (snapchat: taylorrieter1)", "followers_count": 712, "friends_count": 380, "statues_count": 9881 }, "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-01-05T10:00:34.000Z"), "id": 684434326725853184, "text": "Hi, Banksy! https://t.co/OUj0uQWzT1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 39527574, "name": "Kallye Baggett", "screen_name": "kal_e", "lang": "en", "location": "null", "create_at": date("2009-05-12"), "description": "Sometimes the right thing & the hard thing are the same thing. I read that on a tea bag.", "followers_count": 619, "friends_count": 861, "statues_count": 5381 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Park City, UT", "id": "01962b3d283cbfc2", "name": "Park City", "place_type": "city", "bounding_box": rectangle("-111.530974,40.619883 -111.467325,40.686679") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49043, "countyName": "Summit", "cityID": 4958070, "cityName": "Park City" } }
+{ "create_at": datetime("2016-01-05T10:00:34.000Z"), "id": 684434326822481920, "text": "@Southbeach_Mob yes", "in_reply_to_status": 684426666463117312, "in_reply_to_user": 323394237, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 323394237 }}, "user": { "id": 613126014, "name": "Mec⚡️", "screen_name": "meccca__", "lang": "en", "location": "Somewhere chiefin' keef", "create_at": date("2012-06-19"), "description": "#RipAva ❤️", "followers_count": 855, "friends_count": 533, "statues_count": 31111 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cambridge, MD", "id": "b58dc419d99ec532", "name": "Cambridge", "place_type": "city", "bounding_box": rectangle("-76.112715,38.536459 -76.008742,38.584999") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24019, "countyName": "Dorchester", "cityID": 2412400, "cityName": "Cambridge" } }
+{ "create_at": datetime("2016-01-05T10:00:34.000Z"), "id": 684434327040475140, "text": "@KeepTheRamsNStL - We all believe you, INBRED.", "in_reply_to_status": 684434121829974016, "in_reply_to_user": 36859452, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 36859452 }}, "user": { "id": 79819109, "name": "Eric Geller", "screen_name": "ericsgeller", "lang": "en", "location": "Los Angeles", "create_at": date("2009-10-04"), "description": "Born in Paris, France. Raised in Los Angeles. Sports Broadcast Journalist, Field & Web Reporter/Producer at The Beast 980 covering ALL Los Angeles Sports.", "followers_count": 398, "friends_count": 763, "statues_count": 4500 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:34.000Z"), "id": 684434327321587716, "text": "@Aubs89 @nikroybal @Bencjacobs Weaver has as much respect for the Democratic Party as Dick Cheney or Mike Huckabee...", "in_reply_to_status": 684433041377345536, "in_reply_to_user": 80619457, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 80619457, 17022638, 218325695 }}, "user": { "id": 3302993621, "name": "tom Morrison", "screen_name": "tommorrison950", "lang": "en", "location": "Newark, DE", "create_at": date("2015-05-29"), "description": "Obama loyalist. politically engaged. optimist with reality streak.", "followers_count": 581, "friends_count": 620, "statues_count": 44236 }, "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-01-05T10:00:34.000Z"), "id": 684434327526969344, "text": "We are looking for a buyer on Lot 62 Malden Court #Crouse #NC https://t.co/NCDlKIjX6j #realestate https://t.co/D2tAQUUj4o", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3266695,35.4473422"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Crouse", "NC", "realestate" }}, "user": { "id": 2445654229, "name": "RE/MAX Crossroads", "screen_name": "lincolntonremax", "lang": "en", "location": "Lincolnton NC", "create_at": date("2014-04-15"), "description": "null", "followers_count": 71, "friends_count": 43, "statues_count": 6134 }, "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.32187,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37045, "countyName": "Cleveland" } }
+{ "create_at": datetime("2016-01-05T10:00:34.000Z"), "id": 684434327602515968, "text": "@nlpantyhose @swo2212 @mostlyboobz @WeFapToThis @happyrodney Mmmmm rica Mamacita I want you baby", "in_reply_to_status": 684433664395087872, "in_reply_to_user": 566481641, "favorite_count": 0, "coordinate": point("-117.9908857,33.788053"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 566481641, 894989119, 3539078542, 633000829, 2301844219 }}, "user": { "id": 3168000822, "name": "Salvador Huerta", "screen_name": "slvdrhuerta", "lang": "es", "location": "null", "create_at": date("2015-04-22"), "description": "null", "followers_count": 274, "friends_count": 1970, "statues_count": 946 }, "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-05T10:00:34.000Z"), "id": 684434327673778176, "text": "@Stillmatic_12 i really really like that line", "in_reply_to_status": 684433980939190276, "in_reply_to_user": 772409804, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 772409804 }}, "user": { "id": 1329532921, "name": "Tony SooWoo", "screen_name": "OgHasnt", "lang": "en", "location": "Chicago✈Los Angeles ", "create_at": date("2013-04-05"), "description": "Born on southside of Chicago, live in LA. Stay focused and keep going. Lifting is Life, so is music... And also Halo (Prop. 215 & S.B. 420) #RealHipHop", "followers_count": 587, "friends_count": 526, "statues_count": 10612 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:34.000Z"), "id": 684434327829110785, "text": "Only a steady diet of ribs and coffee could have gotten me this far!… https://t.co/KhRbD544Cz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.682,35.8047"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1321314757, "name": "Seth Vlach", "screen_name": "sumofseth", "lang": "en", "location": "null", "create_at": date("2013-04-01"), "description": "I Build Digital Experiences.", "followers_count": 22, "friends_count": 21, "statues_count": 271 }, "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": 6107, "countyName": "Tulare", "cityID": 658422, "cityName": "Posey" } }
+{ "create_at": datetime("2016-01-05T10:00:34.000Z"), "id": 684434327988494337, "text": "Algo debe quedar claro luego de este primer debate, esto será el pasado vs el presente. #ElCambioComienzaHoy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "ElCambioComienzaHoy" }}, "user": { "id": 75869646, "name": "Julio J. Hernández", "screen_name": "JulioJHernandez", "lang": "es", "location": "Rep. Independiente del Zulia", "create_at": date("2009-09-20"), "description": "Lcdo. Comunicación Social Conductor del programa @BatazoEntre2 y Deporte Express por Eclipse 88.3fm todos los sábados de 12m a 2pm.", "followers_count": 1602, "friends_count": 1098, "statues_count": 32863 }, "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-01-05T10:00:34.000Z"), "id": 684434328240144384, "text": "@A_Lewie3 you suck", "in_reply_to_status": 684423648573288448, "in_reply_to_user": 140157880, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 140157880 }}, "user": { "id": 463193157, "name": "Big Kurt", "screen_name": "kurt_k95", "lang": "en", "location": "Morgantown, WV. ", "create_at": date("2012-01-13"), "description": "20. Just a nice guy tryna finish first. WVU'17 #HTTR", "followers_count": 249, "friends_count": 219, "statues_count": 4229 }, "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-05T10:00:34.000Z"), "id": 684434328244191232, "text": "@KristinSavidge Update 2: Still at work about to die of sleep deprivation", "in_reply_to_status": 684373871156305920, "in_reply_to_user": 444889954, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 444889954 }}, "user": { "id": 444889954, "name": "K-Sav", "screen_name": "KristinSavidge", "lang": "en", "location": "Keller, TX ⏩ Stephenville, TX ", "create_at": date("2011-12-23"), "description": "ΑΓΔ | Tarleton 2019 | Don't mess with Texas", "followers_count": 804, "friends_count": 633, "statues_count": 10958 }, "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-01-05T10:00:34.000Z"), "id": 684434328399384576, "text": "Song of the South (1946) FREE immediate ship\nhttps://t.co/bm6RxhZwxM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 39915851, "name": "Brian Pinette", "screen_name": "damienrecords", "lang": "en", "location": "Austin, TEXAS", "create_at": date("2009-05-13"), "description": "Writer-Producer-Director INDIE FILMS, Theatre, Radio, Recordings", "followers_count": 1228, "friends_count": 2022, "statues_count": 91408 }, "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-05T10:00:34.000Z"), "id": 684434328529453056, "text": "Interested in a #Labor #job near #OSHKOSH, WI? This could be a great fit: https://t.co/6GGkDp0rvc #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.5930353,44.0282974"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Labor", "job", "OSHKOSH", "Hiring", "CareerArc" }}, "user": { "id": 118898049, "name": "TMJ-WI Labor Jobs", "screen_name": "tmj_WI_LABOR", "lang": "en", "location": "Wisconsin", "create_at": date("2010-03-01"), "description": "Follow this account for geo-targeted General Labor job tweets in Wisconsin Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 304, "friends_count": 287, "statues_count": 87 }, "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-01-05T10:00:34.000Z"), "id": 684434328726581248, "text": "#Marketing #Job in #Arlington, VA: Marketing Analyst at AvalonBay https://t.co/uUWzNBwWm0 #nowhiring #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.1067698,38.8799697"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "Marketing", "Job", "Arlington", "nowhiring", "Jobs", "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": 200, "friends_count": 13, "statues_count": 360 }, "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-05T10:00:34.000Z"), "id": 684434328734941185, "text": "While everyone is trying to move to nyc, I'm just going to move west.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 31585927, "name": "J", "screen_name": "simplyyjackieee", "lang": "en", "location": "null", "create_at": date("2009-04-15"), "description": "I'm sorry I can't stay over, I have meds to take and my contacts to take out.", "followers_count": 100, "friends_count": 159, "statues_count": 5136 }, "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-05T10:00:34.000Z"), "id": 684434328747651073, "text": "You ain't a bread winner to me if u can't go half on a college tuition", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287023963, "name": "CREAM", "screen_name": "nicchelleeee", "lang": "en", "location": "Compton, CA", "create_at": date("2011-04-23"), "description": "I'm tea on some days , and Hennessy on others", "followers_count": 1128, "friends_count": 794, "statues_count": 74125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:34.000Z"), "id": 684434328965652481, "text": "Mormons lmfao https://t.co/OLm8PCZ0LH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1698382008, "name": "GB 10-7", "screen_name": "lowkeyTJ", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-08-25"), "description": "Nas goat tbh", "followers_count": 263, "friends_count": 195, "statues_count": 8167 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:34.000Z"), "id": 684434328978374657, "text": "@khounsell2 ��❤️", "in_reply_to_status": 684416510782271488, "in_reply_to_user": 1037594588, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1037594588 }}, "user": { "id": 603032952, "name": "brianna knowles❥♛", "screen_name": "briannaknowles4", "lang": "en", "location": "null", "create_at": date("2012-06-08"), "description": "Your BFF was here, love you more than life itself!❤️ -Ivy snapchat- briannna1012", "followers_count": 370, "friends_count": 348, "statues_count": 7524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Southern Pines, NC", "id": "63ca7806a3195a6b", "name": "Southern Pines", "place_type": "city", "bounding_box": rectangle("-79.447992,35.122361 -79.361702,35.246879") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37125, "countyName": "Moore", "cityID": 3763120, "cityName": "Southern Pines" } }
+{ "create_at": datetime("2016-01-05T10:00:34.000Z"), "id": 684434329460592642, "text": "You sir are amazing! Thank you! #NRA #GOP - SUCK IT! You are NOT for true #Americans @realDonaldTrump u sir R weak https://t.co/hfTEILg7Ez", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NRA", "GOP", "Americans" }}, "user_mentions": {{ 25073877 }}, "user": { "id": 22339199, "name": "Calvin Dean", "screen_name": "calvindean", "lang": "en", "location": "Beverly Hills", "create_at": date("2009-02-28"), "description": "Actor - SAG/AFTRA Union Member lives in Beverly Hills, CA THE REAL (LA) *Calvin Dean* plays the Lead in the TV Show Unusual Suspects Episode barefoot homeside", "followers_count": 689, "friends_count": 2009, "statues_count": 2467 }, "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-01-05T10:00:35.000Z"), "id": 684434329577984001, "text": "Can you recommend anyone for this #job? Shift Leader - https://t.co/rA3ZuAc318 #PalmHarbor, FL #Hospitality #Veterans #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.7372972,28.0730569"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "PalmHarbor", "Hospitality", "Veterans", "Hiring" }}, "user": { "id": 2904664975, "name": "Einstein Bros. Jobs", "screen_name": "WorkAtEBB", "lang": "en", "location": "null", "create_at": date("2014-11-19"), "description": "For more information on careers at Einstein Bros. visit http://careers.einsteinnoah.com", "followers_count": 206, "friends_count": 38, "statues_count": 2219 }, "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-01-05T10:00:35.000Z"), "id": 684434330010161152, "text": "these fucking thots man", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2293102587, "name": "Dann", "screen_name": "DanielleGrahaam", "lang": "en", "location": "null", "create_at": date("2014-01-19"), "description": "Tottenville lax & DK3 lax #14, 10.9 ❤️", "followers_count": 588, "friends_count": 503, "statues_count": 15468 }, "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-05T10:00:35.000Z"), "id": 684434330056179712, "text": "@BBQLikeItsScott well I gotta pay rent somehow", "in_reply_to_status": 684434123843211264, "in_reply_to_user": 29061020, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 29061020 }}, "user": { "id": 11195952, "name": "Dapper Bloof", "screen_name": "pikestaff", "lang": "en", "location": "Montana", "create_at": date("2007-12-15"), "description": "Pike | Freelance writer | Patron Saint of Video Games | IRL blood elf | Blizzard Trash since '99 | #NotAllElves :3 | NOTE: I spam game screenshots", "followers_count": 1493, "friends_count": 588, "statues_count": 54498 }, "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-01-05T10:00:35.000Z"), "id": 684434330366558208, "text": "the best nap 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": 625952643, "name": "ast", "screen_name": "slimthumbin", "lang": "en", "location": "null", "create_at": date("2012-07-03"), "description": "ok cool", "followers_count": 2061, "friends_count": 2094, "statues_count": 19695 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bordentown, NJ", "id": "00c11996469e8af5", "name": "Bordentown", "place_type": "city", "bounding_box": rectangle("-74.719622,40.113799 -74.67425,40.183634") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34021, "countyName": "Mercer", "cityID": 3406670, "cityName": "Bordentown" } }
+{ "create_at": datetime("2016-01-05T10:00:35.000Z"), "id": 684434330391674880, "text": "Another day of babysitting ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2598571712, "name": "OgPrettyThug", "screen_name": "SandiferMarisol", "lang": "en", "location": "TX", "create_at": date("2014-07-01"), "description": "Focused✨|Volleyball❤️", "followers_count": 97, "friends_count": 148, "statues_count": 4678 }, "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-05T10:00:35.000Z"), "id": 684434330786095104, "text": "Hurts but I honestly don't give a fuck anymore.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 282771132, "name": "B.A.D", "screen_name": "Barb_Dip", "lang": "en", "location": "lost", "create_at": date("2011-04-15"), "description": "null", "followers_count": 725, "friends_count": 808, "statues_count": 33634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hyde Park, NY", "id": "01c3d12d4fe985e2", "name": "Hyde Park", "place_type": "city", "bounding_box": rectangle("-73.94838,41.735688 -73.852278,41.848195") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36027, "countyName": "Dutchess", "cityID": 3637198, "cityName": "Hyde Park" } }
+{ "create_at": datetime("2016-01-05T10:00:35.000Z"), "id": 684434331083771904, "text": "Update on Joint Waterline Replacement, Street Calming Project on Glascock Street https://t.co/0z9TqhvfQg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.6406859,35.77414893"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21774716, "name": "Marti Hampton", "screen_name": "Martihampton", "lang": "en", "location": "Raleigh, NC", "create_at": date("2009-02-24"), "description": "With over 37 years of experience in the Triangle real estate market, you can trust the Expert... #1 RE/MAX Office in The Carolinas... http://t.co/PENpIQvXLj", "followers_count": 671, "friends_count": 241, "statues_count": 4983 }, "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-05T10:00:35.000Z"), "id": 684434331247316992, "text": "If you're a #Insurance professional in #Hartford, CT, check out this #job: https://t.co/9xtb2wpYqd #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.6850932,41.7637111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Insurance", "Hartford", "job", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 395501501, "name": "CT Veteran Jobs", "screen_name": "tmj_ct_vets", "lang": "en", "location": "Connecticut Non-Metro", "create_at": date("2011-10-21"), "description": "Follow this account for geo-targeted Veteran job tweets in Connecticut Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 202, "friends_count": 117, "statues_count": 146 }, "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-05T10:00:35.000Z"), "id": 684434331591376897, "text": "Baker U ���� @ Liston Stadium https://t.co/YTc0LumGm7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.17829,38.7765"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 250396279, "name": "Alex De'Mon Baird", "screen_name": "ADBaird8", "lang": "en", "location": "Omaha➡️ OP ➡️Baldwin City, KS", "create_at": date("2011-02-10"), "description": "Cornerback and Track athlete for Baker University | #8 | #BTID |Exercise Science-Occupational Therapy | I have set goals that are beyond me, but im commited.", "followers_count": 925, "friends_count": 800, "statues_count": 13842 }, "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-01-05T10:00:35.000Z"), "id": 684434331704545281, "text": "Can you recommend anyone for this #Transportation #job? https://t.co/gtHrZKZYed #Driver #DallasFortWorth, TX #Hiring https://t.co/15ryRhudx4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.769923,32.802955"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Transportation", "job", "Driver", "DallasFortWorth", "Hiring" }}, "user": { "id": 2557323745, "name": "Labatt Food Service", "screen_name": "LabattFoodJobs", "lang": "en", "location": "null", "create_at": date("2014-06-09"), "description": "Labatt Food Service is a privately owned food distribution company. Follow this page for information about our open #job opportunities.", "followers_count": 158, "friends_count": 316, "statues_count": 101 }, "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-05T10:00:35.000Z"), "id": 684434331729661952, "text": "@TotallyTettey oh shit for real haha? I assumed you were in like your 20s or 30s. Well happy bday tomorrow then lol", "in_reply_to_status": 684434127727255552, "in_reply_to_user": 243319313, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 243319313 }}, "user": { "id": 23880967, "name": "Brian Anderson", "screen_name": "blyan", "lang": "en", "location": "Tacoma, WA", "create_at": date("2009-03-11"), "description": "I tweet a lot. I swear a lot. Sports, food, music. Sounders, Canucks, Norwich City, Arsenal, and Seahawks. Gaming/Youtube", "followers_count": 1452, "friends_count": 1674, "statues_count": 116604 }, "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-01-05T10:00:35.000Z"), "id": 684434331847163904, "text": "Happy Birthday fashionista, have the best day!!☺️���������� @brianamachele", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1257957014 }}, "user": { "id": 610920967, "name": "alexa nicole ☕️", "screen_name": "AlexaHarness", "lang": "en", "location": "California✨", "create_at": date("2012-06-17"), "description": "laters baby", "followers_count": 552, "friends_count": 474, "statues_count": 25768 }, "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-05T10:00:35.000Z"), "id": 684434331880718336, "text": "@apmvcth_ i don't care if he is rude in person or ignores me or just smiles but never says a word, i just want to be able to say he was", "in_reply_to_status": 684433213473685505, "in_reply_to_user": 1138471106, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1138471106 }}, "user": { "id": 1138471106, "name": "ana ♡", "screen_name": "apmvcth_", "lang": "en", "location": "fabulous las vegas", "create_at": date("2013-01-31"), "description": "you're the one that i want at the end of the day . ♡", "followers_count": 2086, "friends_count": 855, "statues_count": 90081 }, "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-05T10:00:35.000Z"), "id": 684434332279304192, "text": "@Panther_NoTiger @OneTreeHeal wait that's it? Why the fuck did they call that a war??", "in_reply_to_status": 684434159562035200, "in_reply_to_user": 36588790, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 36588790, 387477491 }}, "user": { "id": 164942222, "name": "Teth-Adam", "screen_name": "_Khatastrophe", "lang": "en", "location": "Wakanda", "create_at": date("2010-07-09"), "description": "One black man of success does not a nation make", "followers_count": 892, "friends_count": 498, "statues_count": 81382 }, "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-05T10:00:35.000Z"), "id": 684434332493156352, "text": "I wouldn't pursues someone 14 had I known and they told me is what I'm saying https://t.co/UvZKq1mFJW", "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": ".CraiG.", "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": 2024, "friends_count": 1276, "statues_count": 117366 }, "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-05T10:00:35.000Z"), "id": 684434332560146432, "text": "@Leanaa_Trainor love you more���� i better see you soon", "in_reply_to_status": 684432179615547394, "in_reply_to_user": 831881466, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 831881466 }}, "user": { "id": 739368733, "name": "meg", "screen_name": "Megan_henninger", "lang": "en", "location": "a$$ on htx", "create_at": date("2012-08-05"), "description": "She was unstoppable and took anything she wanted with a smile", "followers_count": 113, "friends_count": 229, "statues_count": 774 }, "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-05T10:00:35.000Z"), "id": 684434332929388544, "text": "I aint gnnna front ! �� https://t.co/kbi6RgQxFj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 167137988, "name": "#FastLife G.R.M.", "screen_name": "2G_GucciGuWop", "lang": "en", "location": "At Kelly's ", "create_at": date("2010-07-15"), "description": "On The Eastside Of The Map But I Rep The WEST !! 07631 & Its Tatted On Me #6FT⛽️ IG:Regularboyrah", "followers_count": 1420, "friends_count": 2534, "statues_count": 119029 }, "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-01-05T10:00:35.000Z"), "id": 684434333755518977, "text": "Join the South Georgia Medical Center team! See our latest #Nursing #job opening here: https://t.co/hoLGcKZT76 #RN #Valdosta, GA #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.2863,30.8625"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "RN", "Valdosta", "Hiring" }}, "user": { "id": 2842711225, "name": "SGMC Careers", "screen_name": "SGMCCareers", "lang": "en", "location": "Valdosta, Georgia", "create_at": date("2014-10-06"), "description": "SGMC offers employees a warm and welcoming environment in a rapidly growing, dynamic organization. Follow us to learn more about our openings!", "followers_count": 65, "friends_count": 33, "statues_count": 376 }, "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-01-05T10:00:36.000Z"), "id": 684434333776633856, "text": "This day is going by so SLOWWWWWWWW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.6431135,27.8560679"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 616422131, "name": "Analiese Pizaña ⚾", "screen_name": "AnaPizana20", "lang": "en", "location": "Texas", "create_at": date("2012-06-23"), "description": "I'm lit", "followers_count": 932, "friends_count": 2115, "statues_count": 8943 }, "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-05T10:00:36.000Z"), "id": 684434333831004161, "text": "������ https://t.co/2NklBvdBPo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 36154305, "name": "Nette", "screen_name": "spoiled_duh", "lang": "en", "location": "SFASU, TX ", "create_at": date("2009-04-28"), "description": "Psalms 46:5", "followers_count": 688, "friends_count": 703, "statues_count": 16311 }, "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-05T10:00:36.000Z"), "id": 684434333835247616, "text": "@claireeritchher ����", "in_reply_to_status": 684432705900040192, "in_reply_to_user": 167659738, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 167659738 }}, "user": { "id": 423492259, "name": "Brittany", "screen_name": "britt_irvv", "lang": "en", "location": "Nacogdoches, TX", "create_at": date("2011-11-28"), "description": "Proverbs 3:15 SFASU VP of Recruitment #GoGreek", "followers_count": 817, "friends_count": 904, "statues_count": 32641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:36.000Z"), "id": 684434333860364288, "text": "so sean payton choosing between niners and saints...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2344371577, "name": "#GoffTime(ps4)", "screen_name": "Ninernation53", "lang": "en", "location": "null", "create_at": date("2014-02-14"), "description": "kobe, warriors, niners and giants fan vouches/legit proof https://twitter.com/Ninernation53/status/587398635999997952", "followers_count": 858, "friends_count": 257, "statues_count": 58198 }, "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-05T10:00:36.000Z"), "id": 684434334145708032, "text": "Graaaatata https://t.co/EHumQZ95cs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 48928949, "name": "Fiona Glenanne", "screen_name": "AyooooRachel", "lang": "en", "location": "null", "create_at": date("2009-06-19"), "description": "#JustWaitOnIt | UCF'17", "followers_count": 1559, "friends_count": 1040, "statues_count": 82380 }, "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-01-05T10:00:36.000Z"), "id": 684434334233694208, "text": "DETECTIVE CONAN IS 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": 797465712, "name": "アイバン", "screen_name": "_WolfFangFist", "lang": "en", "location": "null", "create_at": date("2012-09-01"), "description": "Equivalent exchange | KCG", "followers_count": 430, "friends_count": 383, "statues_count": 33847 }, "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-05T10:00:36.000Z"), "id": 684434334254673921, "text": "My quick thoughts on 10,000-character tweets: Too much. (Note: Idea communicated in less than 140 characters.)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19871965, "name": "Dan Tennant", "screen_name": "DanTennant", "lang": "en", "location": "Ogden, UT", "create_at": date("2009-02-01"), "description": "I'm the mobile editor at Top Ten Reviews (and good grief, my tweets are my own). Daphinicus online, grew up in Hong Kong. Also Baha'i! Howdy, everybody!", "followers_count": 339, "friends_count": 363, "statues_count": 5825 }, "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-05T10:00:36.000Z"), "id": 684434334309154816, "text": "@JoseySurfin21 Bitch Please! U never even hit us up. U sound like someone else I know. lmao", "in_reply_to_status": 684434143967498240, "in_reply_to_user": 347645308, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 347645308 }}, "user": { "id": 1038568255, "name": "Claudia Candia", "screen_name": "CandiaC87", "lang": "en", "location": "CALI ", "create_at": date("2012-12-26"), "description": "Love yourself like Kanye loves Kanye", "followers_count": 26, "friends_count": 88, "statues_count": 162 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:36.000Z"), "id": 684434334363693056, "text": "https://t.co/WMftc6N8MW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3266695,35.4473422"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2445654229, "name": "RE/MAX Crossroads", "screen_name": "lincolntonremax", "lang": "en", "location": "Lincolnton NC", "create_at": date("2014-04-15"), "description": "null", "followers_count": 71, "friends_count": 43, "statues_count": 6135 }, "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.32187,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37045, "countyName": "Cleveland" } }
+{ "create_at": datetime("2016-01-05T10:00:36.000Z"), "id": 684434334422417408, "text": "Join the Community Health Network team! See our latest #Healthcare #job opening here: https://t.co/OuplPecfOq #Indianapolis, IN #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.1580423,39.7683765"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "Indianapolis", "Hiring" }}, "user": { "id": 3300641331, "name": "Community Health", "screen_name": "WorkAtCHNw", "lang": "en", "location": "null", "create_at": date("2015-05-27"), "description": "Explore current job opportunities at @CHNw, Central Indiana’s leader in providing exceptional health services at over 200 sites of care.", "followers_count": 75, "friends_count": 31, "statues_count": 828 }, "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-01-05T10:00:36.000Z"), "id": 684434334439337984, "text": "Can we just not go coordinator.. Please @Browns, try to go someone who is experienced at the position..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 40358743 }}, "user": { "id": 379343781, "name": "Travis Clark", "screen_name": "Clarknado_15", "lang": "en", "location": "Bluffton, OH", "create_at": date("2011-09-24"), "description": "Student Athlete at Bluffton University #Baseball #BUClassOf2016", "followers_count": 733, "friends_count": 582, "statues_count": 35585 }, "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-05T10:00:36.000Z"), "id": 684434334485319681, "text": "Want to work at IHOP ACG Texas? We're #hiring in #Pasadena, TX! Click for details: https://t.co/YJe0otftt3 #Hospitality #IHOP #htx #Job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.1553944,29.666862"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Pasadena", "Hospitality", "IHOP", "htx", "Job" }}, "user": { "id": 216428496, "name": "IHOP_ACG", "screen_name": "IHOP_ACGTexas", "lang": "en", "location": "Richardson, TX", "create_at": date("2010-11-16"), "description": "ACG Texas is proud to be Texas' largest #IHOP Franchisee. #Hospitality #jobs in 66 restaurants in Texas. Join our Team!", "followers_count": 308, "friends_count": 619, "statues_count": 713 }, "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-01-05T10:00:36.000Z"), "id": 684434334653091840, "text": "I've been up since like 7:30��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 239639909, "name": "aikkinn", "screen_name": "YourGFroleModel", "lang": "en", "location": "Hollywood, CA", "create_at": date("2011-01-17"), "description": "DMV to L.A", "followers_count": 4516, "friends_count": 732, "statues_count": 111175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-05T10:00:36.000Z"), "id": 684434334753796097, "text": "Mystery Fiction Author Katy Munger Selected as 2016 Piedmont Laureate https://t.co/bqKQfquoAq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.6406859,35.77414893"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21774716, "name": "Marti Hampton", "screen_name": "Martihampton", "lang": "en", "location": "Raleigh, NC", "create_at": date("2009-02-24"), "description": "With over 37 years of experience in the Triangle real estate market, you can trust the Expert... #1 RE/MAX Office in The Carolinas... http://t.co/PENpIQvXLj", "followers_count": 671, "friends_count": 241, "statues_count": 4984 }, "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-05T10:00:36.000Z"), "id": 684434334846169089, "text": "My stomach been making noise since like 4th period", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3528205936, "name": "01 ~ 30th", "screen_name": "JeremySantos98", "lang": "en", "location": "null", "create_at": date("2015-09-02"), "description": "Baseball is what I'm about ⚾️", "followers_count": 310, "friends_count": 283, "statues_count": 1532 }, "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-05T10:00:36.000Z"), "id": 684434334955089922, "text": "gustavo �� https://t.co/Jk7uoBKL3o", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2279744022, "name": "mrscurry", "screen_name": "tiffdagoat", "lang": "en", "location": "null", "create_at": date("2014-01-06"), "description": "con mi novio gilly ❤️", "followers_count": 972, "friends_count": 487, "statues_count": 54721 }, "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-05T10:00:36.000Z"), "id": 684434335391428608, "text": "@Jake_Trotter Thank you for your extremely well balanced and thorough reporting of the Big 12 this year. The @TCUFootball nation values you.", "in_reply_to_status": 684428830006640641, "in_reply_to_user": 28584781, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28584781, 454348100 }}, "user": { "id": 386814274, "name": "Brian Zempel", "screen_name": "BrianZempel", "lang": "en", "location": "null", "create_at": date("2011-10-07"), "description": "Horned Frog, Blackhawk dynasty & world champion, Avid Golfer. --Use your powers for good, not evil.", "followers_count": 74, "friends_count": 55, "statues_count": 862 }, "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-05T10:00:36.000Z"), "id": 684434335575846912, "text": "But, of course, Trump supporters wouldn't know or appreciate the difference, and Trump is counting on that. @realDonaldTrump border video", "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": 23345560, "name": "Mikey M", "screen_name": "frelling_cute", "lang": "en", "location": "S.F Bay Area, CA.", "create_at": date("2009-03-08"), "description": "Geeky lover of Movies, TV+Sci Fi+Superheroes. An advocate for gay rights and harsh critic of recent Politics.", "followers_count": 1229, "friends_count": 2012, "statues_count": 71746 }, "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-05T10:00:36.000Z"), "id": 684434335701831681, "text": "@TanBiitches @trace_templeton", "in_reply_to_status": 683714434246782976, "in_reply_to_user": 276593593, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 276593593, 992191603 }}, "user": { "id": 750163723, "name": "Alejandra Marin", "screen_name": "allejandra_m", "lang": "en", "location": "null", "create_at": date("2012-08-10"), "description": "null", "followers_count": 797, "friends_count": 350, "statues_count": 23614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University Gardens, NY", "id": "c6d77d8e185afc2b", "name": "University Gardens", "place_type": "city", "bounding_box": rectangle("-73.739505,40.764839 -73.71591,40.783021") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3676287, "cityName": "University Gardens" } }
+{ "create_at": datetime("2016-01-05T10:00:36.000Z"), "id": 684434335827668992, "text": "CREATIVITY IS ALMOST ENTIRELY DEPENDENT ON YOUR EMOTIONS\nhttps://t.co/FnHnzOF3Ac\n\nhttps://t.co/LybCj5j9c7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20207761, "name": "Grey Cross Studios", "screen_name": "GreyCrossStudio", "lang": "en", "location": "New Orleans, Louisiana", "create_at": date("2009-02-05"), "description": "Assimilation Artist, Sculptor, Art Activist & Photographer. Owner LGBT Emerging Artists Conservatory", "followers_count": 12993, "friends_count": 12946, "statues_count": 18956 }, "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-05T10:00:36.000Z"), "id": 684434336234385408, "text": "Supplemental Health Care: NP II - Community Medicine (#Dallas, TX) https://t.co/eh3K7NQLzu #Healthcare #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.8511814,32.8296623"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Dallas", "Healthcare", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 20831777, "name": "TMJ- DFW Health Jobs", "screen_name": "tmj_dfw_health", "lang": "en", "location": "DFW, Texas", "create_at": date("2009-02-13"), "description": "Follow this account for geo-targeted Healthcare job tweets in DFW, Texas. Need help? Tweet us at @CareerArc!", "followers_count": 501, "friends_count": 306, "statues_count": 1481 }, "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-05T10:00:36.000Z"), "id": 684434336800727040, "text": "@SnakeTaleJohnny @Ammon_Bundy YOURE LOOKIN LIVE", "in_reply_to_status": 684434158781886464, "in_reply_to_user": 213807427, "favorite_count": 0, "coordinate": point("-86.79852979,33.49266315"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 213807427, 4701504438 }}, "user": { "id": 22869133, "name": "Dunc City", "screen_name": "whduncan", "lang": "en", "location": "Birmingham, AL", "create_at": date("2009-03-04"), "description": "Tell your dad I said hey.", "followers_count": 1606, "friends_count": 1017, "statues_count": 103312 }, "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-05T10:00:36.000Z"), "id": 684434336821719040, "text": "For questions about online V1 app golf lessons email me at Jim@JimSuttie.com", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 252144304, "name": "Dr Jim Suttie", "screen_name": "docsuttie", "lang": "en", "location": "1-800-765-3838", "create_at": date("2011-02-14"), "description": "Dr. Suttie is one of the world's best and most-respected golf instructors. The TwinEagles Club, Naples FL", "followers_count": 1077, "friends_count": 194, "statues_count": 233 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Naples, FL", "id": "a3f971b69fefb699", "name": "Naples", "place_type": "city", "bounding_box": rectangle("-81.817216,26.093538 -81.766738,26.211277") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12021, "countyName": "Collier", "cityID": 1247625, "cityName": "Naples" } }
+{ "create_at": datetime("2016-01-05T10:00:36.000Z"), "id": 684434336846757888, "text": "If you're a #CustomerService professional in #Peoria, AZ, check out this #job: https://t.co/AuaV2PNtMC #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.2373779,33.5805955"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CustomerService", "Peoria", "job", "Hiring" }}, "user": { "id": 71943150, "name": "TMJ-PHX CstSrv Jobs", "screen_name": "tmj_phx_cstsrv", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2009-09-05"), "description": "Follow this account for geo-targeted Customer Service job tweets in Phoenix, AZ. Need help? Tweet us at @CareerArc!", "followers_count": 362, "friends_count": 281, "statues_count": 309 }, "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": 427820, "cityName": "Glendale" } }
+{ "create_at": datetime("2016-01-05T10:00:36.000Z"), "id": 684434337018806274, "text": "Feliz dia de los Reyes magos Pueryo Rico...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 147280489, "name": "jorge Gutierrez", "screen_name": "reydelfuegobpa", "lang": "en", "location": "Bronx,N.Y.", "create_at": date("2010-05-23"), "description": "Soy amigo de mis amigos,y aun de mis enemigos... Justo parece quien primero aboga por su causa,mas: Luego viene un testigo quien le descubre...", "followers_count": 84, "friends_count": 380, "statues_count": 860 }, "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-05T10:00:36.000Z"), "id": 684434337056473089, "text": "Want to work at GEICO? We're #hiring in #VirginiaBeach, Virginia! Click for details: https://t.co/YkU0CkXdch #Insurance #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.977985,36.8529263"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "VirginiaBeach", "Insurance", "Job", "Jobs" }}, "user": { "id": 201348710, "name": "GEICO Jobs", "screen_name": "GEICO_Jobs", "lang": "en", "location": "Across the U.S.", "create_at": date("2010-10-11"), "description": "#GEICO is stable, growing & #hiring in Sales, CustServ, IT, Legal, Claims & more! This is our #jobs feed. Follow @GEICOCareers or @GEICOCampus to connect.", "followers_count": 458, "friends_count": 96, "statues_count": 2227 }, "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-05T10:00:36.000Z"), "id": 684434337165643777, "text": "need some attention", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27193478, "name": "theTalent", "screen_name": "Cody_Montana", "lang": "en", "location": "єтσωαн тєnnєѕѕєє", "create_at": date("2009-03-27"), "description": "GOD | B.M.C. | Music", "followers_count": 525, "friends_count": 252, "statues_count": 4640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Etowah, TN", "id": "63e5ee1fce041f1d", "name": "Etowah", "place_type": "city", "bounding_box": rectangle("-84.544416,35.304216 -84.505208,35.385812") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47107, "countyName": "McMinn", "cityID": 4724480, "cityName": "Etowah" } }
+{ "create_at": datetime("2016-01-05T10:00:36.000Z"), "id": 684434337278898176, "text": "@NickMillerOU tru. Those are p good. I have nothing interesting about myself to say though.", "in_reply_to_status": 684434137923633153, "in_reply_to_user": 3534930617, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3534930617 }}, "user": { "id": 212040747, "name": "sad dad club", "screen_name": "gettingspoopy", "lang": "en", "location": "Detroit, MI", "create_at": date("2010-11-04"), "description": "Sad and stupid", "followers_count": 134, "friends_count": 216, "statues_count": 2840 }, "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-01-05T10:00:36.000Z"), "id": 684434337308225536, "text": "AHA LOOKS LIKE THEY DO BETCH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 115507507, "name": "Supa Vic. ➳", "screen_name": "l0stmermaid_", "lang": "en", "location": "dark side of the moon ", "create_at": date("2010-02-18"), "description": "⚯͛☮☪ॐ➳ // #1017 // SAV", "followers_count": 1288, "friends_count": 372, "statues_count": 70511 }, "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-05T10:00:36.000Z"), "id": 684434337429716992, "text": "deactivate your twitter nate https://t.co/kGtSpXkLsc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 555641713, "name": "dyl", "screen_name": "dylanceeee", "lang": "en", "location": "seattle//corvallis", "create_at": date("2012-04-16"), "description": "luv music luv the PNW || SUCKER MAGAZINE", "followers_count": 493, "friends_count": 821, "statues_count": 11740 }, "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-01-05T10:00:36.000Z"), "id": 684434337454931968, "text": "Note to self: If they don't give you credits for being like them, separate yourself and highlight your differences. Shine.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3257116784, "name": "Ashura B. Lovelady", "screen_name": "ABLovelady", "lang": "en", "location": "in the mud.", "create_at": date("2015-06-26"), "description": "Blood Type A-Positive", "followers_count": 141, "friends_count": 553, "statues_count": 1030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:36.000Z"), "id": 684434337467506688, "text": "I hope we got new gurbs next semester �� Hawthorne should never b that drub", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4709941333, "name": "Cae", "screen_name": "caefuccbitches", "lang": "en", "location": "null", "create_at": date("2016-01-04"), "description": "you snooze you lose ☄", "followers_count": 46, "friends_count": 64, "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-01-05T10:00:36.000Z"), "id": 684434337530556416, "text": "@kylie_giedemann @kellygiedemann happy birthday ������", "in_reply_to_status": -1, "in_reply_to_user": 546451672, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 546451672, 546466673 }}, "user": { "id": 2259168093, "name": "riley deitz", "screen_name": "rileydeitz1", "lang": "en", "location": "south jersey", "create_at": date("2014-01-02"), "description": "null", "followers_count": 404, "friends_count": 542, "statues_count": 3633 }, "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-01-05T10:00:36.000Z"), "id": 684434337681530880, "text": "I'm at J Alexander's Restaurant in Baton Rouge, LA https://t.co/EDxJkinWG2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-91.09030724,30.39175629"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 14622786, "name": "Josh Jackson", "screen_name": "audrum1", "lang": "en", "location": "Baton Rouge, LA", "create_at": date("2008-05-01"), "description": "Auburn grad, Social Media/Digital Media specialist, Musician. My opinions are my own and do not represent anyone else.", "followers_count": 189, "friends_count": 755, "statues_count": 3912 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inniswold, LA", "id": "fbcefe96d630085b", "name": "Inniswold", "place_type": "city", "bounding_box": rectangle("-91.092831,30.363623 -91.040089,30.412138") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2237270, "cityName": "Inniswold" } }
+{ "create_at": datetime("2016-01-05T10:00:36.000Z"), "id": 684434337740144640, "text": "@LokiSnax Nay, my friend! 'Tis midday! #feastyoureyes https://t.co/KpWWELNVfO", "in_reply_to_status": 684433184814010368, "in_reply_to_user": 210643313, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "feastyoureyes" }}, "user_mentions": {{ 210643313 }}, "user": { "id": 2321004153, "name": "Jen•R•Tist", "screen_name": "JenRutherford06", "lang": "en", "location": "Texas, USA, Midgard.", "create_at": date("2014-02-02"), "description": "Heimdall?! Open the bifrost! -Me, every single day.", "followers_count": 597, "friends_count": 279, "statues_count": 10827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:36.000Z"), "id": 684434337782087683, "text": "MultiCare Health System #Nursing #Job: Staff RN - Allenmore Hospital ED (#Tacoma, WA) https://t.co/TAiFPY2l7N #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.4442906,47.2528768"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "Job", "Tacoma", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 120647454, "name": "TMJ-WAT Nursing Jobs", "screen_name": "tmj_WAT_NURSING", "lang": "en", "location": "Tacoma, WA", "create_at": date("2010-03-06"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Tacoma, WA. Need help? Tweet us at @CareerArc!", "followers_count": 402, "friends_count": 282, "statues_count": 361 }, "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-05T10:00:37.000Z"), "id": 684434338419613696, "text": "#Tualatin, OR #SupplyChain #Job: Warehouse Loader (Lead) at DPI Specialty Foods https://t.co/AbEPxYRYqB #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.7639851,45.3840077"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Tualatin", "SupplyChain", "Job", "Jobs", "Hiring" }}, "user": { "id": 3143485394, "name": "DPI Careers", "screen_name": "DPICareers", "lang": "en", "location": "Nationwide", "create_at": date("2015-04-07"), "description": "We are on an exciting growth journey and need exceptional talent to deliver our vision. Come and join the winning team, apply today!", "followers_count": 8, "friends_count": 0, "statues_count": 54 }, "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-01-05T10:00:37.000Z"), "id": 684434338683973633, "text": "Keep it 8 mo then 92, ��", "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": "Ivy", "screen_name": "xcallmepoison", "lang": "en", "location": "256 ✈️ 404 Spelman College", "create_at": date("2011-05-26"), "description": "18. my DMs are closed.", "followers_count": 2466, "friends_count": 2381, "statues_count": 17159 }, "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-01-05T10:00:37.000Z"), "id": 684434338881081344, "text": "I'm lying in bed with shin splints. I feel worthless. How do people do this 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": 20995940, "name": "Meredith L Stoever", "screen_name": "mlstoever", "lang": "en", "location": "ÜT: 38.906289,-77.05816", "create_at": date("2009-02-16"), "description": "Triathlete, Nutrition and Anthropology obsessed, Terp, blogger, and zombie killer...", "followers_count": 84, "friends_count": 90, "statues_count": 1717 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berwyn Heights, MD", "id": "3ebbdd99701b0c90", "name": "Berwyn Heights", "place_type": "city", "bounding_box": rectangle("-76.923112,38.983058 -76.898339,38.998739") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2406925, "cityName": "Berwyn Heights" } }
+{ "create_at": datetime("2016-01-05T10:00:37.000Z"), "id": 684434339195654145, "text": "Lolz @ all the idiots saying Obama wants to take away our guns. NO. If you're mentally fit &not a criminal you'll have no prob buying a gun.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14343691, "name": "Barefoot Seamstress", "screen_name": "shayycotts", "lang": "en", "location": "New York", "create_at": date("2008-04-09"), "description": "Sic transit gloria. Glory fades. I'm Shay Cotts. Pins&needles, threads&thimbles. Fashion Design BFA @ FIT. Art History MA @ SUNY Purchase. NYC/Westchester.", "followers_count": 593, "friends_count": 355, "statues_count": 29119 }, "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-01-05T10:00:37.000Z"), "id": 684434339275403264, "text": "I have no friends lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 110293196, "name": "1017", "screen_name": "CANDYCIROC", "lang": "en", "location": "Mount Healthy Heights, OH", "create_at": date("2010-01-31"), "description": "Neosia.", "followers_count": 7372, "friends_count": 6303, "statues_count": 77071 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Healthy Heights, OH", "id": "7de09bdd995e495b", "name": "Mount Healthy Heights", "place_type": "city", "bounding_box": rectangle("-84.574432,39.257815 -84.564011,39.285236") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3952780, "cityName": "Mount Healthy Heights" } }
+{ "create_at": datetime("2016-01-05T10:00:37.000Z"), "id": 684434339384311808, "text": "@bosco_the_boss @addie_wolff @gentry_autumn leave her alone part 3", "in_reply_to_status": 680880330044084224, "in_reply_to_user": 3181986468, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3181986468, 2327119091, 2200241876 }}, "user": { "id": 1573184922, "name": "k", "screen_name": "katieephann_", "lang": "en", "location": "tx", "create_at": date("2013-07-06"), "description": "jalens ugly", "followers_count": 285, "friends_count": 199, "statues_count": 462 }, "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-05T10:00:37.000Z"), "id": 684434339711430656, "text": "@AndreaTantaros Poor ol' Bill just ain't the bad boy he once was. Wrinkled..voice crackling..grey as a badger.go home dude!!\n\n@cbinflux", "in_reply_to_status": 684368676695486464, "in_reply_to_user": 36452050, "favorite_count": 0, "coordinate": point("-116.037706,36.2240357"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 36452050, 329688737 }}, "user": { "id": 2821285640, "name": "Tina Trenner", "screen_name": "rowdy_tt", "lang": "en", "location": "NV", "create_at": date("2014-09-20"), "description": "Constitutional Freedom loving Western born woman. Horse n dogs fresh air rural life. Broadcaster n outspoken political conservative.", "followers_count": 3227, "friends_count": 3389, "statues_count": 27772 }, "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-05T10:00:37.000Z"), "id": 684434339728232448, "text": "COMPREHENSIVE DEVICES...“TESSKET WITH TASSKET” from the album 'ACE of SPADE' https://t.co/I6u7ErsGMo https://t.co/MwjoHwdcKz", "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": 1179, "friends_count": 5002, "statues_count": 116403 }, "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-05T10:00:37.000Z"), "id": 684434339854073857, "text": "@kellymoconnell I LOOSE 2 OF MY FAVORITES IN A FEW MONTHS HATE IT, @bblakelyQ13Fox UR GONNA BE MISSED SORRY TO SEE YOU GO @kellymoconnell", "in_reply_to_status": -1, "in_reply_to_user": 69001087, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 69001087, 718827319, 69001087 }}, "user": { "id": 952806320, "name": "Frank Addicks", "screen_name": "freeotz57", "lang": "en", "location": "NISQUALLY, WASHINGTON", "create_at": date("2012-11-16"), "description": "DISABLED US AIR FORCE VETERAN, CAMPAIGNING TO END PROHIBITION, SOUNDERS, SEAHAWKS, MARINERS Baltimore Born Seattle Transplant", "followers_count": 189, "friends_count": 464, "statues_count": 984 }, "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-05T10:00:37.000Z"), "id": 684434339933765632, "text": "Can you recommend anyone for this #job? Web Designer - https://t.co/J2OIBU5rGm #holidayjobs #BocaRaton, FL #Retail #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.0830984,26.3586885"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "holidayjobs", "BocaRaton", "Retail", "Hiring" }}, "user": { "id": 2798260664, "name": "Boston Proper Jobs", "screen_name": "BostonProperJob", "lang": "en", "location": "null", "create_at": date("2014-09-08"), "description": "Boston Proper provides fashions for women who are fearlessly feminine, enviably chic and who possess the poise and confidence to “wear it like no one else“.", "followers_count": 30, "friends_count": 0, "statues_count": 460 }, "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-05T10:00:37.000Z"), "id": 684434339933908992, "text": "I truly & endlessly love my mom & dad . absolutely no idea where I'd be without them both .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2511393561, "name": "alexis buccellato", "screen_name": "lexx__xo", "lang": "en", "location": "ig : lexxmarieee_", "create_at": date("2014-04-26"), "description": "babygirl so blessed , so loved", "followers_count": 474, "friends_count": 516, "statues_count": 3197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Middlesex, NJ", "id": "9e9d157857a4205c", "name": "Middlesex", "place_type": "city", "bounding_box": rectangle("-74.527044,40.555399 -74.469693,40.59266") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34035, "countyName": "Somerset", "cityID": 3445900, "cityName": "Middlesex" } }
+{ "create_at": datetime("2016-01-05T10:00:37.000Z"), "id": 684434340034523136, "text": "i haven't heard one song from her that i didn't like", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3249519047, "name": "petty flacko", "screen_name": "pettyprincesa", "lang": "en", "location": "null", "create_at": date("2015-05-12"), "description": "woah", "followers_count": 117, "friends_count": 81, "statues_count": 1092 }, "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-05T10:00:37.000Z"), "id": 684434340139397120, "text": "Does age matter 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": 2775038619, "name": "IG: thebamster__", "screen_name": "TheBamster_", "lang": "en", "location": "DET ✈️ D(M)V", "create_at": date("2014-09-19"), "description": "Take a Step into My World... Aspiring Model For BOOKINGS: bookingjusten@gmail.com", "followers_count": 1613, "friends_count": 553, "statues_count": 8268 }, "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-05T10:00:37.000Z"), "id": 684434340344762368, "text": "Spring Break, wya", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2205086072, "name": "H Fen", "screen_name": "F1Fenley", "lang": "en", "location": "null", "create_at": date("2013-11-20"), "description": "fishing, or with @Maariiahh_", "followers_count": 890, "friends_count": 225, "statues_count": 21622 }, "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-05T10:00:37.000Z"), "id": 684434340495929346, "text": "@bscha4a @GiraffeAttack14 https://t.co/D00i8nBiKV", "in_reply_to_status": 684434103836536832, "in_reply_to_user": 2292089402, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2292089402, 2199770802 }}, "user": { "id": 391071169, "name": "Spartan-051", "screen_name": "DShep150", "lang": "en", "location": "Delta Halo", "create_at": date("2011-10-14"), "description": "UNSC ODST, ONI Operative \n\nFeet First Into Hell", "followers_count": 541, "friends_count": 476, "statues_count": 58137 }, "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-05T10:00:37.000Z"), "id": 684434341137526784, "text": "teen wolf is back tonight ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "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": 1091, "friends_count": 861, "statues_count": 78851 }, "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-05T10:00:37.000Z"), "id": 684434341179568128, "text": "Kinda hard to trust people when you constantly get fucked over but that's part of life right?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 420546920, "name": "syd", "screen_name": "Sydniee09", "lang": "en", "location": "Buffalo, NY", "create_at": date("2011-11-24"), "description": "what feels like the end is often the beginning snapchat/insta sydniee09", "followers_count": 783, "friends_count": 328, "statues_count": 54134 }, "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-05T10:00:37.000Z"), "id": 684434341204725760, "text": "I'm at Fashion Centre at Pentagon City in Arlington, VA https://t.co/hXpCVW1d7R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.0608896,38.86313324"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 96471329, "name": "Ramon Cordero", "screen_name": "cordero_7", "lang": "en", "location": "Orlando, FL", "create_at": date("2009-12-12"), "description": "Florida Hospital Human Resources Recruiter. Currently working on my masters degree in Human Resources Management.", "followers_count": 270, "friends_count": 1022, "statues_count": 449 }, "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-05T10:00:37.000Z"), "id": 684434341305434113, "text": "Craving some sushi. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 634248934, "name": "Katlynn", "screen_name": "K_Noneaker", "lang": "en", "location": "Auburn, AL", "create_at": date("2012-07-12"), "description": "Auburn University '19 • War Damn Eagle • Kappa Alpha Theta • IG & SC: K_Noneaker", "followers_count": 300, "friends_count": 270, "statues_count": 5141 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atoka, TN", "id": "014fa6605f6fcfaf", "name": "Atoka", "place_type": "city", "bounding_box": rectangle("-89.855206,35.390621 -89.74451,35.466582") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47167, "countyName": "Tipton", "cityID": 4702340, "cityName": "Atoka" } }
+{ "create_at": datetime("2016-01-05T10:00:37.000Z"), "id": 684434341343043584, "text": "@DLoesch @NRA @NRAILA #2a https://t.co/pkl9DLYbzY", "in_reply_to_status": -1, "in_reply_to_user": 7702542, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "2a" }}, "user_mentions": {{ 7702542, 21829541, 531636273 }}, "user": { "id": 461430421, "name": "Brett Loomis", "screen_name": "bloomis914", "lang": "en", "location": "Northville, Michigan", "create_at": date("2012-01-11"), "description": "Proud Husband to One, Father of Two and Coach of Many. Pushing my luck since 1968. MTB, baseball, hockey, golf and #2A supporter.", "followers_count": 308, "friends_count": 1325, "statues_count": 937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madison Heights, MI", "id": "e67427d9b4126602", "name": "Madison Heights", "place_type": "city", "bounding_box": rectangle("-83.126332,42.475983 -83.084518,42.534826") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2650560, "cityName": "Madison Heights" } }
+{ "create_at": datetime("2016-01-05T10:00:37.000Z"), "id": 684434341359824896, "text": "@KuzeyLiL profil foton şahane .sakın değiştirme bana bakıyosun gibi hissediyprum", "in_reply_to_status": -1, "in_reply_to_user": 120148307, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "user_mentions": {{ 120148307 }}, "user": { "id": 198411791, "name": "melike", "screen_name": "pek_simet", "lang": "en", "location": "California, USA", "create_at": date("2010-10-03"), "description": "3 canımdan can parçası, ninemden kalan bikaç fincanla bir kaç da cam parçası ortasından fay hattı gecen bir arsa, memleket gemlik bursa", "followers_count": 5323, "friends_count": 954, "statues_count": 39740 }, "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-05T10:00:37.000Z"), "id": 684434341406093312, "text": "There is no time like the freezing cold present to sign up for your Winter Dining Plan! https://t.co/ITWHaK5ux3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 498258667, "name": "Drexel Campus Dining", "screen_name": "DrexelDining", "lang": "en", "location": "Drexel University Campus", "create_at": date("2012-02-20"), "description": "The Twitter for all things Drexel Campus Dining! We're here for you-get to know Drexel Campus Dining! #DCD #Hans", "followers_count": 652, "friends_count": 1425, "statues_count": 932 }, "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-05T10:00:37.000Z"), "id": 684434341544366080, "text": "idc if you're rich or poor, black or white, fat or skinny, ugly or not, give me respect and I will give you the same in return", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 259417446, "name": "Logan Lair", "screen_name": "loganlair", "lang": "en", "location": "i am second", "create_at": date("2011-03-01"), "description": "We grew up cheering on the underdogs because it was within them that we saw ourselves | snapchat : itsloganlair", "followers_count": 1152, "friends_count": 400, "statues_count": 16204 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wolfforth, TX", "id": "017e37957dac1dd8", "name": "Wolfforth", "place_type": "city", "bounding_box": rectangle("-102.026275,33.4907 -101.991693,33.54877") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4879972, "cityName": "Wolfforth" } }
+{ "create_at": datetime("2016-01-05T10:00:38.000Z"), "id": 684434342282719232, "text": "@_chiefkeev ��", "in_reply_to_status": 684432757783564288, "in_reply_to_user": 308630774, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 308630774 }}, "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": 1794, "friends_count": 1762, "statues_count": 89756 }, "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-05T10:00:38.000Z"), "id": 684434342303563780, "text": "dead, baby .. omg .. Ima just let you be https://t.co/BwBWEzoxdw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3092875879, "name": "01/16", "screen_name": "irianazjyne", "lang": "en", "location": "houston", "create_at": date("2015-03-17"), "description": "null", "followers_count": 970, "friends_count": 443, "statues_count": 1569 }, "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-01-05T10:00:38.000Z"), "id": 684434342387384320, "text": "#Transportation #Job alert: CDL-A Driver Recent... | Watkins and Shepard | #Conover, NC https://t.co/O46chC7xzC https://t.co/FN6mTzSWN1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2186933,35.7065217"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Transportation", "Job", "Conover" }}, "user": { "id": 2841515944, "name": "WKSH Jobs", "screen_name": "JoinWKSH", "lang": "en", "location": "null", "create_at": date("2014-10-22"), "description": "Great jobs for great drivers We are looking for talented Owner Operators and company drivers. Get the facts and get on the road with Watkins Shepard.", "followers_count": 72, "friends_count": 1, "statues_count": 276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conover, NC", "id": "4380b04f6eef3d09", "name": "Conover", "place_type": "city", "bounding_box": rectangle("-81.267049,35.682082 -81.159804,35.747116") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37035, "countyName": "Catawba", "cityID": 3714340, "cityName": "Conover" } }
+{ "create_at": datetime("2016-01-05T10:00:38.000Z"), "id": 684434342689378306, "text": "Having a room right next to my parents ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2683735783, "name": "Asanti Mokwala", "screen_name": "asanti25", "lang": "en", "location": "DBU", "create_at": date("2014-07-26"), "description": "God first•2 Sport Athlete•chis•c/o2020•DB•", "followers_count": 248, "friends_count": 148, "statues_count": 1519 }, "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-05T10:00:38.000Z"), "id": 684434342752354305, "text": "Why are you brown and fish at the same time https://t.co/rvZXSNs3Vf", "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": 538, "friends_count": 411, "statues_count": 18175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-05T10:00:38.000Z"), "id": 684434342844694528, "text": "Go outside to find your dog has eaten a pair of you shoes for breakfast.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 85025671, "name": "Michael Guilford", "screen_name": "michaelguilford", "lang": "en", "location": "Columbus Ohio ", "create_at": date("2009-10-24"), "description": "null", "followers_count": 445, "friends_count": 240, "statues_count": 2819 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hilliard, OH", "id": "903bac8876c50289", "name": "Hilliard", "place_type": "city", "bounding_box": rectangle("-83.189851,40.004114 -83.096934,40.077511") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3935476, "cityName": "Hilliard" } }
+{ "create_at": datetime("2016-01-05T10:00:38.000Z"), "id": 684434343226261504, "text": "@ATTCares I have", "in_reply_to_status": 684432737223180288, "in_reply_to_user": 62643312, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 62643312 }}, "user": { "id": 52816701, "name": "Stephen Salyers", "screen_name": "Stephen_Salyers", "lang": "en", "location": "ÜT: 34.088656,-118.371366", "create_at": date("2009-07-01"), "description": "Celebrity Makeup Artist & Celebrity Groomer", "followers_count": 178, "friends_count": 183, "statues_count": 6847 }, "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-05T10:00:38.000Z"), "id": 684434343234826244, "text": "One thing about them tables the Do Turn ✊��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3313409627, "name": "NERA!", "screen_name": "Neraaa4", "lang": "en", "location": "null", "create_at": date("2015-06-08"), "description": "life's what you make it", "followers_count": 247, "friends_count": 167, "statues_count": 10984 }, "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-05T10:00:38.000Z"), "id": 684434343301742592, "text": "The shitty thing about growing up is that you no longer get any stickers or goodies in your dentist bag anymore.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 343302996, "name": "emily", "screen_name": "the__remedy", "lang": "en", "location": "California | Nevada", "create_at": date("2011-07-27"), "description": "these things happen", "followers_count": 236, "friends_count": 392, "statues_count": 4539 }, "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-05T10:00:38.000Z"), "id": 684434343377244160, "text": "Hope 4th better than 3rd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2938991510, "name": "#FreeBandz", "screen_name": "chulo_av8", "lang": "en", "location": "null", "create_at": date("2014-12-21"), "description": "#RestPops❤️...", "followers_count": 400, "friends_count": 176, "statues_count": 4992 }, "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-05T10:00:38.000Z"), "id": 684434343394148353, "text": "If all signs point ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 270701494, "name": "YoungOG", "screen_name": "BvnksLikeAshley", "lang": "en", "location": "en la casa de tu mai ", "create_at": date("2011-03-22"), "description": "21. Miami. An avid food lover, tv enthusiast, & a connoisseur of all music. IG:BvnksLikeAshley", "followers_count": 345, "friends_count": 262, "statues_count": 27284 }, "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-01-05T10:00:38.000Z"), "id": 684434343431802880, "text": "This #BusinessMgmt #job might be a great fit for you: District Manager Trainee-Retail Sales - https://t.co/IYCDGmeIgE #Algood, TN #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.4485849,36.1958946"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BusinessMgmt", "job", "Algood", "Hiring" }}, "user": { "id": 22363501, "name": "TMJ-TN Mgmt. Jobs", "screen_name": "tmj_tn_mgmt", "lang": "en", "location": "Tennessee", "create_at": date("2009-03-01"), "description": "Follow this account for geo-targeted Business/Mgmt. job tweets in Tennessee Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 364, "friends_count": 242, "statues_count": 116 }, "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-01-06T10:00:00.000Z"), "id": 684796571423227905, "text": "Golly gee wilikers, it's 28ºF and fair. #MorrisWeather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.9,45.59"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MorrisWeather" }}, "user": { "id": 1931463259, "name": "Morris Weather", "screen_name": "MorrisMNWeather", "lang": "en", "location": "Morris, MN", "create_at": date("2013-10-03"), "description": "A place to find the current weather in Morris, MN. Run by @bman4789\n\nScheduled tweets are at 7 AM, 12 PM, 3 PM, 6 PM, and 10 PM. Daily forecast at 6 AM.", "followers_count": 22, "friends_count": 1, "statues_count": 1794 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morris, MN", "id": "dc943406a248a125", "name": "Morris", "place_type": "city", "bounding_box": rectangle("-95.933929,45.564348 -95.883709,45.607334") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27149, "countyName": "Stevens", "cityID": 2744242, "cityName": "Morris" } }
+{ "create_at": datetime("2016-01-06T10:00:00.000Z"), "id": 684796571469271040, "text": "@j0nhill ��������", "in_reply_to_status": 684791627626381312, "in_reply_to_user": 362634831, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 362634831 }}, "user": { "id": 22226639, "name": "Francis Tran", "screen_name": "frank_shits", "lang": "en", "location": "SAC - SD ", "create_at": date("2009-02-27"), "description": "Move for me, i'll move for you.", "followers_count": 249, "friends_count": 288, "statues_count": 8209 }, "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-06T10:00:00.000Z"), "id": 684796571691556864, "text": "ah ye bud", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2990344099, "name": "Taylor Noe", "screen_name": "Taylornoway", "lang": "en", "location": "Scottsbluff, NE. ", "create_at": date("2015-01-19"), "description": "insta: taylornoway, uh uh uh", "followers_count": 594, "friends_count": 420, "statues_count": 8043 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scottsbluff, NE", "id": "53d3452aa005cb99", "name": "Scottsbluff", "place_type": "city", "bounding_box": rectangle("-103.721759,41.846129 -103.626501,41.903056") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31157, "countyName": "Scotts Bluff", "cityID": 3144245, "cityName": "Scottsbluff" } }
+{ "create_at": datetime("2016-01-06T10:00:00.000Z"), "id": 684796571951566848, "text": "Que puta huevaaaaa quiero estar dormida ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2237061295, "name": "mich", "screen_name": "michmart23", "lang": "en", "location": "en tu corazon ", "create_at": date("2013-12-08"), "description": "póngase trucha", "followers_count": 573, "friends_count": 461, "statues_count": 18996 }, "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-01-06T10:00:00.000Z"), "id": 684796571980935168, "text": "He's the freaking greatest! https://t.co/IJ60KQuctm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 436763041, "name": "Briana", "screen_name": "briiwii3", "lang": "en", "location": "Sick Sad Little World ", "create_at": date("2011-12-14"), "description": "party on, wayne.", "followers_count": 982, "friends_count": 927, "statues_count": 70379 }, "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-06T10:00:00.000Z"), "id": 684796572052361216, "text": "Ripley SW Limestone Co. Temp: 46.9°F Wind:3.1mph Pressure: 1007.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": 42, "friends_count": 33, "statues_count": 47144 }, "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-06T10:00:00.000Z"), "id": 684796572274692098, "text": "❤️❤️❤️❤️❤️❤️❤️❤️❤️❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 218591102, "name": "Mم is typing...", "screen_name": "MIIXIXVIII", "lang": "en", "location": "null", "create_at": date("2010-11-22"), "description": "null", "followers_count": 1190, "friends_count": 458, "statues_count": 33027 }, "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-06T10:00:00.000Z"), "id": 684796572484374529, "text": "#elnino #storm #surferspoint #nothappening #comebackanotherday @ Surfer's Point & Ventura Pier https://t.co/6krx6WsKbS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.29780786,34.27495013"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "elnino", "storm", "surferspoint", "nothappening", "comebackanotherday" }}, "user": { "id": 51653656, "name": "WCStone Photography", "screen_name": "stoneworrior", "lang": "en", "location": "ÜT: 34.17474,-118.95499", "create_at": date("2009-06-27"), "description": "16 years U.S. Marine, currently in IUOE local 12 heavy equipment operator in Los Angeles. Tech/gadget/photography lover", "followers_count": 43, "friends_count": 93, "statues_count": 410 }, "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-01-06T10:00:00.000Z"), "id": 684796572488581120, "text": "Fun little side boob action to start the day off right �� #tattoos #colortattoos #route9tattoo… https://t.co/P1mzoiSN06", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.40462756,42.29858853"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "tattoos", "colortattoos", "route9tattoo" }}, "user": { "id": 2793338471, "name": "Thaís", "screen_name": "_thaisv", "lang": "en", "location": "ig: @thaisvtattoos | @_thaisv", "create_at": date("2014-09-29"), "description": "| tattoo artist | Route 9 Tattoos Framingham, MA | foreign born | everyone's favorite lesbian | Ca$hed | your vibe attracts your tribe ♡ |", "followers_count": 444, "friends_count": 692, "statues_count": 3838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Framingham, MA", "id": "dbd1411661903bb0", "name": "Framingham", "place_type": "city", "bounding_box": rectangle("-71.496927,42.257263 -71.374193,42.352931") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2524960, "cityName": "Framingham" } }
+{ "create_at": datetime("2016-01-06T10:00:00.000Z"), "id": 684796572564090880, "text": "@ND4Bernie @crazybenghazi04 @danmericaCNN @the_intercept Your 401K depends on WS.", "in_reply_to_status": 684796369060687877, "in_reply_to_user": 35129324, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3368690836, 2588206733, 36890646, 2329066872 }}, "user": { "id": 35129324, "name": "Lisa Arbogast Grant", "screen_name": "lisaarbogrant", "lang": "en", "location": "null", "create_at": date("2009-04-24"), "description": "It's a charitable & compassionate view to understand that not all people are the same or think the same. Especially in politics and religion.", "followers_count": 886, "friends_count": 1715, "statues_count": 4588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mayo, SC", "id": "00aab0fc1382e919", "name": "Mayo", "place_type": "city", "bounding_box": rectangle("-81.880887,35.062339 -81.834986,35.098496") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45083, "countyName": "Spartanburg", "cityID": 4545430, "cityName": "Mayo" } }
+{ "create_at": datetime("2016-01-06T10:00:00.000Z"), "id": 684796572941484033, "text": "I hope you know you're capable, brave and significant, even when it feels like you're not. ~ Unknown #quote", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "quote" }}, "user": { "id": 81968978, "name": "Joyce Hill", "screen_name": "JoyceLHill", "lang": "en", "location": "Austin", "create_at": date("2009-10-12"), "description": "Currently based in Austin. My passions are screenwriting, great food, travel, wine, music and I love Los Angeles! And can you tell that I'm a Star Trek geek?", "followers_count": 226, "friends_count": 320, "statues_count": 557 }, "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-06T10:00:00.000Z"), "id": 684796573100957696, "text": "I liked a @YouTube video from @KarianneJean https://t.co/QbnvPYTJrC \"Cactus\" Official Music Video", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 10228272, 57579719 }}, "user": { "id": 194275156, "name": "Armando Williams", "screen_name": "RaydenWilliams", "lang": "en", "location": "Portsmouth, VA", "create_at": date("2010-09-23"), "description": "An upcoming 25-year-old singer-songwriter/musician/actor/record producer who's ready 2 take on the world by storm. Arianator/Chaster/Daminator/Jillianator/etc.", "followers_count": 1993, "friends_count": 4049, "statues_count": 18372 }, "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-06T10:00:00.000Z"), "id": 684796573218377729, "text": "@Kara_Ashley17 damn yo", "in_reply_to_status": 684796397468647424, "in_reply_to_user": 503999054, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 503999054 }}, "user": { "id": 393619788, "name": "BENGALS 11-5", "screen_name": "KingVae_", "lang": "en", "location": "LC", "create_at": date("2011-10-18"), "description": "Returning future college dropout. People call me Drew but my real name is Ugly. #WhoDey", "followers_count": 610, "friends_count": 412, "statues_count": 23457 }, "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-06T10:00:00.000Z"), "id": 684796573268586497, "text": "We're #hiring! Click to apply: Residential Lawn Specialist 100889 - https://t.co/gIFDSt9mD3 #Sales #Greenville, NC #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.3663538,35.612661"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Sales", "Greenville", "Job", "Jobs", "CareerArc" }}, "user": { "id": 119115249, "name": "TMJ-NC Skl. Trades", "screen_name": "tmj_NC_skltrd", "lang": "en", "location": "North Carolina", "create_at": date("2010-03-02"), "description": "Follow this account for geo-targeted Skilled Trade job tweets in North Carolina from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 302, "friends_count": 278, "statues_count": 18 }, "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-06T10:00:00.000Z"), "id": 684796573327429632, "text": "������ unlike some of u lucky punks https://t.co/Tiplgwq6fs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 301737395, "name": "Elle Woods", "screen_name": "cailstateofmind", "lang": "en", "location": "NC made, PA raised, FL living", "create_at": date("2011-05-19"), "description": "better a diamond with a flaw than a pebble without.", "followers_count": 862, "friends_count": 459, "statues_count": 41258 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valley View, PA", "id": "7c8a0c7f18c3277d", "name": "Valley View", "place_type": "city", "bounding_box": rectangle("-76.720096,39.933994 -76.69153,39.961314") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42133, "countyName": "York", "cityID": 4279682, "cityName": "Valley View" } }
+{ "create_at": datetime("2016-01-06T10:00:00.000Z"), "id": 684796573402796032, "text": "me and you @ashleymaeoliver https://t.co/0QvYYRReCT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1639435896 }}, "user": { "id": 1701272268, "name": "sorya", "screen_name": "soryaelizabeth", "lang": "en", "location": "Canby, Oregon", "create_at": date("2013-08-26"), "description": "how bad can a good girl get", "followers_count": 686, "friends_count": 681, "statues_count": 22426 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Canby, OR", "id": "03e962133a656983", "name": "Canby", "place_type": "city", "bounding_box": rectangle("-122.714308,45.247922 -122.668671,45.294585") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41005, "countyName": "Clackamas", "cityID": 4110750, "cityName": "Canby" } }
+{ "create_at": datetime("2016-01-06T10:00:00.000Z"), "id": 684796573415505920, "text": "@RyanWynne_ ����", "in_reply_to_status": 684793083721060352, "in_reply_to_user": 347463695, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 347463695 }}, "user": { "id": 2766795244, "name": "Andre Flint", "screen_name": "Andre_Flint2", "lang": "en", "location": "The Rodeo ", "create_at": date("2014-09-10"), "description": "LaFlame says let your ambition carry you", "followers_count": 411, "friends_count": 413, "statues_count": 6802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sheffield, OH", "id": "01a0e1ba32da743c", "name": "Sheffield", "place_type": "city", "bounding_box": rectangle("-82.131676,41.41826 -82.066778,41.489664") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39093, "countyName": "Lorain", "cityID": 3972060, "cityName": "Sheffield" } }
+{ "create_at": datetime("2016-01-06T10:00:00.000Z"), "id": 684796573495259137, "text": "@Foolio_Joshh I did that the whole week before Christmas and afterwards. I'm tired of being fake ��", "in_reply_to_status": 684796165683044357, "in_reply_to_user": 298826309, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 298826309 }}, "user": { "id": 75431189, "name": "Kylaaa", "screen_name": "IAmKyla_", "lang": "en", "location": "Bgsu18", "create_at": date("2009-09-18"), "description": "La vie est belle ✨ A peace of mind is better than anything. Good things take time. #nomorehesitation2k16", "followers_count": 1224, "friends_count": 991, "statues_count": 79390 }, "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-06T10:00:00.000Z"), "id": 684796574170415104, "text": "I love how comfy my bed 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": 3502425432, "name": "Nanii", "screen_name": "vierrakanani", "lang": "en", "location": "Anchorage, AK", "create_at": date("2015-09-09"), "description": "null", "followers_count": 272, "friends_count": 316, "statues_count": 1795 }, "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-06T10:00:00.000Z"), "id": 684796574593990656, "text": "@AmberRemarc https://t.co/tFkMzYoj3V", "in_reply_to_status": 684795741231955968, "in_reply_to_user": 341091514, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 341091514 }}, "user": { "id": 298573256, "name": "Antwonlokko", "screen_name": "4eyezLokko", "lang": "en", "location": "Norwalk, CA", "create_at": date("2011-05-14"), "description": "LKC❤", "followers_count": 533, "friends_count": 422, "statues_count": 41595 }, "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-06T10:00:00.000Z"), "id": 684796574623338497, "text": "Interested in a #Transportation #job near #Cressona, PA? This could be a great fit: https://t.co/K9oBjl9rhJ #driver https://t.co/bJK4cjS3ar", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.1927222,40.6267574"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Transportation", "job", "Cressona", "driver" }}, "user": { "id": 2414879204, "name": "JOBS at AIM", "screen_name": "JobsAtAim", "lang": "en", "location": "www.aimntls.com", "create_at": date("2014-03-27"), "description": "STABILITY. ADVANCEMENT. IMPACT. We're looking for professionals like you to join our team!\nApply to our open #jobs below, and follow us here @aimntls", "followers_count": 26, "friends_count": 4, "statues_count": 51 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cressona, PA", "id": "49f2b5d95ecb52e5", "name": "Cressona", "place_type": "city", "bounding_box": rectangle("-76.210234,40.621947 -76.181554,40.640275") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42107, "countyName": "Schuylkill", "cityID": 4217152, "cityName": "Cressona" } }
+{ "create_at": datetime("2016-01-06T10:00:00.000Z"), "id": 684796574711562240, "text": "Dude sitting behind me just walking onboard with a whole pizza.... We could be friends. #Delta #Monsterlife #travel", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.529455,42.884573"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Delta", "Monsterlife", "travel" }}, "user": { "id": 437152601, "name": "Elliott Miller", "screen_name": "ElliottMiller12", "lang": "en", "location": "null", "create_at": date("2011-12-14"), "description": "null", "followers_count": 323, "friends_count": 138, "statues_count": 286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gerald R. Ford International Airport (GRR)", "id": "07d9e3a21e087006", "name": "Gerald R. Ford International Airport (GRR)", "place_type": "poi", "bounding_box": rectangle("-85.52945509999999,42.8845729 -85.529455,42.884573") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26081, "countyName": "Kent", "cityID": 2629580, "cityName": "Forest Hills" } }
+{ "create_at": datetime("2016-01-06T10:00:00.000Z"), "id": 684796574820466688, "text": "2228679d111c24e59182a25032fbce589220e76a910f48b4d427ca81c5b78d96c314a95d336e5edb0ef5d2ce9990dfcc269d8a374f2cab76b2ffde1fddbf01f6642a41000000", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.48070314,39.54293611"), "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": "null", "followers_count": 131, "friends_count": 1, "statues_count": 687679 }, "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": 6033, "countyName": "Lake" } }
+{ "create_at": datetime("2016-01-06T10:00:01.000Z"), "id": 684796574904389632, "text": "Today is filled with proctor cache setup for ACCESS testing. Sounds fun", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 410298044, "name": "Jason", "screen_name": "jason_seybert", "lang": "en", "location": "Johnstown, CO", "create_at": date("2011-11-11"), "description": "Husband, Father, Apple Fanboy, Sports fan; Director of Education Services for Weld RE-5J which is 1:1 iPads", "followers_count": 305, "friends_count": 953, "statues_count": 1666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Johnstown, CO", "id": "01696e2a6a788ac1", "name": "Johnstown", "place_type": "city", "bounding_box": rectangle("-104.980602,40.291194 -104.886654,40.349186") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8123, "countyName": "Weld", "cityID": 839855, "cityName": "Johnstown" } }
+{ "create_at": datetime("2016-01-06T10:00:01.000Z"), "id": 684796575034519552, "text": "You niggas really ain't shit and I hope y'all fuckin know", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 72461692, "name": "Briॐ", "screen_name": "sollldeeep", "lang": "en", "location": "tri•state", "create_at": date("2009-09-07"), "description": "Psalm 24:3", "followers_count": 277, "friends_count": 89, "statues_count": 10758 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Danbury, CT", "id": "a5b6bdd8008412b1", "name": "Danbury", "place_type": "city", "bounding_box": rectangle("-73.545122,41.352784 -73.400835,41.463455") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 918430, "cityName": "Danbury" } }
+{ "create_at": datetime("2016-01-06T10:00:01.000Z"), "id": 684796575109877760, "text": "Tyler clown�� https://t.co/4JKYwmWpGD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1200795325, "name": "Rày Da Goon", "screen_name": "rhayster", "lang": "en", "location": "St. Louis Park", "create_at": date("2013-02-20"), "description": "Prolly not smoking a blunt in the cut", "followers_count": 525, "friends_count": 273, "statues_count": 1231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minnetonka, MN", "id": "555209e7a34626a2", "name": "Minnetonka", "place_type": "city", "bounding_box": rectangle("-93.523339,44.890915 -93.398853,44.978956") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743252, "cityName": "Minnetonka" } }
+{ "create_at": datetime("2016-01-06T10:00:01.000Z"), "id": 684796575156203520, "text": "@crazy4bieeber2 https://t.co/icGlUUDA8L", "in_reply_to_status": -1, "in_reply_to_user": 186923737, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 186923737 }}, "user": { "id": 2447401812, "name": "Christian", "screen_name": "KarinasHusband", "lang": "en", "location": "Massena, NY", "create_at": date("2014-04-16"), "description": "Karina Mossow is my life", "followers_count": 480, "friends_count": 1996, "statues_count": 17320 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Massena, NY", "id": "57ea83a184f3142d", "name": "Massena", "place_type": "city", "bounding_box": rectangle("-74.924113,44.898982 -74.833932,44.963117") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36089, "countyName": "St. Lawrence", "cityID": 3646019, "cityName": "Massena" } }
+{ "create_at": datetime("2016-01-06T10:00:01.000Z"), "id": 684796575353311234, "text": "#CES2016 (@ Las Vegas Convention Center in Las Vegas, NV) https://t.co/zsAHlKMERk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.15341282,36.13122002"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "CES2016" }}, "user": { "id": 40033920, "name": "Alberto Vasquez", "screen_name": "albertovasquez1", "lang": "en", "location": "chile", "create_at": date("2009-05-14"), "description": "Humano | Genuinamente perno", "followers_count": 388, "friends_count": 796, "statues_count": 1913 }, "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-06T10:00:01.000Z"), "id": 684796575441354756, "text": "Wind 0 mph --. Barometer 30.28 in, Falling. Temperature 48.9 °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": 20005 }, "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-06T10:00:01.000Z"), "id": 684796575793688576, "text": "@EmWatson hey", "in_reply_to_status": -1, "in_reply_to_user": 166739404, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 166739404 }}, "user": { "id": 736001940, "name": "John Sempa", "screen_name": "Stars_N_Bars2", "lang": "en", "location": "null", "create_at": date("2012-08-03"), "description": "My style is impetuous, my defense is impregnable, and I'm just ferocious. I want your heart. I want to eat your children.", "followers_count": 499, "friends_count": 437, "statues_count": 15821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Old Forge, PA", "id": "3800ac12d05ae3ee", "name": "Old Forge", "place_type": "city", "bounding_box": rectangle("-75.760287,41.348826 -75.721942,41.392452") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42069, "countyName": "Lackawanna", "cityID": 4256576, "cityName": "Old Forge" } }
+{ "create_at": datetime("2016-01-06T10:00:01.000Z"), "id": 684796575810367489, "text": "@beeeeklav HP marathon FTW!", "in_reply_to_status": 684775144473137152, "in_reply_to_user": 336121082, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 336121082 }}, "user": { "id": 1426737494, "name": "Alex Piorkowski", "screen_name": "A_Powkip", "lang": "en", "location": "null", "create_at": date("2013-05-13"), "description": "All my dreamers put your hands in the sky", "followers_count": 102, "friends_count": 88, "statues_count": 1507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodridge, IL", "id": "eb7b63633846877b", "name": "Woodridge", "place_type": "city", "bounding_box": rectangle("-88.087168,41.69084 -88.001354,41.779451") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1783245, "cityName": "Woodridge" } }
+{ "create_at": datetime("2016-01-06T10:00:01.000Z"), "id": 684796575839858688, "text": "Oggi è Il giorno de la Befana, papà ma detto che è mio giorno... Stronzo #befana2016", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "hashtags": {{ "befana2016" }}, "user": { "id": 40972635, "name": "Cry-lo Ren", "screen_name": "Romagnoodles", "lang": "en", "location": "Alaska please", "create_at": date("2009-05-18"), "description": "Running away to Alaska to write❄ It's my goal in life for Vanessa Redgrave to brush my hair.", "followers_count": 207, "friends_count": 311, "statues_count": 33184 }, "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-06T10:00:01.000Z"), "id": 684796576112324608, "text": "@HaleyAmparano this is not a joke u nearly had me killed in there", "in_reply_to_status": 684796253918478336, "in_reply_to_user": 2376540444, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2376540444 }}, "user": { "id": 3168903302, "name": "Nades", "screen_name": "FarraNadeen", "lang": "en", "location": "null", "create_at": date("2015-04-23"), "description": "I love rolls royces & hay", "followers_count": 369, "friends_count": 281, "statues_count": 1868 }, "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-06T10:00:01.000Z"), "id": 684796576233951232, "text": "Que pasada... �� @ The Lion King - Musical https://t.co/rSEIgpoT63", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.985672,40.7578621"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 905988528, "name": "Triky Ramírez", "screen_name": "trikyramirez", "lang": "es", "location": "null", "create_at": date("2012-10-26"), "description": "B.A.L.O.N.C.E.S.T.O.", "followers_count": 217, "friends_count": 101, "statues_count": 3731 }, "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-06T10:00:01.000Z"), "id": 684796576338804736, "text": "David is so fucking #gungho behind Veronica #HAHN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "gungho", "HAHN" }}, "user": { "id": 21927461, "name": "Adaryl Fisher", "screen_name": "ADARYLTFISHER", "lang": "en", "location": "SHREVEPORT, LA.", "create_at": date("2009-02-25"), "description": "A bio tells only so much, hit that follow button and learn a lot more.....", "followers_count": 633, "friends_count": 479, "statues_count": 12748 }, "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-06T10:00:01.000Z"), "id": 684796576561147905, "text": "@TYKNOWSTY bc of what people said he was pushing for", "in_reply_to_status": 684781807816974336, "in_reply_to_user": 1038874267, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1038874267 }}, "user": { "id": 338830892, "name": "Nick Gurera™", "screen_name": "N_Gurera_05", "lang": "en", "location": "Raymore, MO", "create_at": date("2011-07-19"), "description": "It's not about how hard you can hit, it's about how hard you can get hit, and keep moving forward. ///// sc-ngurera5", "followers_count": 315, "friends_count": 203, "statues_count": 7706 }, "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-06T10:00:01.000Z"), "id": 684796576649351168, "text": "I voted #TreeMayhem #Sweepstakes. Vote: https://t.co/Ea2se3Fhxc. No Purch or Quote Nec. Ends 1/6/16. See Rules https://t.co/WMi3HsgLBD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TreeMayhem", "Sweepstakes" }}, "user": { "id": 610010783, "name": "sawalton1980", "screen_name": "sawalton1980", "lang": "en", "location": "Columbus, GA", "create_at": date("2012-06-16"), "description": "Love God first my Husband family and I love to fish and to read and my Ga bulldog", "followers_count": 24, "friends_count": 24, "statues_count": 200 }, "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-06T10:00:01.000Z"), "id": 684796576682917892, "text": "I am blesssssed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 185397029, "name": "❁Nisveta❁", "screen_name": "whatevergrandpa", "lang": "en", "location": "Saint Louis, MO", "create_at": date("2010-08-31"), "description": "author of #crookedcarrots peep it: http://tinyurl.com/o4xa4mp✨artist/writer/bookmaker/media monarch/traveler/feminist/bosniak✨reach out: fejzicnisveta@gmail.com", "followers_count": 637, "friends_count": 358, "statues_count": 29923 }, "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-06T10:00:01.000Z"), "id": 684796576745689088, "text": "@taylor_herman11 thanks Taylor ����", "in_reply_to_status": 684790020582973440, "in_reply_to_user": 1895736924, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1895736924 }}, "user": { "id": 863712260, "name": "Noga", "screen_name": "MichaelNoga6", "lang": "en", "location": "Andover Highschool - 17'", "create_at": date("2012-10-05"), "description": "BANG BROS | Baseball/Football #6️⃣. You Don't Score Until You Score", "followers_count": 1485, "friends_count": 933, "statues_count": 8404 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Andover, MN", "id": "219ccc645d458eee", "name": "Andover", "place_type": "city", "bounding_box": rectangle("-93.40681,45.21178 -93.265395,45.304142") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27003, "countyName": "Anoka", "cityID": 2701486, "cityName": "Andover" } }
+{ "create_at": datetime("2016-01-06T10:00:01.000Z"), "id": 684796577236520962, "text": "I love this one as much as the original version �� https://t.co/2BB11fzUaP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 253879687, "name": "RESSE✨", "screen_name": "CRT___", "lang": "en", "location": "United States", "create_at": date("2011-02-17"), "description": "Princeton's world❣ SC: Resseeee", "followers_count": 2170, "friends_count": 1886, "statues_count": 59369 }, "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-06T10:00:01.000Z"), "id": 684796577265901569, "text": "@MonkOnTheRadioWant to play @martybrownusa Make You Feel My Love on ur radio,email ilikeitthatwaymusic@yahoo.com https://t.co/4Kokpw0CHq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1552469791 }}, "user": { "id": 1283634092, "name": "Belinda Loyd", "screen_name": "BelleLoyd", "lang": "en", "location": "Home Town of Marty Brown", "create_at": date("2013-03-20"), "description": "FROM MACEO, KY. THREE CHILDREN, .RETIRED, PROMOTES COUNTRY ARTIST MARTY BROWN", "followers_count": 780, "friends_count": 1413, "statues_count": 7863 }, "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-01-06T10:00:01.000Z"), "id": 684796577274310661, "text": "50/50 >>>", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4149198274, "name": "Caramel'Mocha .", "screen_name": "Dahtred", "lang": "en", "location": "null", "create_at": date("2015-11-09"), "description": "God Above Anything &; Everything • 21♐ • A Mother First ❤ • iGrind For US . . #PrettySmileWithAPrettyFace", "followers_count": 178, "friends_count": 176, "statues_count": 9819 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Apopka, FL", "id": "18ff21ef9765522d", "name": "Apopka", "place_type": "city", "bounding_box": rectangle("-81.579909,28.664067 -81.479167,28.757424") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1201700, "cityName": "Apopka" } }
+{ "create_at": datetime("2016-01-06T10:00:01.000Z"), "id": 684796577358057472, "text": "A new year means new furniture collections! Browse our selection and fall in love with the latest furniture designs: https://t.co/2rc5CFUT7B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2253873786, "name": "OCfurniture", "screen_name": "OCFurniture", "lang": "en", "location": "Orange, California", "create_at": date("2013-12-19"), "description": "Specializing in modern, contemporary and traditional home furniture at discount prices", "followers_count": 121, "friends_count": 132, "statues_count": 445 }, "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-06T10:00:01.000Z"), "id": 684796577467084800, "text": "Like Us on Facebook, Stay Current with Our Most Up to Date Offers! Click Here:...https://t.co/EDWZARFKlr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.482,37.7425"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 166665278, "name": "Mandy Fung", "screen_name": "lapetiterose681", "lang": "en", "location": "null", "create_at": date("2010-07-14"), "description": "null", "followers_count": 10, "friends_count": 19, "statues_count": 34 }, "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-06T10:00:01.000Z"), "id": 684796577593061376, "text": "Temp: 64.2°F | Humidity: 72% | Wind: NNE @ 1.6 mph | Barometer: 30.16 in | Dewpoint: 55.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": 23, "friends_count": 1, "statues_count": 164272 }, "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-06T10:00:01.000Z"), "id": 684796577739870208, "text": "We back. ✨��✨ @ Philadelphia City Hall https://t.co/PpUxfSEKvc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.16389444,39.95224722"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 153581371, "name": "gaelen mccartney", "screen_name": "gaelenmccartney", "lang": "en", "location": "null", "create_at": date("2010-06-08"), "description": "#Newburgh #NY #Temple #TylerSchoolOfArt #HootaThon #ChildrensHospitalOfPhiladelphia #FTK", "followers_count": 532, "friends_count": 621, "statues_count": 14093 }, "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-06T10:00:01.000Z"), "id": 684796577790038017, "text": "Sauna time ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 133582363, "name": "Craig", "screen_name": "LuieeMurksz", "lang": "en", "location": "Arizona ", "create_at": date("2010-04-15"), "description": "mind your business. \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\nJaylalalala", "followers_count": 1917, "friends_count": 145, "statues_count": 38928 }, "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-06T10:00:01.000Z"), "id": 684796577957842944, "text": "All Children's Hospital #Nursing #Job: RN III (#StPetersburg, FL) https://t.co/hv5zKU68y2 #Veterans #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.6398606,27.764309"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "Job", "StPetersburg", "Veterans", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22154487, "name": "Tampa Bay Nursing", "screen_name": "tmj_tpa_nursing", "lang": "en", "location": "Tampa Bay, FL", "create_at": date("2009-02-27"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Tampa Bay, FL. Need help? Tweet us at @CareerArc!", "followers_count": 567, "friends_count": 301, "statues_count": 497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "St Petersburg, FL", "id": "5d231ed8656fcf5a", "name": "St Petersburg", "place_type": "city", "bounding_box": rectangle("-82.758209,27.694323 -82.587597,27.897116") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1263000, "cityName": "St. Petersburg" } }
+{ "create_at": datetime("2016-01-06T10:00:01.000Z"), "id": 684796578033348608, "text": "@SickssHabbit @ooh_martha THATS NOT HOW YOU SPELL MY NAME!���� plus I never go to Starbucks so I know I'm late on that��", "in_reply_to_status": 684796365604392960, "in_reply_to_user": 573136318, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 573136318, 1869883710 }}, "user": { "id": 3167871157, "name": "Saderzzz", "screen_name": "y_sadiee", "lang": "en", "location": "Yuma, AZ", "create_at": date("2015-04-22"), "description": "high there! ♍️", "followers_count": 426, "friends_count": 208, "statues_count": 9398 }, "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-06T10:00:01.000Z"), "id": 684796578113040385, "text": "but you still fucked with him ? ���� https://t.co/Gx6wvPEI1U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2527598616, "name": "Selena", "screen_name": "SellySosa", "lang": "en", "location": "California, USA", "create_at": date("2014-05-27"), "description": "don't dm me, I'm pregnant. #gherbohive", "followers_count": 7514, "friends_count": 4403, "statues_count": 23156 }, "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-01-06T10:00:01.000Z"), "id": 684796578268184576, "text": "01/06@13:00 - Temp 39.2F, WC 39.2F. Wind 2.0mph SSE, Gust 6.0mph. Bar 30.535in, Falling quickly. Rain 0.02in. Hum 40%. UV 1.9. SolarRad 471.", "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": 46074 }, "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-06T10:00:01.000Z"), "id": 684796578276573184, "text": "I want a Rolex on my wrist", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 855001458, "name": "Dyllie", "screen_name": "DyllanHarris", "lang": "en", "location": "null", "create_at": date("2012-09-30"), "description": "My names Dyllan and this is my twweeeeeeeeter", "followers_count": 591, "friends_count": 470, "statues_count": 5761 }, "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-06T10:00:01.000Z"), "id": 684796578377240576, "text": "#DonaldTrump, please make #AmericaGreatAgain by leaving.\n\n(And take #HillaryClinton with you.)\n\n#feelthebern\n#votebernie \n#Bernie2016\n#Iowa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.357048,42.4817447"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DonaldTrump", "AmericaGreatAgain", "HillaryClinton", "feelthebern", "votebernie", "Bernie2016", "Iowa" }}, "user": { "id": 1941799362, "name": "Jon Foote", "screen_name": "dadiniowa", "lang": "en", "location": "Waterloo, Iowa", "create_at": date("2013-10-06"), "description": "Husband, Dad and all around good guy.... most of the time. Passions include Family, Music, Books, Social Issues, Religion, Science, Art and Beer!", "followers_count": 305, "friends_count": 260, "statues_count": 4140 }, "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-01-06T10:00:01.000Z"), "id": 684796578440204288, "text": "\"can i get all of your attention please.\" *looks but turns volume up*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1539230852, "name": "Skye Vodka", "screen_name": "skyeerdayerday", "lang": "en", "location": "above ", "create_at": date("2013-06-22"), "description": "#bittergirlz", "followers_count": 661, "friends_count": 593, "statues_count": 6510 }, "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-06T10:00:01.000Z"), "id": 684796578599710720, "text": "@joseph_asan https://t.co/JyEAx2xwtV", "in_reply_to_status": -1, "in_reply_to_user": 3383876849, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3383876849 }}, "user": { "id": 358704890, "name": "E-jir", "screen_name": "IjirSesoo", "lang": "en", "location": "FIT, Melbourne Florida ", "create_at": date("2011-08-20"), "description": "null", "followers_count": 439, "friends_count": 389, "statues_count": 1494 }, "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-01-06T10:00:01.000Z"), "id": 684796578624835584, "text": "0h 30m 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": 2532 }, "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-06T10:00:01.000Z"), "id": 684796578746335232, "text": "Congratulations Tatiana, I'm so proud of you. Keep up the good work ���� https://t.co/qstqJCYoYA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2909185620, "name": "P R I N C E $ $", "screen_name": "kayericaj", "lang": "en", "location": "HOU$TON", "create_at": date("2014-11-24"), "description": "null", "followers_count": 329, "friends_count": 249, "statues_count": 3887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-06T10:00:02.000Z"), "id": 684796579031691264, "text": "my calculus teacher is asking us about memes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2424895091, "name": "jom", "screen_name": "joncopes", "lang": "en", "location": "Raleigh, NC", "create_at": date("2014-03-20"), "description": "stay at home dad", "followers_count": 158, "friends_count": 120, "statues_count": 7961 }, "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-06T10:00:02.000Z"), "id": 684796579069362178, "text": "People pay thousands a month to listen to my mom's advice about their business. I get frustrated when she tries to help. I need to listen...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3245688319, "name": "Trey Ramirez", "screen_name": "RTREYR1", "lang": "en", "location": "Moses Lake, WA", "create_at": date("2015-06-14"), "description": "null", "followers_count": 191, "friends_count": 170, "statues_count": 2693 }, "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-01-06T10:00:02.000Z"), "id": 684796579526606848, "text": "Wind 3.0 mph SW. Barometer 30.435 in, Falling. Temperature 42.1 °F. Rain today 0.00 in. Humidity 42%", "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": 16676 }, "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-06T10:00:02.000Z"), "id": 684796579837009920, "text": "@GraceLescavage https://t.co/5OpJRdVDvE", "in_reply_to_status": -1, "in_reply_to_user": 452292345, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 452292345 }}, "user": { "id": 449276144, "name": "rg", "screen_name": "rileygerenda", "lang": "en", "location": "null", "create_at": date("2011-12-28"), "description": "life is a party once you realize it is ⚯͛ Albright", "followers_count": 648, "friends_count": 362, "statues_count": 9012 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minersville, PA", "id": "ffd9b96d46c01672", "name": "Minersville", "place_type": "city", "bounding_box": rectangle("-76.272734,40.683483 -76.245584,40.698588") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42107, "countyName": "Schuylkill", "cityID": 4250088, "cityName": "Minersville" } }
+{ "create_at": datetime("2016-01-06T10:00:02.000Z"), "id": 684796579845386240, "text": "@shunte_ u so cute Friend ��������", "in_reply_to_status": 684501841443426305, "in_reply_to_user": 191996560, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 191996560 }}, "user": { "id": 79712807, "name": "C-DayAug.28 ♿", "screen_name": "TheRealMonay", "lang": "en", "location": "♿♿♿", "create_at": date("2009-10-04"), "description": "I Just Want My Mooski Bacc♿", "followers_count": 1101, "friends_count": 930, "statues_count": 10315 }, "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-06T10:00:02.000Z"), "id": 684796580503916544, "text": "Wind 6.0 mph SE. Barometer 30.256 in, Falling Rapidly. Temperature 41.6 °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": 118, "friends_count": 0, "statues_count": 110130 }, "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-06T10:00:02.000Z"), "id": 684796580726239232, "text": "Come down to #Vinna for a leg burner! Racked #squat, double crunch, ice skater, toe touch, 500m… https://t.co/TsXMemrgJp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.27334501,44.97752723"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Vinna", "squat" }}, "user": { "id": 96684929, "name": "Brian Gervais, DC", "screen_name": "VinnaChiro", "lang": "en", "location": "Minneapolis, MN", "create_at": date("2009-12-13"), "description": "Chiropractor at Vinna Chiropractic, Certified Active Release (ART) Provider & Instructor, CCSP, SFMA/FMS. Avid golfer, love ice fishing.", "followers_count": 1134, "friends_count": 1694, "statues_count": 1140 }, "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-06T10:00:02.000Z"), "id": 684796580810063873, "text": "Productive, Pragmatic, Patience #GOALS https://t.co/7CNXchSLuD #MICOLE", "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": {{ "GOALS", "MICOLE" }}, "user": { "id": 87381820, "name": "MICOLE", "screen_name": "_itsMICOLE", "lang": "en", "location": "New York", "create_at": date("2009-11-03"), "description": "Doer of things + Lover of ideas. @MTV + #MICOLE. @TwoTheFilm director + DJ", "followers_count": 684, "friends_count": 407, "statues_count": 24545 }, "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-06T10:00:02.000Z"), "id": 684796581128740866, "text": "I'm coo with talking to my self all day lol since nobody else does", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 554043432, "name": "JOEYY BOYY WIT 2 YYs", "screen_name": "joeyyy_boyy", "lang": "en", "location": "mindyourznigga", "create_at": date("2012-04-14"), "description": "HAPPY HOLIDAYS MFS!!!! snapchat: yourguyjoe instagram: joeyy_boyy_", "followers_count": 1059, "friends_count": 1210, "statues_count": 25793 }, "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-01-06T10:00:02.000Z"), "id": 684796581351174145, "text": "Wind 2.1 mph WSW. Barometer 30.217 in, Falling. Temperature 58.7 °F. Rain today 0.00 in. Humidity 56%", "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": 56040 }, "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-06T10:00:02.000Z"), "id": 684796581380517888, "text": "Wind 0.7 mph SE. Barometer 30.44 in, Falling quickly. Temperature 32.0 °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 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": 156465 }, "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-06T10:00:02.000Z"), "id": 684796581694967810, "text": "Brownfield where is the fun at? ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1130131585, "name": "Jeremy Castillo", "screen_name": "JrmyCstllo3", "lang": "en", "location": "null", "create_at": date("2013-01-28"), "description": "WTAMU", "followers_count": 518, "friends_count": 672, "statues_count": 4934 }, "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-06T10:00:02.000Z"), "id": 684796581716078592, "text": "#amazing #tacos... never get enough (at @Tacodeli in West Lake Hills, TX) https://t.co/Q2q1PAUBsL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.826256,30.290559"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "amazing", "tacos" }}, "user_mentions": {{ 16196430 }}, "user": { "id": 27244809, "name": "Carlos D. Sanchez", "screen_name": "sanchezdavid69", "lang": "en", "location": "Austin, TX", "create_at": date("2009-03-28"), "description": "Lets go where the wind takes us...", "followers_count": 174, "friends_count": 332, "statues_count": 1785 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Lake Hills, TX", "id": "75965f8c6ee9a4b0", "name": "West Lake Hills", "place_type": "city", "bounding_box": rectangle("-97.82848,30.273069 -97.787232,30.308076") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4877632, "cityName": "West Lake Hills" } }
+{ "create_at": datetime("2016-01-06T10:00:02.000Z"), "id": 684796581988544512, "text": "Watch out by @TrillSammyy ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3242554800 }}, "user": { "id": 865684128, "name": "Cameron Mack Jr.", "screen_name": "CameronMackJr", "lang": "en", "location": "Bryan, TX", "create_at": date("2012-10-06"), "description": "#24 Instagram: Cameronmackjr", "followers_count": 1917, "friends_count": 926, "statues_count": 23111 }, "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-01-06T10:00:02.000Z"), "id": 684796582227656704, "text": "Happy Wednesday!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 326640882, "name": "West of Lenin", "screen_name": "WestofLenin", "lang": "en", "location": "Fremont Center of The Universe", "create_at": date("2011-06-30"), "description": "Fremont's 88-seat performance space for live theatre!", "followers_count": 974, "friends_count": 849, "statues_count": 5163 }, "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-06T10:00:02.000Z"), "id": 684796582403899393, "text": "Nahhhhh see this is too far https://t.co/rfN3gmgQyV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3289391543, "name": "ems", "screen_name": "_emilyweaver", "lang": "en", "location": "666", "create_at": date("2015-05-18"), "description": "bless up ~ RIP Belle ~ #RevIR", "followers_count": 571, "friends_count": 484, "statues_count": 11065 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwich, CT", "id": "ca059e12283afa1e", "name": "Norwich", "place_type": "city", "bounding_box": rectangle("-72.175195,41.490724 -72.030241,41.607853") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9011, "countyName": "New London", "cityID": 956200, "cityName": "Norwich" } }
+{ "create_at": datetime("2016-01-06T10:00:02.000Z"), "id": 684796582500286465, "text": "you seriously say things to piss 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": 2225881824, "name": "em", "screen_name": "emilyanngannon", "lang": "en", "location": "birdville", "create_at": date("2013-12-01"), "description": "God is so good", "followers_count": 794, "friends_count": 777, "statues_count": 4372 }, "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-06T10:00:02.000Z"), "id": 684796582563311617, "text": "#SupportOriginMelissa 45.1°F Wind:1.6mph Pressure: 30.36hpa 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": 306370 }, "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-06T10:00:02.000Z"), "id": 684796582617743360, "text": "64.1F (Feels: 64.1F) - Humidity: 66% - Wind: 11.4mph ESE - Gust: 12.1mph - 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": 221284 }, "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-06T10:00:02.000Z"), "id": 684796582768676864, "text": "Tb wit my dawg before the storm y'all have fun down there in San Antonio #WestSquaaa @ArmyAllAmerican https://t.co/TqNOQ5LaXb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WestSquaaa" }}, "user_mentions": {{ 67106398 }}, "user": { "id": 165540365, "name": "Kahlil McKenzie Jr.", "screen_name": "KM1_Era", "lang": "en", "location": "Tha Lab ", "create_at": date("2010-07-11"), "description": "If you stay ready you never have to get ready | University of Tennessee Football | VOLSZN | Team God", "followers_count": 28567, "friends_count": 881, "statues_count": 37192 }, "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-06T10:00:02.000Z"), "id": 684796582798036992, "text": "\"So do you suffer from vaginal dryness or are you just not that into me?\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 393165067, "name": ".", "screen_name": "SwaglessDillon", "lang": "en", "location": "San Antonio, TX", "create_at": date("2011-10-17"), "description": "i like pizza rolls and long walks on the beach Proud American", "followers_count": 3799, "friends_count": 3340, "statues_count": 65535 }, "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-06T10:00:02.000Z"), "id": 684796583116935168, "text": "Get this look at Prinzzesa Boutique Dress Shop. Located at 4740 SW 72 AVE Miami, call… https://t.co/cylm4kmeIq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.3109131,25.7268429"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325801865, "name": "Prinzzesa Boutique", "screen_name": "PRINZZESAboutiq", "lang": "en", "location": "4700 SW 72 AVE", "create_at": date("2011-06-28"), "description": "Known as one of the best boutiques in Miami. 305-661-1110 Open Monday - Saturday 11am - 6:30 pm", "followers_count": 480, "friends_count": 601, "statues_count": 2204 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glenvar Heights, FL", "id": "f546db03397f6e09", "name": "Glenvar Heights", "place_type": "city", "bounding_box": rectangle("-80.334426,25.690519 -80.292923,25.733912") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1226100, "cityName": "Glenvar Heights" } }
+{ "create_at": datetime("2016-01-06T10:00:03.000Z"), "id": 684796583611858945, "text": "you need to chill", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2531573937, "name": "Kailey Qucsai", "screen_name": "qucsaikailey", "lang": "en", "location": "Monroe, OH", "create_at": date("2014-05-05"), "description": "when I am afraid I put my trust in you. psalm 56:3 // Monroe cheerleader", "followers_count": 478, "friends_count": 373, "statues_count": 7810 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monroe, OH", "id": "000aa1f231c00b1d", "name": "Monroe", "place_type": "city", "bounding_box": rectangle("-84.462794,39.413012 -84.278288,39.476285") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39017, "countyName": "Butler", "cityID": 3951310, "cityName": "Monroe" } }
+{ "create_at": datetime("2016-01-06T10:00:03.000Z"), "id": 684796583657918465, "text": "�� https://t.co/BRKl2R4iaP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 345763701, "name": "sasha", "screen_name": "Almightysash", "lang": "en", "location": "null", "create_at": date("2011-07-30"), "description": "next level", "followers_count": 1277, "friends_count": 464, "statues_count": 34856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-06T10:00:03.000Z"), "id": 684796583888683010, "text": "Temp: 55.1°F Wind:1.5mph Pressure: 30.300hpa 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": 56041 }, "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-06T10:00:03.000Z"), "id": 684796583922122752, "text": "Morning clouds/afternoon sun this afternoon, high 49 (9 C). Low 33 (1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15953346, "name": "Amarillo Weather", "screen_name": "AmarilloWeather", "lang": "en", "location": "Amarillo, Texas", "create_at": date("2008-08-22"), "description": "Forecasts and advisories for Amarillo, Texas", "followers_count": 4045, "friends_count": 395, "statues_count": 8714 }, "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-01-06T10:00:03.000Z"), "id": 684796583997648898, "text": "What sucks is this two hour layover to fly less than an hour back to LA. -__-", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3002661502, "name": "Gg", "screen_name": "gagewayne_", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-01-27"), "description": ".21. Dancer. Choreographer. IG: gagewayne_ SC: gagewayne_", "followers_count": 320, "friends_count": 661, "statues_count": 1481 }, "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-06T10:00:03.000Z"), "id": 684796584534634497, "text": "@z100rocks @StJude https://t.co/xYhTZsLyqY", "in_reply_to_status": -1, "in_reply_to_user": 619188591, "favorite_count": 0, "coordinate": point("-89.0186596,37.80446281"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 619188591, 9624042 }}, "user": { "id": 28021175, "name": "DMac", "screen_name": "Dave_mckenzie", "lang": "en", "location": "Crainville, Illinois", "create_at": date("2009-03-31"), "description": "Radio DJ @z100rocks Host. HR/Safety Pro. Play-by-Play for Carterville Lions Sports", "followers_count": 515, "friends_count": 1287, "statues_count": 4746 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Herrin, IL", "id": "ef04eacbee05f667", "name": "Herrin", "place_type": "city", "bounding_box": rectangle("-89.06759,37.773879 -88.999855,37.847052") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17199, "countyName": "Williamson", "cityID": 1734358, "cityName": "Herrin" } }
+{ "create_at": datetime("2016-01-06T10:00:03.000Z"), "id": 684796584752758784, "text": "wow I'll be 21 next year?? strange", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 557350537, "name": "mel", "screen_name": "melody_ensor", "lang": "en", "location": "snapchat | melodyensor", "create_at": date("2012-04-18"), "description": "this is the story of a girl", "followers_count": 557, "friends_count": 322, "statues_count": 13755 }, "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-01-06T10:00:03.000Z"), "id": 684796584865828864, "text": "Just Trynna smoke 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": 1233126660, "name": "CHRIS", "screen_name": "yung_squishh", "lang": "en", "location": "null", "create_at": date("2013-03-01"), "description": "Just Wait On It @nattiecakes_", "followers_count": 1101, "friends_count": 880, "statues_count": 16187 }, "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-06T10:00:03.000Z"), "id": 684796584870166529, "text": "1 Rusty Blackbird (Euphagus carolinus) - Portal--Willow Tank - 2016-01-05 13:50 https://t.co/GVtErNyeP7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-109.0595016,31.8145008"), "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": 9943 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4003, "countyName": "Cochise" } }
+{ "create_at": datetime("2016-01-06T10:00:03.000Z"), "id": 684796584886988801, "text": "1 Gray Hawk (Buteo plagiatus) - San Pedro RNCA--San Pedro House and trails - 2016-01-05 09:35 https://t.co/CEjtgJtO2e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.1408997,31.5478992"), "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": 9943 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4003, "countyName": "Cochise", "cityID": 466845, "cityName": "Sierra Vista Southeast" } }
+{ "create_at": datetime("2016-01-06T10:00:03.000Z"), "id": 684796584937275392, "text": "1 Sprague's Pipit (Anthus spragueii) - Evergreen Turf Sod Farm--Tweedy/Pretzer (bird from perimeter roads only) - 2016-01-04 17:02", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.5906882,32.6505048"), "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": 9943 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 422360, "cityName": "Eloy" } }
+{ "create_at": datetime("2016-01-06T10:00:03.000Z"), "id": 684796584941490176, "text": "3 Sprague's Pipit (Anthus spragueii) - Santa Cruz Flats: Pretzer grassfields - 2016-01-05 09:55 https://t.co/rhN7hgQ8as", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.581025,32.6501546"), "retweet_count": 0, "lang": "de", "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": 9943 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 422360, "cityName": "Eloy" } }
+{ "create_at": datetime("2016-01-06T10:00:03.000Z"), "id": 684796584966623232, "text": "1 Broad-billed Hummingbird (Cynanthus latirostris) - Battiste Bed & Breakfast - 2016-01-05 08:00 https://t.co/Peb2sS4Use", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.2475023,31.4204234"), "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": 9943 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4003, "countyName": "Cochise", "cityID": 466845, "cityName": "Sierra Vista Southeast" } }
+{ "create_at": datetime("2016-01-06T10:00:03.000Z"), "id": 684796585138470912, "text": "1.10.15\n\n@classicalbumsun New York presents Carole King… https://t.co/22LHVmY35z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.953064,40.7268066"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 294025447 }}, "user": { "id": 2809302219, "name": "Good Room", "screen_name": "GoodRoomBK", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2014-10-05"), "description": "98 Meserole Avenue, Brooklyn http://soundcloud.com/goodroombk | goodroomgoodvibes@gmail.com", "followers_count": 5353, "friends_count": 5257, "statues_count": 1732 }, "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-06T10:00:03.000Z"), "id": 684796585272819712, "text": "#workforce (@ Papyrus in Bridgewater, NJ) https://t.co/yJdQ6Qqnto", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.61871147,40.58496839"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "workforce" }}, "user": { "id": 2252038083, "name": "Lisa Ann Kedenburg", "screen_name": "HKLisaAnn", "lang": "en", "location": "Hillsborough, NJ", "create_at": date("2013-12-30"), "description": "Hello Kitty Obsessed!!!", "followers_count": 470, "friends_count": 2056, "statues_count": 15450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bridgewater, NJ", "id": "0036e10c05fcd097", "name": "Bridgewater", "place_type": "city", "bounding_box": rectangle("-74.673756,40.5739 -74.597615,40.636758") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34035, "countyName": "Somerset", "cityID": 3444430, "cityName": "Martinsville" } }
+{ "create_at": datetime("2016-01-06T10:00:03.000Z"), "id": 684796585339936768, "text": "should I dye my hair more blonde", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 243262088, "name": "soap", "screen_name": "oksoph", "lang": "en", "location": "r.i.p. dlp 11/1/14", "create_at": date("2011-01-26"), "description": "carnitas burrito bowl white rice no beans veggies pico de gallo corn cheese sour cream guac tortilla on the side • AΓΔ https://vine.co/v/huqLEwUJ5uM #xlvii", "followers_count": 1607, "friends_count": 966, "statues_count": 39017 }, "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-06T10:00:03.000Z"), "id": 684796585562259460, "text": "Que syrah, Syrah! ☝��️#winecountry #napavalley #californiadreamin @ Auberge du Soleil https://t.co/ejpgp8ORv8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.40620591,38.49336594"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "napavalley", "californiadreamin" }}, "user": { "id": 62342397, "name": "Y A N S Y", "screen_name": "YansyStarship", "lang": "en", "location": "Somewhere Sunny ", "create_at": date("2009-08-02"), "description": "throwing lamps at people that need to lighten up", "followers_count": 272, "friends_count": 381, "statues_count": 4109 }, "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": 6095, "countyName": "Solano" } }
+{ "create_at": datetime("2016-01-06T10:00:03.000Z"), "id": 684796585767665664, "text": "This explains my life so much https://t.co/kuO3YTLGRc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 112607970, "name": "Lexie Pairrett", "screen_name": "LIKEWOAHXLEXIE", "lang": "en", "location": "Cedar Falls, IA", "create_at": date("2010-02-08"), "description": "G☮☮D VIBES ONLY ; UNI 2019", "followers_count": 556, "friends_count": 299, "statues_count": 17656 }, "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-01-06T10:00:03.000Z"), "id": 684796585813741568, "text": "Guilty cause nobody's about to love you like I will", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 92774905, "name": "Bons❣", "screen_name": "Bls_xo", "lang": "en", "location": "Cambridge, MA", "create_at": date("2009-11-26"), "description": "I love sunflowers", "followers_count": 1263, "friends_count": 1005, "statues_count": 66687 }, "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-01-06T10:00:03.000Z"), "id": 684796585885089792, "text": "Writing up blurbs for #PortlandHeat books 5 & 6 has me so excited about both of these. Can't seem to write 5 fast enough! Love this duo!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PortlandHeat" }}, "user": { "id": 259412296, "name": "Annabeth Albert", "screen_name": "AnnabethAlbert", "lang": "en", "location": "Pacific Northwest", "create_at": date("2011-03-01"), "description": "Put a Little Rainbow in Your Romance. Writer, mom, coffee & yarn addict. Quirky hero lover. Pubbed by @CarinaPress & @KensingtonBooks & repped by @epubagent", "followers_count": 2006, "friends_count": 1417, "statues_count": 25517 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McMinnville, OR", "id": "2a732a9dca7f3c87", "name": "McMinnville", "place_type": "city", "bounding_box": rectangle("-123.25428,45.185598 -123.128954,45.239298") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41071, "countyName": "Yamhill", "cityID": 4145000, "cityName": "McMinnville" } }
+{ "create_at": datetime("2016-01-06T10:00:03.000Z"), "id": 684796585964888065, "text": "6 Bronzed Cowbird (Molothrus aeneus) - Walmart Parking lot, Benson - 2016-01-04 14:10 https://t.co/UB8ErC1r0s", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.3134686,31.9727152"), "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": 9948 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4003, "countyName": "Cochise", "cityID": 405770, "cityName": "Benson" } }
+{ "create_at": datetime("2016-01-06T10:00:03.000Z"), "id": 684796586099097601, "text": "Where the hell is lethario", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 258042684, "name": "Juice Lord BigWoe", "screen_name": "EzekielDaDragon", "lang": "en", "location": "Off to See the Wizard", "create_at": date("2011-02-26"), "description": "Unicorn Dragon ✨ #E4HAM\n\n Cleopatra... catch her", "followers_count": 1411, "friends_count": 833, "statues_count": 149436 }, "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-01-06T10:00:03.000Z"), "id": 684796586556141568, "text": "Finished with a 3.3 last semester .. I WANT a 4.0 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": 295661326, "name": "Jeff Odia", "screen_name": "YOBO_Smooth", "lang": "en", "location": "On The Road To The Money ", "create_at": date("2011-05-09"), "description": "Aware of everything, but only focused on 1 thing : Me & My life .. Follow me on Insta : @trill__smooth #GNS", "followers_count": 1516, "friends_count": 617, "statues_count": 56656 }, "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-06T10:00:03.000Z"), "id": 684796586757591041, "text": "Can you find Billings on the map? Just try it at https://t.co/0B2M0L3rQz #Billings", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-108.501,45.7833"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Billings" }}, "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": 978, "friends_count": 312, "statues_count": 2515138 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Billings, MT", "id": "662aa8db4557a744", "name": "Billings", "place_type": "city", "bounding_box": rectangle("-108.692983,45.723722 -108.432965,45.871169") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30111, "countyName": "Yellowstone", "cityID": 3006550, "cityName": "Billings" } }
+{ "create_at": datetime("2016-01-06T10:00:03.000Z"), "id": 684796586770092032, "text": "Last Friday we was litt❗️ Come Turn Up with me this Friday. \n‼️ FRIDAY‼️\n#NewEra\nAll hosted by… https://t.co/Dg3R7IGnyG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.9247818,41.3047409"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NewEra" }}, "user": { "id": 80644726, "name": "Promoter", "screen_name": "DarkSkinPapi203", "lang": "en", "location": "Connecticut, New Haven203", "create_at": date("2009-10-07"), "description": "Club Promoter. #XtremeEntertainment #TurnUpKing Want Me To Host Your Next Event Email Me: Tytuspatterson@yahoo.com", "followers_count": 774, "friends_count": 441, "statues_count": 26431 }, "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-01-06T10:00:03.000Z"), "id": 684796586816192512, "text": "Cloudy this afternoon, high 48 (9 C). Low 33 (1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712333, "name": "LubbockWeather", "screen_name": "LubbockWeather", "lang": "en", "location": "Lubbock, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Lubbock, Texas", "followers_count": 1186, "friends_count": 93, "statues_count": 8047 }, "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-06T10:00:03.000Z"), "id": 684796586887614465, "text": "Show them youngins some love ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 86746614, "name": "King Scottie", "screen_name": "IHateKwame", "lang": "en", "location": " Sauce island", "create_at": date("2009-11-01"), "description": "DON'T BE SURPRISED IF I TAKE OFF ON YOU", "followers_count": 2078, "friends_count": 845, "statues_count": 111323 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Blythewood, SC", "id": "01decaa561b6b77c", "name": "Blythewood", "place_type": "city", "bounding_box": rectangle("-81.011997,34.171487 -80.941854,34.246753") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45079, "countyName": "Richland", "cityID": 4507255, "cityName": "Blythewood" } }
+{ "create_at": datetime("2016-01-06T10:00:03.000Z"), "id": 684796586921070592, "text": "���� https://t.co/eDWXpTnTIM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4237113432, "name": "Natalie ❤", "screen_name": "Naaat____", "lang": "en", "location": "null", "create_at": date("2015-11-14"), "description": "Ghs", "followers_count": 73, "friends_count": 83, "statues_count": 351 }, "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-06T10:00:03.000Z"), "id": 684796587055386624, "text": "N4TRQ-3 Near High Rise Bridge, Chesapeake, VA https://t.co/UvlSrTZeDt #APRS 13b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.27167,36.76033"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "APRS" }}, "user": { "id": 56001919, "name": "Arthur", "screen_name": "n4trqjedi", "lang": "en", "location": "APRS N of Chesapeake, VA", "create_at": date("2009-07-11"), "description": "Electronics Technician and Amateur Radio Operator N4TRQ\r\nhttp://n4trq.wordpress.com", "followers_count": 428, "friends_count": 693, "statues_count": 47285 }, "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-06T10:00:03.000Z"), "id": 684796587118182401, "text": "if I keep watching Next on MTV2 I might kms", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3133553666, "name": "Haley Bennett", "screen_name": "haybennettt", "lang": "en", "location": "null", "create_at": date("2015-04-02"), "description": "hails ya", "followers_count": 510, "friends_count": 344, "statues_count": 2286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Liberty Lake, WA", "id": "004403151d4fa6ba", "name": "Liberty Lake", "place_type": "city", "bounding_box": rectangle("-117.109206,47.631153 -117.047943,47.694973") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5339335, "cityName": "Liberty Lake" } }
+{ "create_at": datetime("2016-01-06T10:00:03.000Z"), "id": 684796587390971906, "text": "@LisaCim why do I get a feeling you stole song lyrics. ....", "in_reply_to_status": 684794437977923584, "in_reply_to_user": 21178973, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21178973 }}, "user": { "id": 994116685, "name": "#MidnightHurricane", "screen_name": "DavidCimorelli", "lang": "en", "location": "Florida", "create_at": date("2012-12-06"), "description": "Followed by my idols @cimorelliband. @rudymancuso @davedays @LilyNelsenMusic @chrisandtravis @brittanilouisetaylor follow me too if that helps lol", "followers_count": 631, "friends_count": 801, "statues_count": 13046 }, "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-01-06T10:00:04.000Z"), "id": 684796587533438976, "text": "@emma_johnson32 @_Jonesbreezy11 @sydneyperry_14 @Jadecuba8 @_Cydd", "in_reply_to_status": -1, "in_reply_to_user": 442481965, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 442481965, 840599983, 811291490, 619656057, 370489905 }}, "user": { "id": 2998037750, "name": "Melissa Vivian", "screen_name": "_melissavivian_", "lang": "en", "location": "wherever the wind takes me", "create_at": date("2015-01-26"), "description": "Heartless", "followers_count": 193, "friends_count": 132, "statues_count": 3783 }, "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-01-06T10:00:04.000Z"), "id": 684796587910893568, "text": "Want to work at rue21? We're #hiring in #Liberal, KS! Click for details: https://t.co/FcqlTHWI3V #Retail #rue21 #Job https://t.co/DXWJRLiwtQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-100.920999,37.0430812"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Liberal", "Retail", "rue21", "Job" }}, "user": { "id": 431763008, "name": "rue21", "screen_name": "rue21jobs", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2011-12-08"), "description": "null", "followers_count": 597, "friends_count": 2, "statues_count": 794 }, "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-01-06T10:00:04.000Z"), "id": 684796587957026816, "text": "Soo when's the next Bo rage? @JakeTucker_ ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 498441367 }}, "user": { "id": 458024626, "name": "Colton Summers", "screen_name": "coltonSummers27", "lang": "en", "location": "Xicoténcatl, Tamaulipas", "create_at": date("2012-01-07"), "description": "Miss you CSS - Senior @ Owasso - Never Tap", "followers_count": 666, "friends_count": 485, "statues_count": 1834 }, "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-01-06T10:00:04.000Z"), "id": 684796588200476673, "text": "I want a real love, that Jada and that Will loveeee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1523611776, "name": "Viccccc", "screen_name": "vic_lombardiiii", "lang": "en", "location": "null", "create_at": date("2013-06-16"), "description": "null", "followers_count": 3137, "friends_count": 2798, "statues_count": 34465 }, "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-06T10:00:04.000Z"), "id": 684796588338745344, "text": "Assistant Director - Financial Counselor - Loyola University Chicago: (#Chicago, IL) https://t.co/MH1osAq7eq #Veterans #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6570368,41.9998468"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Chicago", "Veterans", "Job", "Jobs", "Hiring" }}, "user": { "id": 395556906, "name": "IL Veteran Jobs", "screen_name": "tmj_il_vets", "lang": "en", "location": "Illinois Non-Metro", "create_at": date("2011-10-21"), "description": "Follow this account for geo-targeted Veteran job tweets in Illinois Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 180, "friends_count": 121, "statues_count": 191 }, "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-06T10:00:04.000Z"), "id": 684796588460408832, "text": "it must feel really awesome to have a boyfriend who just constantly cheats on you, right Kathryn?? :-)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 952911360, "name": "liv", "screen_name": "ImAliveaRuis", "lang": "en", "location": "Gardner, KS", "create_at": date("2012-11-16"), "description": "18. My tweets are prob just lyrics. http://alivearuis15.tumblr.com", "followers_count": 320, "friends_count": 278, "statues_count": 5769 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gardner, KS", "id": "00f1f5014dc531da", "name": "Gardner", "place_type": "city", "bounding_box": rectangle("-94.964408,38.788889 -94.871212,38.855976") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20091, "countyName": "Johnson", "cityID": 2025425, "cityName": "Gardner" } }
+{ "create_at": datetime("2016-01-06T10:00:04.000Z"), "id": 684796588485521408, "text": "@Derrick_Story So, great. It can feed dogs while you’re away. Is there a facility for the other end of the equation?!", "in_reply_to_status": 684794374224363520, "in_reply_to_user": 15070440, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15070440 }}, "user": { "id": 15884617, "name": "Bill Lloyd", "screen_name": "wklj", "lang": "en", "location": "California", "create_at": date("2008-08-17"), "description": "This space for rent.\n\nhttp://www.strava.com/athletes/347", "followers_count": 351, "friends_count": 264, "statues_count": 8151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio Rengstorff Area, Mountain View", "id": "59bcebcc59e5486e", "name": "San Antonio Rengstorff Area", "place_type": "neighborhood", "bounding_box": rectangle("-122.116865,37.393136 -122.091646,37.409674") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 649670, "cityName": "Mountain View" } }
+{ "create_at": datetime("2016-01-06T10:00:04.000Z"), "id": 684796588728954881, "text": "When you lowkey look like your grandmom �� https://t.co/UEeoUB46WE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 336322424, "name": "hazel", "screen_name": "belllllc", "lang": "en", "location": "null", "create_at": date("2011-07-15"), "description": "hey its me Bella", "followers_count": 516, "friends_count": 1157, "statues_count": 15300 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Voorhees, NJ", "id": "0124cc1067268adc", "name": "Voorhees", "place_type": "city", "bounding_box": rectangle("-74.988864,39.810025 -74.908642,39.87514") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34035, "countyName": "Somerset", "cityID": 3476250, "cityName": "Voorhees" } }
+{ "create_at": datetime("2016-01-06T10:00:04.000Z"), "id": 684796589022433280, "text": "Listen up, there's only five days left to purchase your ticket to the @hughacheson dinner and… https://t.co/maEUezz0AD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.37479987,33.96336388"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14119916 }}, "user": { "id": 30270468, "name": "The Door", "screen_name": "thedooronline", "lang": "en", "location": "NYC, CHI, PHL, LA, AUS", "create_at": date("2009-04-10"), "description": "The Door is a consulting, marketing & PR idea house working with culinary, entertainment, consumer goods clients & more. http://instagram.com/thedooronline", "followers_count": 4200, "friends_count": 2772, "statues_count": 6072 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Athens, GA", "id": "01f9c9fd7bb1aa6b", "name": "Athens", "place_type": "city", "bounding_box": rectangle("-83.521866,33.877554 -83.277693,34.035918") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13059, "countyName": "Clarke", "cityID": 1303440, "cityName": "Athens-Clarke County unified government (balance)" } }
+{ "create_at": datetime("2016-01-06T10:00:04.000Z"), "id": 684796589299384320, "text": "13:00:01 |Temp: 41.6ºF | Wind Chill 41.6ºF |Dew Point 25.7ºF | Rain today: 0.00 inches | Wind: 1.0 mph from the ENE, 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": 88093 }, "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-06T10:00:04.000Z"), "id": 684796589374877698, "text": "gonna close things our with \"Sometimes\" @MrTigerwilson1 #houseday @rhythmandsoul w. @thelegendarycw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "houseday" }}, "user_mentions": {{ 1358302656, 19306562, 17510991 }}, "user": { "id": 17510991, "name": "Chris Washington", "screen_name": "thelegendarycw", "lang": "en", "location": "Mt. Vernon but I rep da BX", "create_at": date("2008-11-20"), "description": "The Favorite DJ of your Favorite DJ in the mix Monday thru Thursday 12noon til 1pm on http://www.rhythmandsoulradio.com", "followers_count": 1365, "friends_count": 610, "statues_count": 12861 }, "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-06T10:00:04.000Z"), "id": 684796590058520577, "text": "Some people have the shittiest morals ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2267747167, "name": "ginger bitch ✨", "screen_name": "Jessyyy_xo", "lang": "en", "location": "null", "create_at": date("2013-12-29"), "description": "get jinky with it", "followers_count": 116, "friends_count": 595, "statues_count": 2392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elfers, FL", "id": "f385a5dfa0c1d8ca", "name": "Elfers", "place_type": "city", "bounding_box": rectangle("-82.739856,28.193916 -82.692769,28.233075") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12101, "countyName": "Pasco", "cityID": 1220275, "cityName": "Elfers" } }
+{ "create_at": datetime("2016-01-06T10:00:04.000Z"), "id": 684796590108848131, "text": "@Tayda_TooMuch hey �� !!", "in_reply_to_status": 684790063163686912, "in_reply_to_user": 299138503, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 299138503 }}, "user": { "id": 323045397, "name": "j a n u a r y 18 ♑", "screen_name": "__likaXOXO", "lang": "en", "location": "null", "create_at": date("2011-06-23"), "description": "snapchat: @xoxo_msm", "followers_count": 1337, "friends_count": 1032, "statues_count": 30635 }, "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-01-06T10:00:04.000Z"), "id": 684796590255665154, "text": "Smule Adds In-App Messaging to Sing! Karaoke App https://t.co/A5KdWXRyhp", "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": 17693, "friends_count": 17537, "statues_count": 66683 }, "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-06T10:00:04.000Z"), "id": 684796590347816960, "text": "Such Wisdom #Coexist https://t.co/Sa2t0k8b38", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Coexist" }}, "user": { "id": 417723, "name": "Paul L. McCord Jr.", "screen_name": "paulmccord", "lang": "en", "location": "Oklahoma City, Oklahoma", "create_at": date("2007-01-01"), "description": "John 14:6 - Ham Radio - K5GLH - Storm enthusiast", "followers_count": 1396, "friends_count": 1509, "statues_count": 38762 }, "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-06T10:00:04.000Z"), "id": 684796590524108800, "text": "College is so stressful I don't promote it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1122389881, "name": "Autumn Stemple", "screen_name": "autumn_nicole10", "lang": "en", "location": "va beach bby ", "create_at": date("2013-01-26"), "description": "in love with my princess Aubrie Bay ❤", "followers_count": 179, "friends_count": 125, "statues_count": 7761 }, "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-06T10:00:04.000Z"), "id": 684796590561837057, "text": "Just posted a photo @ Maldonado Home https://t.co/AoHFhRLH7s", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-120.51047714,46.57437843"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3119230753, "name": "Priscilla Maldonado", "screen_name": "sweetcurves309", "lang": "en", "location": "null", "create_at": date("2015-03-30"), "description": "null", "followers_count": 14, "friends_count": 72, "statues_count": 331 }, "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-06T10:00:04.000Z"), "id": 684796590628802560, "text": "Told Stacy (my boss) I wasn't coming in today cause I was sick and she just gave me so much attitude", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2778351928, "name": "Quinn", "screen_name": "QuinnTietz", "lang": "en", "location": "Forney, TX", "create_at": date("2014-09-20"), "description": "Senior 2016 |18| Keep it Simple.", "followers_count": 402, "friends_count": 292, "statues_count": 4374 }, "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.696499 -96.387296,32.778267") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48257, "countyName": "Kaufman", "cityID": 4826604, "cityName": "Forney" } }
+{ "create_at": datetime("2016-01-06T10:00:04.000Z"), "id": 684796590649815040, "text": "I will find out...office or condo's 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": 2233135597, "name": "Stephanie F Johnson", "screen_name": "StephFainJohn", "lang": "en", "location": "Denton, TX", "create_at": date("2013-12-06"), "description": "Hairstyle Designer", "followers_count": 88, "friends_count": 231, "statues_count": 950 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Southlake, TX", "id": "3cf92a36fad89185", "name": "Southlake", "place_type": "city", "bounding_box": rectangle("-97.20323,32.914293 -97.098767,33.007009") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4869032, "cityName": "Southlake" } }
+{ "create_at": datetime("2016-01-06T10:00:04.000Z"), "id": 684796590909882368, "text": "@PcolaHabitat @pnj @JulioDiazPNJ @cityofpensacola this is awesome! @weartv", "in_reply_to_status": 684795364713443328, "in_reply_to_user": 1262852629, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1262852629, 14834379, 785025025, 156003685, 33502398 }}, "user": { "id": 2468046780, "name": "Ricki Vann ☕️", "screen_name": "WEARRicki", "lang": "en", "location": "Pensacola, FL", "create_at": date("2014-04-28"), "description": "@WEARTV Reporter. @UFJSchool grad. Ephesians 4:1-6 News tip: rickivann@sbgtv.com. #Periscope: https://www.periscope.tv/RickiVann", "followers_count": 1902, "friends_count": 2032, "statues_count": 4300 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Pensacola, FL", "id": "0febf38d0a2b15bc", "name": "West Pensacola", "place_type": "city", "bounding_box": rectangle("-87.292972,30.397861 -87.240187,30.453919") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12033, "countyName": "Escambia", "cityID": 1276675, "cityName": "West Pensacola" } }
+{ "create_at": datetime("2016-01-06T10:00:04.000Z"), "id": 684796591094415360, "text": "I'm in bed right now but in exactly an hour I'll be getting chips and salsa, how sad 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": 3244472202, "name": "ty", "screen_name": "TyyHen", "lang": "en", "location": "null", "create_at": date("2015-06-13"), "description": "no pasa nada ¯\\_(ツ)_/¯", "followers_count": 179, "friends_count": 190, "statues_count": 4113 }, "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-06T10:00:04.000Z"), "id": 684796591241179138, "text": "Ok at least puts some pants on fam. You gotta outchea like he an omega https://t.co/w7OsQkE9pH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 268645403, "name": "Teddy Chill 4:09", "screen_name": "sophistica_TED", "lang": "en", "location": "Bordeaux, France", "create_at": date("2011-03-18"), "description": "Working. Touching road...", "followers_count": 713, "friends_count": 212, "statues_count": 70048 }, "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-06T10:00:04.000Z"), "id": 684796591354462208, "text": "The #TwoBrosRoundhouse Brew N View is back! Tonight we feature Ex_Machina @ 7:30 p.m. No cover! https://t.co/EJyvyrzvTr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TwoBrosRoundhouse" }}, "user": { "id": 36999884, "name": "Two Brothers Brewing", "screen_name": "TwoBrothersBeer", "lang": "en", "location": "Warrenville, IL", "create_at": date("2009-05-01"), "description": "After a trip to France that fostered their passion for beer, brothers Jim and Jason Ebel founded Two Brothers Brewing Company in 1996. Cheers!", "followers_count": 24374, "friends_count": 339, "statues_count": 2691 }, "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-06T10:00:04.000Z"), "id": 684796591362854912, "text": "@iamdanielskye come visit me !! haha", "in_reply_to_status": 684575375108734977, "in_reply_to_user": 1629171145, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1629171145 }}, "user": { "id": 2914418684, "name": "☪malia samantha☪", "screen_name": "cx_maliaa", "lang": "en", "location": "null", "create_at": date("2014-11-29"), "description": "wasssup. Daniel Skye (:", "followers_count": 383, "friends_count": 1029, "statues_count": 2652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:04.000Z"), "id": 684796591467663360, "text": "@alisabeth777 now that's the all time worst", "in_reply_to_status": 684791373338300418, "in_reply_to_user": 2938529383, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2938529383 }}, "user": { "id": 391718160, "name": "natasha allen", "screen_name": "tashabrianne", "lang": "en", "location": "null", "create_at": date("2011-10-15"), "description": "I'm an acquired taste, if you don't like me, acquire some taste.", "followers_count": 340, "friends_count": 359, "statues_count": 2085 }, "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-06T10:00:05.000Z"), "id": 684796591836758016, "text": "I love the New belly rings Taylor 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": 2292947338, "name": "Jah-Ney ✨", "screen_name": "_NeyBellaa", "lang": "en", "location": "Cleveland, OH", "create_at": date("2014-01-19"), "description": "I Got Every Reason to feel like I'm that b**** || WHHS Varsity Cheerleader \nDiamond's World", "followers_count": 894, "friends_count": 897, "statues_count": 6597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warrensville Heights, OH", "id": "166b716425d5bb87", "name": "Warrensville Heights", "place_type": "city", "bounding_box": rectangle("-81.555545,41.423881 -81.488156,41.457349") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3980990, "cityName": "Warrensville Heights" } }
+{ "create_at": datetime("2016-01-06T10:00:05.000Z"), "id": 684796591945875460, "text": "Jane the Virgin is such a cute show honestly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3545451494, "name": "jac jac", "screen_name": "lmaojaci", "lang": "en", "location": "tacompton", "create_at": date("2015-09-12"), "description": "but did you die tho?", "followers_count": 825, "friends_count": 780, "statues_count": 15481 }, "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-06T10:00:05.000Z"), "id": 684796592117813250, "text": "@_ambition1995 appreciate it bro", "in_reply_to_status": 684790326934945794, "in_reply_to_user": 298948052, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 298948052 }}, "user": { "id": 153670458, "name": "Charles Scott", "screen_name": "cwsjr_", "lang": "en", "location": "Meridian, MS", "create_at": date("2010-06-08"), "description": "Blessed and highly favored! MCMBB #0", "followers_count": 955, "friends_count": 696, "statues_count": 12576 }, "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-06T10:00:05.000Z"), "id": 684796592398831616, "text": "I'm pretty", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 513920338, "name": "hilary bank$", "screen_name": "_myrakaye", "lang": "en", "location": "okc", "create_at": date("2012-03-03"), "description": "Put yourself first.", "followers_count": 12823, "friends_count": 1256, "statues_count": 18322 }, "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-06T10:00:05.000Z"), "id": 684796592487022593, "text": "Nada más falta que le diga que se haga un tatuaje que diga forever", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 279888011, "name": "Bibi Gallegos", "screen_name": "BibiGallegos", "lang": "es", "location": "null", "create_at": date("2011-04-09"), "description": "V❤️", "followers_count": 419, "friends_count": 352, "statues_count": 31035 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tarrytown, NY", "id": "5bc740e49d25a1a5", "name": "Tarrytown", "place_type": "city", "bounding_box": rectangle("-73.894691,41.046491 -73.830903,41.087041") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3673176, "cityName": "Tarrytown" } }
+{ "create_at": datetime("2016-01-06T10:00:05.000Z"), "id": 684796592541446144, "text": "Grabe tama yung January. Eksakto! Haha ���� https://t.co/EQ5LMwNd9a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 960485504, "name": "Orlando Ramos", "screen_name": "ItsOrlandoRamos", "lang": "en", "location": "Philippines ✈ United States", "create_at": date("2012-11-20"), "description": "⚫Dragon Slayer❌ ⚫snapchat: itsorlandoramos", "followers_count": 1238, "friends_count": 649, "statues_count": 24371 }, "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-06T10:00:05.000Z"), "id": 684796592688267264, "text": "Simple man. Blues, food, ass, and Polo..das it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325116562, "name": "Jody Jo", "screen_name": "PimpTight_Boo", "lang": "en", "location": "Missouri ", "create_at": date("2011-06-27"), "description": "Mississippi born Georgia raised", "followers_count": 1550, "friends_count": 1093, "statues_count": 115766 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Leonard Wood, MO", "id": "6cccdaa30057d3d1", "name": "Fort Leonard Wood", "place_type": "city", "bounding_box": rectangle("-92.156058,37.692152 -92.081902,37.796725") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29169, "countyName": "Pulaski", "cityID": 2925264, "cityName": "Fort Leonard Wood" } }
+{ "create_at": datetime("2016-01-06T10:00:05.000Z"), "id": 684796592776286209, "text": "Cloudy this afternoon, high 53 (12 C). Low 38 (3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712562, "name": "ElPasoWX", "screen_name": "ElPasoWX", "lang": "en", "location": "El Paso, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for El Paso, Texas", "followers_count": 526, "friends_count": 93, "statues_count": 8105 }, "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-06T10:00:05.000Z"), "id": 684796592889696256, "text": "Lookin ruff! #keystosuccess�� #theydontwantyoutogetmoney @ Pembroke Lakes Mall https://t.co/2Mt9lY3b53", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.3040355,26.01046022"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "keystosuccess", "theydontwantyoutogetmoney" }}, "user": { "id": 1064740350, "name": "ChocThaGawd", "screen_name": "BETTY_MARLEY", "lang": "en", "location": "null", "create_at": date("2013-01-05"), "description": "You could never play me, I'm from CRP baby", "followers_count": 206, "friends_count": 287, "statues_count": 5911 }, "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-06T10:00:05.000Z"), "id": 684796593367871488, "text": "@DenverAsshole hahaha", "in_reply_to_status": 684796329445306368, "in_reply_to_user": 1391064296, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 1391064296 }}, "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": 20313, "friends_count": 16654, "statues_count": 86554 }, "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-06T10:00:05.000Z"), "id": 684796593384611840, "text": "@YunggTwinn yu already broski Ima hit yu when I'm out der", "in_reply_to_status": -1, "in_reply_to_user": 497152953, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 497152953 }}, "user": { "id": 241731487, "name": "Cold Hearted ✈", "screen_name": "_BigDaddyy_", "lang": "en", "location": "Road to Success ", "create_at": date("2011-01-22"), "description": "The world is so typical, I just wanna be higher than that Instagram: bigdaddyy_", "followers_count": 632, "friends_count": 539, "statues_count": 63314 }, "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-06T10:00:05.000Z"), "id": 684796594097631232, "text": "It go down 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": 48374034, "name": "Dy'Anna❤️", "screen_name": "GUDWEED_MORESEX", "lang": "en", "location": "❤ Living In Shakaki World❤ ", "create_at": date("2009-06-18"), "description": "TRUST NONE .!#LongLiveKAKI ❤ Rest Up Juelz & #LONGLIVEQUISE", "followers_count": 3049, "friends_count": 2486, "statues_count": 63914 }, "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-06T10:00:05.000Z"), "id": 684796594336755715, "text": "Rockin Dries since 2012, why the sudden hype? :(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1666348254, "name": "Dad Bod Gawd", "screen_name": "mgpolo92", "lang": "en", "location": "null", "create_at": date("2013-08-12"), "description": "Tampa//Orlando.", "followers_count": 462, "friends_count": 278, "statues_count": 20183 }, "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-06T10:00:05.000Z"), "id": 684796594382868484, "text": "someone NEEDS to do a youtube smash of ALL best parts of Avery family phone calls LOL\n#MakingAMurderer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MakingAMurderer" }}, "user": { "id": 3076245165, "name": "DavKind", "screen_name": "DavKind76", "lang": "en", "location": "I follow back wrestling fans ", "create_at": date("2015-03-06"), "description": "#WWE Wrestling fan~LIVE tweeting #RAW, PPVs and sometimes #WWENxT", "followers_count": 2805, "friends_count": 2275, "statues_count": 13642 }, "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-06T10:00:05.000Z"), "id": 684796594454048768, "text": "Reporter Hammers Trump On Extra-Marital Affair&#8230; &#8216;The Donald&#8217; Gives Unexpected Reply https://t.co/eSh6ha77WQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1285069740, "name": "Olga Vazquez", "screen_name": "terracotta05", "lang": "en", "location": "California", "create_at": date("2013-03-20"), "description": "Retired from Healthcare. Its all about Faith in God, Family, Art (Design Mosaic, Paint), Cooking, Gardening, as well as, Study Real Estate .", "followers_count": 1464, "friends_count": 2200, "statues_count": 22765 }, "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-06T10:00:05.000Z"), "id": 684796594483494912, "text": "@druzoom3 thanks", "in_reply_to_status": 684793531735642113, "in_reply_to_user": 118123396, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 118123396 }}, "user": { "id": 288971043, "name": "GBesterII", "screen_name": "LowkeyGB", "lang": "en", "location": "Wisconsin, USA", "create_at": date("2011-04-27"), "description": "Expect Thee Unexpected. UWO #OurTime", "followers_count": 1102, "friends_count": 1005, "statues_count": 11203 }, "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-01-06T10:00:05.000Z"), "id": 684796594621812736, "text": "This sounds like a Bond theme, in the best possible way.\n\n#nowplaying Under the Influence @ElleKingMusic", "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": {{ 14754932 }}, "user": { "id": 17297735, "name": "94/7fm", "screen_name": "947fm", "lang": "en", "location": "#portland oregon", "create_at": date("2008-11-10"), "description": "#dec2rem", "followers_count": 9311, "friends_count": 2032, "statues_count": 14424 }, "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-06T10:00:05.000Z"), "id": 684796594814873600, "text": "I need to its been like 2.5 weeks.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3307622590, "name": "Victoria.", "screen_name": "yarbroughplease", "lang": "en", "location": "PA-ATL ", "create_at": date("2015-06-03"), "description": "Your second life begins when you realize that you only have one.", "followers_count": 396, "friends_count": 394, "statues_count": 877 }, "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-06T10:00:05.000Z"), "id": 684796595007705088, "text": "Always wonder how would people act if there wasn't no social media.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 606878355, "name": "Cristian Love", "screen_name": "_504cityboy", "lang": "en", "location": "NewOrleans✈️Dallas✈️OKC", "create_at": date("2012-06-12"), "description": "Trust no one The Devil was once an Angel | #6'0 Freshman FS for The University of Tulsa'19| #Hurricanes | R.I.P Vell | R.I.P Lil Steve | R.I.P Lil Spazz", "followers_count": 1095, "friends_count": 727, "statues_count": 27418 }, "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-06T10:00:05.000Z"), "id": 684796595070582784, "text": "Wow! Well said! @gavindawson @MikeBacsik @1053SS https://t.co/FLnFlHXt6i", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20809475, 35105353, 132734157 }}, "user": { "id": 3048306462, "name": "Jamie Campbell", "screen_name": "fyrfyter5415", "lang": "en", "location": "Crowley, TX", "create_at": date("2015-02-27"), "description": "Husband, Father, Son, Brother. Specialist and Firefighter saved by God's grace, blessed beyond measure.", "followers_count": 16, "friends_count": 96, "statues_count": 297 }, "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-06T10:00:05.000Z"), "id": 684796595108474882, "text": "Today has been the worst day I've had in 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": 543636633, "name": "Brendan", "screen_name": "BeeBrendan", "lang": "en", "location": "null", "create_at": date("2012-04-02"), "description": "All my homies philosophers", "followers_count": 717, "friends_count": 446, "statues_count": 6329 }, "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-06T10:00:05.000Z"), "id": 684796595154628611, "text": "@SaltyWalty70 NEVER GIVE UP", "in_reply_to_status": 683454589249687552, "in_reply_to_user": 492566353, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 492566353 }}, "user": { "id": 859036754, "name": "❁demz❁", "screen_name": "demifg", "lang": "en", "location": "VA", "create_at": date("2012-10-02"), "description": "Romans 12:2 ~Affiliated with CBS~ Life Goes On(-:", "followers_count": 548, "friends_count": 289, "statues_count": 19401 }, "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-01-06T10:00:05.000Z"), "id": 684796595234189312, "text": "The senior Marines make us text someone to tell them we love them before we jump & it makes me feel like an idiot", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 566463796, "name": "(hey)den", "screen_name": "HaydenHayess", "lang": "en", "location": "College Station, TX", "create_at": date("2012-04-29"), "description": "United States Marine Corps", "followers_count": 282, "friends_count": 194, "statues_count": 2589 }, "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-01-06T10:00:05.000Z"), "id": 684796595293011968, "text": "@Pingaso_Robert2 https://t.co/o1zN59QNnf", "in_reply_to_status": -1, "in_reply_to_user": 3254773588, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3254773588 }}, "user": { "id": 417441060, "name": "Angel MasterChef", "screen_name": "AIRDRIGUEZ", "lang": "en", "location": "by God", "create_at": date("2011-11-20"), "description": "luv beth", "followers_count": 1182, "friends_count": 115, "statues_count": 28617 }, "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-06T10:00:05.000Z"), "id": 684796595301400581, "text": "@jjazzzb ������", "in_reply_to_status": 684796241151160320, "in_reply_to_user": 3491363477, "favorite_count": 0, "coordinate": point("-76.62293496,39.4666306"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3491363477 }}, "user": { "id": 176147462, "name": "solo", "screen_name": "BigA_MFG4Life", "lang": "en", "location": "null", "create_at": date("2010-08-08"), "description": "January 12th poppin", "followers_count": 1408, "friends_count": 1853, "statues_count": 18109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cockeysville, MD", "id": "014f4cd64f60daea", "name": "Cockeysville", "place_type": "city", "bounding_box": rectangle("-76.672912,39.453647 -76.58212,39.534252") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2418250, "cityName": "Cockeysville" } }
+{ "create_at": datetime("2016-01-06T10:00:05.000Z"), "id": 684796595339046912, "text": "@_Tymberlyn yessssss, too excited!������������", "in_reply_to_status": 684796040550064128, "in_reply_to_user": 592132666, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 592132666 }}, "user": { "id": 1600243574, "name": "Lay✨", "screen_name": "Lashelle_Berry", "lang": "en", "location": "Paris, Ile-de-France✈️", "create_at": date("2013-07-16"), "description": "//Isn’t it scary knowing that any time could be the last time you talk to someone? Always keep that in mind// positive vibes❤️", "followers_count": 2226, "friends_count": 1750, "statues_count": 25377 }, "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-06T10:00:05.000Z"), "id": 684796595456495616, "text": "@MicheleJosue Your film was a powerful & emotional tribute to your friend, Matt. I am truly sorry for your loss. BTW, you are beautiful. ��", "in_reply_to_status": -1, "in_reply_to_user": 97932940, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 97932940 }}, "user": { "id": 311721857, "name": "William Kaszycki", "screen_name": "Kaszycki", "lang": "en", "location": "Long Beach, CA", "create_at": date("2011-06-05"), "description": "I used to rock sweatpants in Elementary School. I had buckteeth & big ears. I'm one uncoordinated individual. I can't say 'manipulative' w/o stuttering.", "followers_count": 28, "friends_count": 69, "statues_count": 242 }, "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-06T10:00:05.000Z"), "id": 684796595523551233, "text": "Cheerios with cherries and coconut 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": 20335491, "name": "Michael Sparks", "screen_name": "Ponyzord", "lang": "en", "location": "Yakima, WA", "create_at": date("2009-02-07"), "description": "Musician, Humorist, Babydangler. Rogue extraordinaire. Sold my soul for three Yodels and half a Ding-Dong. Battletag: ponyzord#1825", "followers_count": 421, "friends_count": 1102, "statues_count": 24464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Union Gap, WA", "id": "168a4e9135ee2989", "name": "Union Gap", "place_type": "city", "bounding_box": rectangle("-120.530315,46.530631 -120.470195,46.57774") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53077, "countyName": "Yakima", "cityID": 5373290, "cityName": "Union Gap" } }
+{ "create_at": datetime("2016-01-06T10:00:05.000Z"), "id": 684796595544682496, "text": "@MLBGIFs hey handsome are the better of them all baseball players", "in_reply_to_status": 684794170192465921, "in_reply_to_user": 2575581534, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2575581534 }}, "user": { "id": 3126595180, "name": "Ray Zahn", "screen_name": "ZahnRay", "lang": "en", "location": "null", "create_at": date("2015-03-29"), "description": "null", "followers_count": 14, "friends_count": 75, "statues_count": 622 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tenafly, NJ", "id": "fa9e955670752b3c", "name": "Tenafly", "place_type": "city", "bounding_box": rectangle("-73.984572,40.899734 -73.927398,40.937822") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3472420, "cityName": "Tenafly" } }
+{ "create_at": datetime("2016-01-06T10:00:05.000Z"), "id": 684796595561349124, "text": "Way to push the envelope in a kid's movie.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 182179084, "name": "Connor Ferguson", "screen_name": "CFergOnDeck", "lang": "en", "location": "Stankonia", "create_at": date("2010-08-23"), "description": "Proud intellectual, film major at OCCC and diehard Pokemon fan. Always doing big things. Life's all about seeing the bigger picture.", "followers_count": 394, "friends_count": 993, "statues_count": 23756 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nichols Hills, OK", "id": "d36c005b82579397", "name": "Nichols Hills", "place_type": "city", "bounding_box": rectangle("-97.558488,35.535983 -97.528492,35.559102") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4051800, "cityName": "Nichols Hills" } }
+{ "create_at": datetime("2016-01-06T10:00:05.000Z"), "id": 684796595712331776, "text": "Cloudy this afternoon, high 54 (12 C). Low 43 (6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712512, "name": "AbileneWeather", "screen_name": "AbileneWeather", "lang": "en", "location": "Abilene, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Abilene, Texas", "followers_count": 930, "friends_count": 93, "statues_count": 8145 }, "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-06T10:00:05.000Z"), "id": 684796595775340544, "text": "@IhateJKuhn I still think it's dumb to pay that much though, but hey, they're the most comfy shoes ever https://t.co/Ijlw33oPm3", "in_reply_to_status": 684763686863527936, "in_reply_to_user": 380711917, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 380711917 }}, "user": { "id": 82189956, "name": "Duell", "screen_name": "DuellOsborne", "lang": "en", "location": "null", "create_at": date("2009-10-13"), "description": "null", "followers_count": 498, "friends_count": 476, "statues_count": 12259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairburn, GA", "id": "0006d55513b0c159", "name": "Fairburn", "place_type": "city", "bounding_box": rectangle("-84.626791,33.509481 -84.531197,33.59282") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1328380, "cityName": "Fairburn" } }
+{ "create_at": datetime("2016-01-06T10:00:06.000Z"), "id": 684796595871858690, "text": "Don't waste my time.. Seriously", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2321433462, "name": "LikMoe", "screen_name": "Arab_migo", "lang": "en", "location": "everywhere.", "create_at": date("2014-01-31"), "description": "15AG", "followers_count": 508, "friends_count": 187, "statues_count": 4490 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westwood, MI", "id": "5ef83697018ccdd9", "name": "Westwood", "place_type": "city", "bounding_box": rectangle("-85.649338,42.288789 -85.608758,42.336593") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26077, "countyName": "Kalamazoo", "cityID": 2686380, "cityName": "Westwood" } }
+{ "create_at": datetime("2016-01-06T10:00:06.000Z"), "id": 684796595901214720, "text": "Listen to For the cause by k.suse #np on #SoundCloud\nhttps://t.co/Ox5jWQfwjN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "np", "SoundCloud" }}, "user": { "id": 315451034, "name": "k.suse saves", "screen_name": "ikanive666", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2011-06-11"), "description": "jus your ordinary amazing super human...", "followers_count": 217, "friends_count": 624, "statues_count": 1369 }, "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-06T10:00:06.000Z"), "id": 684796595963998208, "text": "Decisions...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1287107520, "name": "Alex", "screen_name": "TheAlexOrtiz", "lang": "en", "location": "United States Navy ⚓️", "create_at": date("2013-03-21"), "description": "don't give up on your goals.", "followers_count": 146, "friends_count": 202, "statues_count": 5900 }, "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-06T10:00:06.000Z"), "id": 684796596039626753, "text": "Run, nigga, run... https://t.co/gVVf6emqpB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35818208, "name": "NIGris Elba", "screen_name": "hosienation", "lang": "en", "location": "Nashville", "create_at": date("2009-04-27"), "description": "Realist. Observational humorist. Political strategist. Satirist. Writer. Sent to end the Confederacy. *Offensive Material* All tweets in TechNIGcolor...", "followers_count": 5384, "friends_count": 1441, "statues_count": 109669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Smyrna, TN", "id": "f9eb63ab4f7dd5db", "name": "Smyrna", "place_type": "city", "bounding_box": rectangle("-86.624429,35.879451 -86.458783,36.029213") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47149, "countyName": "Rutherford", "cityID": 4769420, "cityName": "Smyrna" } }
+{ "create_at": datetime("2016-01-06T10:00:06.000Z"), "id": 684796596307931136, "text": "Today is one of those days when I need one good song to put 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": 15626483, "name": "Steve Netniss", "screen_name": "snetniss", "lang": "en", "location": "California, USA", "create_at": date("2008-07-27"), "description": "Educator | Speaker | Author of POTENTIAL: The Assyrian Quest for Identity", "followers_count": 783, "friends_count": 200, "statues_count": 6049 }, "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-06T10:00:06.000Z"), "id": 684796596349870080, "text": "I don't even like spicy food. All of a sudden smh hungry A F", "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": "408", "create_at": date("2013-12-18"), "description": "null", "followers_count": 339, "friends_count": 254, "statues_count": 12278 }, "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-06T10:00:06.000Z"), "id": 684796596588916736, "text": "I forgot High Schoolers are already back in school ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 848427684, "name": "kash reneè", "screen_name": "kash4short", "lang": "en", "location": "null", "create_at": date("2012-09-26"), "description": "Chocolate Goddess ✨✨ ..........#TxSU snapchat: iamkashh -DMs are closed", "followers_count": 6350, "friends_count": 1341, "statues_count": 56813 }, "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-06T10:00:06.000Z"), "id": 684796596651823106, "text": "I'm boutta leave y'all alone gimme bout 20 minutes 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": "null", "create_at": date("2011-03-20"), "description": "null", "followers_count": 7304, "friends_count": 1381, "statues_count": 290686 }, "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-01-06T10:00:06.000Z"), "id": 684796596735717376, "text": "Wind 0 mph NE. Barometer 1012.2 hPa, Falling. Temperature 83.1 °F. Rain today 0.00 in. Humidity 10%", "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": 140493 }, "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-06T10:00:06.000Z"), "id": 684796596848963584, "text": "Adele really has me crying in class", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2818750952, "name": "nina isabella", "screen_name": "ninaisabellaaa", "lang": "en", "location": "Lubbock, TX", "create_at": date("2014-09-18"), "description": "null", "followers_count": 1060, "friends_count": 1212, "statues_count": 7746 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wolfforth, TX", "id": "017e37957dac1dd8", "name": "Wolfforth", "place_type": "city", "bounding_box": rectangle("-102.026275,33.4907 -101.991693,33.54877") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4879972, "cityName": "Wolfforth" } }
+{ "create_at": datetime("2016-01-06T10:00:06.000Z"), "id": 684796596874248193, "text": "Touchdown! ✈ Key West, FL ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user": { "id": 30450506, "name": "Archie Francisco", "screen_name": "a12chi3", "lang": "en", "location": "Virginia Beach", "create_at": date("2009-04-11"), "description": "F/A-18EF mechanic @USNavy / Be thankful for what you have. You have no idea how many people would love to have what you got. \nSC: un4g8abl3 IG: a12chi3", "followers_count": 512, "friends_count": 293, "statues_count": 17713 }, "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-06T10:00:06.000Z"), "id": 684796596891041793, "text": "@barstoolsports hey OBJ what time the Giants game this weekend? #stfuBiach", "in_reply_to_status": 684795277853736960, "in_reply_to_user": 22637974, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "stfuBiach" }}, "user_mentions": {{ 22637974 }}, "user": { "id": 36236028, "name": "Larry", "screen_name": "ToAsT_JoNeS", "lang": "en", "location": "null", "create_at": date("2009-04-28"), "description": "#Natitude", "followers_count": 78, "friends_count": 168, "statues_count": 19764 }, "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-01-06T10:00:06.000Z"), "id": 684796597025161216, "text": "Meanwhile in America...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30644317, "name": "αlpha male", "screen_name": "chuckicheese", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-04-12"), "description": "I once did something. It was great. Bitches was like aww shit. And I was like yeah, bitches. Favorite: @Kamari_Danae. married to @banterbarbie", "followers_count": 1594, "friends_count": 572, "statues_count": 50447 }, "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-06T10:00:06.000Z"), "id": 684796597364867074, "text": "@stephenmusgrave @BrianDavidPlatt @HarsCoveAssoc I'm just bitter abt the pedXing signs that were on Jersey Ave. Only 1 survivor left.", "in_reply_to_status": 684572825949224960, "in_reply_to_user": 14499603, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14499603, 2249040150, 91194394 }}, "user": { "id": 2275506834, "name": "Ana Flaxes", "screen_name": "2askylark", "lang": "en", "location": "Jersey City, New Jersey", "create_at": date("2014-01-03"), "description": "Loves Science. And coffee.", "followers_count": 133, "friends_count": 164, "statues_count": 1441 }, "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-06T10:00:06.000Z"), "id": 684796597780234241, "text": "https://t.co/wG3gfW6oI8 @PackWrestle @pat_popolizio @NGWIZZZ https://t.co/noqfFKBXqe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 163892052, 847812061, 367577957 }}, "user": { "id": 2618640843, "name": "Duanesburg Wrestling", "screen_name": "DburgWrestling", "lang": "en", "location": "Duanesburg, NY", "create_at": date("2014-06-17"), "description": "This is an update page about Duanesburg Wrestling. Any negative or foul talk/ language will not be tolerated.", "followers_count": 86, "friends_count": 91, "statues_count": 243 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Schenectady, NY", "id": "389fab2fead253c3", "name": "Schenectady", "place_type": "city", "bounding_box": rectangle("-73.983266,42.763657 -73.893018,42.842612") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady", "cityID": 3665508, "cityName": "Schenectady" } }
+{ "create_at": datetime("2016-01-06T10:00:06.000Z"), "id": 684796598342139904, "text": "I could go buy rillos if I was gonna do that I just have NO ID ���� https://t.co/reR29s0zzQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 228222856, "name": "Tom Sawyer", "screen_name": "Nonchxlxnt_", "lang": "en", "location": "-VA-✈-TX-", "create_at": date("2010-12-18"), "description": "Mikey B | My presence is a present kiss my ass", "followers_count": 854, "friends_count": 451, "statues_count": 26349 }, "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-06T10:00:06.000Z"), "id": 684796598665138176, "text": "@DaveBolsen wow you're a complete dick. #mute @tldluvs2live @thib1145", "in_reply_to_status": 684781080247848960, "in_reply_to_user": 4681593972, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "mute" }}, "user_mentions": {{ 4681593972, 2866146572, 424824007 }}, "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": 1021, "friends_count": 1619, "statues_count": 22799 }, "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-06T10:00:06.000Z"), "id": 684796598707081216, "text": "Morning clouds/afternoon sun this afternoon, high 57 (14 C). Low 42 (6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712530, "name": "MidlandWeather", "screen_name": "MidlandWeather", "lang": "en", "location": "Midland/Odessa, TX", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Midland/Odessa, TX", "followers_count": 878, "friends_count": 93, "statues_count": 8129 }, "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-01-06T10:00:06.000Z"), "id": 684796598837231616, "text": "VGo robot helps fight Ebola, allows Audi to service vehicles, and helps Jetblue customers. https://t.co/3i3AwHNMG9 #telepresence", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "telepresence" }}, "user": { "id": 209242836, "name": "Charles Johnson", "screen_name": "Charles_CDJ", "lang": "en", "location": "Boston, MA", "create_at": date("2010-10-28"), "description": "Entrepreneur, startup lover and self-improvement enthusiast. Biz Dev @VecnaInc. Entrepreneur Coach at @ideaNEU. Advisor @SumuApp.", "followers_count": 2783, "friends_count": 2296, "statues_count": 2957 }, "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-06T10:00:06.000Z"), "id": 684796599109824513, "text": "February 11 plssss hurry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 338035141, "name": "shakiera.", "screen_name": "xbella_amor", "lang": "en", "location": "null", "create_at": date("2011-07-18"), "description": "I'm feeling way better. | resteasyjerrell | ❤️", "followers_count": 828, "friends_count": 415, "statues_count": 12096 }, "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-01-06T10:00:06.000Z"), "id": 684796599126458369, "text": "Que sea lo que Dios quiera", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 743842081, "name": "Cristian Marin", "screen_name": "CrisMarinR", "lang": "es", "location": "null", "create_at": date("2012-08-07"), "description": "null", "followers_count": 761, "friends_count": 812, "statues_count": 5820 }, "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-06T10:00:06.000Z"), "id": 684796599298437124, "text": "Temp: 32.4°F - Dew Point: 20.1° - Wind: 9.2 mph - Gust: 18.3 - Rain Today: 0.00in. - Pressure: 30.24in, - 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": 11615 }, "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-06T10:00:06.000Z"), "id": 684796599298473984, "text": "@Thomasismyuncle The same can be said for Cruz...and he doesn't seem to want the establishment's votes.", "in_reply_to_status": 684795037549486080, "in_reply_to_user": 423561145, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 423561145 }}, "user": { "id": 18083420, "name": "Laura McGinnis", "screen_name": "Txsleuthe", "lang": "en", "location": "Texas", "create_at": date("2008-12-12"), "description": "Republican Mom from DFW area. I watch C-Span to follow House and Senate. For the 2016 election, I am voting for Marco Rubio.", "followers_count": 3410, "friends_count": 2562, "statues_count": 204189 }, "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-06T10:00:06.000Z"), "id": 684796599374082049, "text": "I dunno Y kya pero kung pipili sana kung ano man yun yung pangmasa ndi namimili ng tao parangSiLiza @lizasoberano https://t.co/VtX6dkCmqX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 284291853 }}, "user": { "id": 2981689625, "name": "Tita Whiw", "screen_name": "TitaWhiw", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "God Bless the Good Hearts❤️", "followers_count": 393, "friends_count": 101, "statues_count": 12476 }, "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-06T10:00:06.000Z"), "id": 684796599906746368, "text": "@JaT_Gooru illegally", "in_reply_to_status": 684796561663102977, "in_reply_to_user": 1271421144, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3050055692 }}, "user": { "id": 1271421144, "name": "Matt Hallman", "screen_name": "matthewjhallman", "lang": "en", "location": "null", "create_at": date("2013-03-15"), "description": "Do it all for the big man upstairs ☝️ University of South Carolina '19 Psalms 23:4", "followers_count": 251, "friends_count": 186, "statues_count": 3905 }, "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-06T10:00:07.000Z"), "id": 684796600053579777, "text": "DOUBLE YOUR Social ad spend. Great recommendation from IDC #CES2016 #smm #advertising https://t.co/FoOn70okXj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CES2016", "smm", "advertising" }}, "user": { "id": 15066876, "name": "Jim Harris", "screen_name": "JimHarris", "lang": "en", "location": "Toronto, Canada", "create_at": date("2008-06-09"), "description": "Journalist at CES 2016! Tweeting on #Mobile, #IoT, Tech Innovation, Big Data Analytics, Cloud Computing & Smartphones. #1 Interntn'l Bestselling Author #CES2016", "followers_count": 142664, "friends_count": 92703, "statues_count": 25867 }, "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-06T10:00:07.000Z"), "id": 684796600409968641, "text": "Oakland Raiders are finally trending in the right direction https://t.co/oTaYhpGO91 #Raiders #NFL https://t.co/QUtqdfAWiC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.19952326,37.75171477"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Raiders", "NFL" }}, "user": { "id": 2764094358, "name": "Raider Nation UK", "screen_name": "RAIDERNATlON", "lang": "en", "location": "London, UK", "create_at": date("2014-08-24"), "description": "UK fans of the Oakland Raiders - Tweeting the latest news from #RaiderNation of the #NFL", "followers_count": 904, "friends_count": 894, "statues_count": 9598 }, "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-06T10:00:07.000Z"), "id": 684796600477216772, "text": "@akelleherauthor Discount available, non amazon paperbacks, 3 or more\nhttps://t.co/hmuXzwEpA9\nContact if interested https://t.co/QrJbFRIvvc", "in_reply_to_status": -1, "in_reply_to_user": 17960722, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17960722 }}, "user": { "id": 197017206, "name": "John Rataczak", "screen_name": "johnrataczak", "lang": "en", "location": "Toledo, OH", "create_at": date("2010-09-30"), "description": "John Rataczak, Ph.D. Teacher, pastor, now writing Christian books. http://www.eleutherosbooks.com", "followers_count": 12712, "friends_count": 14897, "statues_count": 20313 }, "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-06T10:00:07.000Z"), "id": 684796600795844608, "text": "@FabioWhat pijatela con @Uberbored_80", "in_reply_to_status": 684796397628059648, "in_reply_to_user": 284455533, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 284455533, 345020176 }}, "user": { "id": 395268713, "name": "pinupz", "screen_name": "pinupz", "lang": "it", "location": "null", "create_at": date("2011-10-21"), "description": "scrivo cazzate, non dico bugie.", "followers_count": 1088, "friends_count": 1354, "statues_count": 10958 }, "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-06T10:00:07.000Z"), "id": 684796600879714304, "text": "@calgarydreamer TERRRRYYYYY!! Happy new year. :) #bufferchat", "in_reply_to_status": 684796458424377344, "in_reply_to_user": 70693316, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "bufferchat" }}, "user_mentions": {{ 70693316 }}, "user": { "id": 1710801594, "name": "Christin Kardos", "screen_name": "ChristinKardos", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2013-08-29"), "description": "Beer & bacon analyst, ♥ #Marines, aka クリス / Tink, #Digital Cheerleader/ #CMGR w/ @convince, @SBizHour | Mod #CMGRHangout #CloudTalk", "followers_count": 15578, "friends_count": 11196, "statues_count": 77543 }, "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-06T10:00:07.000Z"), "id": 684796600925827072, "text": "It's #wednesday - let the countdown to the weekend begin! We figured we'd help get the party… https://t.co/VeXD7CCt88", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0440674,40.7212791"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wednesday" }}, "user": { "id": 1475209182, "name": "Talde JC", "screen_name": "TaldeJC", "lang": "en", "location": "8 Erie St, Jersey City, NJ", "create_at": date("2013-06-01"), "description": "Casual Asian-American Restaurant/Bar | Chef Dale Talde, David Massoni, & John Bush of Talde BK | Anthony Carrino & John Colaneri of Brunelleschi Construction", "followers_count": 1810, "friends_count": 745, "statues_count": 913 }, "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-06T10:00:07.000Z"), "id": 684796601143963649, "text": "1/6/2016 - 12:00\nTemp: 49.4F \nHum: 62%\nWind: 0.0 mph\nBaro: 30.260in. & 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": 105, "friends_count": 52, "statues_count": 48749 }, "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-06T10:00:07.000Z"), "id": 684796601219461120, "text": "Amazon prime gonna make me broke", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30090785, "name": "Sophia Rae", "screen_name": "SupSophia", "lang": "en", "location": "SEA | PHX | SAN", "create_at": date("2009-04-09"), "description": "Create. Exercise. Eat. Adventure. Love.", "followers_count": 218, "friends_count": 195, "statues_count": 6367 }, "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-06T10:00:07.000Z"), "id": 684796601248944129, "text": "How can I not", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 375719444, "name": "FREE MASTER TREEF", "screen_name": "TreefaDotCom", "lang": "en", "location": "Mid Air ", "create_at": date("2011-09-18"), "description": "||| F.R.E.E |||", "followers_count": 1887, "friends_count": 2196, "statues_count": 19087 }, "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-06T10:00:07.000Z"), "id": 684796601408163840, "text": "@Granberry__ You're ultimately right thought. Just be honest with ya lady. If she doesn't trust you do to lies. Its never the same", "in_reply_to_status": 684795349836300288, "in_reply_to_user": 3873257536, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3873257536 }}, "user": { "id": 3116796879, "name": "Fluke Skywalker", "screen_name": "Ashesofolympus", "lang": "en", "location": "SHE-EYE-CAGO", "create_at": date("2015-03-25"), "description": "I just want my beard as long as the Eiffle tower", "followers_count": 161, "friends_count": 150, "statues_count": 12273 }, "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-06T10:00:07.000Z"), "id": 684796601492111361, "text": "Extra sleeves please https://t.co/NJV5sGaEr6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 78177079, "name": "Jerard Ajami", "screen_name": "ajami30", "lang": "en", "location": "Dallas, TX", "create_at": date("2009-09-28"), "description": "Instagram: @ajami30 Email: Jajami30@gmail.com", "followers_count": 749, "friends_count": 393, "statues_count": 16696 }, "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-06T10:00:07.000Z"), "id": 684796601655771136, "text": "Afternoon thunderstorms this afternoon, high 67 (19 C). Low 55 (13 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712593, "name": "CorpusChristiWX", "screen_name": "CorpusChristiWX", "lang": "en", "location": "Corpus Christi, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Corpus Christi, Texas", "followers_count": 1013, "friends_count": 93, "statues_count": 8120 }, "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-06T10:00:07.000Z"), "id": 684796601689227264, "text": "Biggest snub of all https://t.co/6ASSG8RzIU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21457879, "name": "Richard Flores", "screen_name": "RFlores91", "lang": "en", "location": "Surprise,Arizona", "create_at": date("2009-02-20"), "description": "Hockey Geek. @Cronkite_ASU alum.Technical Director at KGME/KFYI.Former Play by play voice of @ASUACHAhockey. Sometimes you can hear me on @foxsports910.", "followers_count": 500, "friends_count": 759, "statues_count": 32021 }, "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-06T10:00:07.000Z"), "id": 684796601705992192, "text": "ON GOD 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": 2378391259, "name": "✨RICH BABY✨", "screen_name": "laritsunami", "lang": "en", "location": "yo daddy house", "create_at": date("2014-03-08"), "description": "null", "followers_count": 754, "friends_count": 316, "statues_count": 31516 }, "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-06T10:00:07.000Z"), "id": 684796601928294401, "text": "@bedard_lauren thank you ��", "in_reply_to_status": 684791469987598336, "in_reply_to_user": 2235348272, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2235348272 }}, "user": { "id": 863712260, "name": "Noga", "screen_name": "MichaelNoga6", "lang": "en", "location": "Andover Highschool - 17'", "create_at": date("2012-10-05"), "description": "BANG BROS | Baseball/Football #6️⃣. You Don't Score Until You Score", "followers_count": 1485, "friends_count": 933, "statues_count": 8405 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Andover, MN", "id": "219ccc645d458eee", "name": "Andover", "place_type": "city", "bounding_box": rectangle("-93.40681,45.21178 -93.265395,45.304142") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27003, "countyName": "Anoka", "cityID": 2701486, "cityName": "Andover" } }
+{ "create_at": datetime("2016-01-06T10:00:07.000Z"), "id": 684796602133966849, "text": "Love me forever don't never get tired of me, why tf would you lie 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": 336869688, "name": "K A Y O N C É♥️", "screen_name": "itsKaytoyou", "lang": "en", "location": "null", "create_at": date("2011-07-16"), "description": "| august 8 ♌️| #UWG19. I didn't choose the petty life, the petty life chose me", "followers_count": 972, "friends_count": 601, "statues_count": 38884 }, "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-06T10:00:07.000Z"), "id": 684796602666586112, "text": "I love this tweet https://t.co/n0epcOijzN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 314293810, "name": "jess", "screen_name": "jesssskautz", "lang": "en", "location": "null", "create_at": date("2011-06-09"), "description": "miley what's good", "followers_count": 391, "friends_count": 380, "statues_count": 8622 }, "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-06T10:00:07.000Z"), "id": 684796602884714496, "text": "@SweetHutBakery it sure was", "in_reply_to_status": 684796391898636288, "in_reply_to_user": 755760806, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 755760806 }}, "user": { "id": 2955755519, "name": "Bruce B", "screen_name": "iambigheadbruce", "lang": "en", "location": "Atlanta, GA", "create_at": date("2015-01-01"), "description": "null", "followers_count": 126, "friends_count": 308, "statues_count": 3655 }, "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-06T10:00:07.000Z"), "id": 684796602893135874, "text": "Omg so sexy https://t.co/r5Wf6N7p5F", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3085361298, "name": "Katie McCue", "screen_name": "KKMcCue", "lang": "en", "location": "CT h860", "create_at": date("2015-03-15"), "description": "Some lose all mind and become soul, insane. Some lose all soul and become mind, intellectual. Some lose both and become accepted.", "followers_count": 83, "friends_count": 85, "statues_count": 15593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wethersfield, CT", "id": "0e97d4804c3d8363", "name": "Wethersfield", "place_type": "city", "bounding_box": rectangle("-72.708224,41.66877 -72.617316,41.729879") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 984970, "cityName": "Wethersfield" } }
+{ "create_at": datetime("2016-01-06T10:00:07.000Z"), "id": 684796603052339200, "text": "@dinana_valencia happy birthday!! ��", "in_reply_to_status": -1, "in_reply_to_user": 634982819, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 634982819 }}, "user": { "id": 433110341, "name": "Michaela", "screen_name": "_mickyyousofine", "lang": "en", "location": "Chandler, AZ", "create_at": date("2011-12-09"), "description": "Love is patient, love is kind, love conquers all. 18", "followers_count": 1265, "friends_count": 675, "statues_count": 24788 }, "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-06T10:00:07.000Z"), "id": 684796603627106304, "text": "@AhmedBhatti20 ہر بچے کو سکول بجھوانا ممکن نہیں گھروں کا خرچہ پورا نہیں ہوتا اگر بچے بھی کام نہ کریں \nتعلیم کا فروغ غربت کے خاتمے سے جڑا ہے", "in_reply_to_status": 684794175133478912, "in_reply_to_user": 1024841660, "favorite_count": 0, "retweet_count": 0, "lang": "ur", "is_retweet": false, "user_mentions": {{ 1024841660 }}, "user": { "id": 151259045, "name": "عمران ملک", "screen_name": "imran_nomi", "lang": "en", "location": "null", "create_at": date("2010-06-02"), "description": "کورا کاغذ", "followers_count": 929, "friends_count": 232, "statues_count": 16428 }, "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-06T10:00:07.000Z"), "id": 684796603643736064, "text": ".@MeridianAudio system on the @intel stand at #CES2016 connected to the @jaguar F-Pace. https://t.co/nTfoyoaZdF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CES2016" }}, "user_mentions": {{ 28542546, 2803191, 349714440 }}, "user": { "id": 46404882, "name": "Matt Jones", "screen_name": "SoundMatt", "lang": "en", "location": "Portland, OR", "create_at": date("2009-06-11"), "description": "Director of Future Technology at Jaguar Land Rover; President of the GENIVI Alliance; Board Member of the Linux Foundation #allviewsaremyown", "followers_count": 343, "friends_count": 526, "statues_count": 189 }, "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-06T10:00:08.000Z"), "id": 684796604545548288, "text": "In my math class rn our teacher made us call into a radio station to win Journey tickets and one of us won???", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 437222891, "name": "Annie Hill", "screen_name": "Ann_Hill77", "lang": "en", "location": "Lake Oswego ", "create_at": date("2011-12-14"), "description": "what a day to be alive", "followers_count": 551, "friends_count": 284, "statues_count": 23929 }, "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-06T10:00:08.000Z"), "id": 684796604726009857, "text": "Glad my neighbors have a fork lift. Crate headed to Denver for ISE show next week, truck shows… https://t.co/S2lbT5gaO6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.199902,40.039617"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 179072285, "name": "SmithFly", "screen_name": "smithflydesigns", "lang": "en", "location": "Troy, Ohio", "create_at": date("2010-08-16"), "description": "Modular Fly Fishing Gear - Made in the USA", "followers_count": 2828, "friends_count": 2216, "statues_count": 4662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Troy, OH", "id": "14a79681d2bdaa7d", "name": "Troy", "place_type": "city", "bounding_box": rectangle("-84.262816,39.995246 -84.169631,40.082253") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39109, "countyName": "Miami", "cityID": 3977588, "cityName": "Troy" } }
+{ "create_at": datetime("2016-01-06T10:00:08.000Z"), "id": 684796604784627712, "text": "Cloudy this afternoon, high 43 (6 C). Low 40 (4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712706, "name": "TulsaWX", "screen_name": "TulsaWX", "lang": "en", "location": "Tulsa, Oklahoma", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Tulsa, Oklahoma", "followers_count": 517, "friends_count": 93, "statues_count": 8182 }, "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-06T10:00:08.000Z"), "id": 684796604818264065, "text": "One week they love me, the next week they hate me. Both weeks I got paid and that's all that's important", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 614119212, "name": "CaptainSplash", "screen_name": "BigEasyEMitch", "lang": "en", "location": "Zanarkand", "create_at": date("2012-06-20"), "description": "Listen to my story, this may be our last chance", "followers_count": 399, "friends_count": 130, "statues_count": 46857 }, "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-06T10:00:08.000Z"), "id": 684796604960866304, "text": "Oddly enough I'm happy to be heading back 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": 433153935, "name": "G-Baby", "screen_name": "GerDaJuiceMan", "lang": "en", "location": "Townsville", "create_at": date("2011-12-09"), "description": "Ger (Jair)", "followers_count": 1095, "friends_count": 905, "statues_count": 40669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lutz, FL", "id": "4b121d921ac7e762", "name": "Lutz", "place_type": "city", "bounding_box": rectangle("-82.502573,28.090958 -82.394598,28.172502") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1241775, "cityName": "Lutz" } }
+{ "create_at": datetime("2016-01-06T10:00:08.000Z"), "id": 684796604986048512, "text": "@sidneyjuzz what", "in_reply_to_status": -1, "in_reply_to_user": 1332792038, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1332792038 }}, "user": { "id": 1321394916, "name": "Brandon Grant", "screen_name": "grizzly__adamz", "lang": "en", "location": "null", "create_at": date("2013-04-01"), "description": "∞Sidney Dorothy Juzwiak∞", "followers_count": 296, "friends_count": 304, "statues_count": 1696 }, "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-06T10:00:08.000Z"), "id": 684796604998524928, "text": "Wind 7.0 mph SSE. Barometer 29.709 in, Steady. Temperature 36.6 °F. Rain today 0.26 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": 192, "friends_count": 58, "statues_count": 240541 }, "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-06T10:00:08.000Z"), "id": 684796605015314436, "text": "@megan_ranger @fettywap @fettywap @fettywap @fettywap", "in_reply_to_status": 684796297656705025, "in_reply_to_user": 420185676, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 420185676, 2601175671, 2601175671, 2601175671, 2601175671 }}, "user": { "id": 3035216263, "name": "YaLocalWhiteGirl", "screen_name": "Madisynmaree15", "lang": "en", "location": "under a ☁️ of marajuana ", "create_at": date("2015-02-21"), "description": "We are the new Americana, high on legal marijuana, raised on Biggie and Nirvanaaa Rest in paradise Page. -OSU-", "followers_count": 320, "friends_count": 206, "statues_count": 3176 }, "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-06T10:00:08.000Z"), "id": 684796605195792388, "text": "So u care ���� https://t.co/dS1rLJOuSs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2611080289, "name": "Justin", "screen_name": "Nelz_J1", "lang": "en", "location": "Fayettville ,NC", "create_at": date("2014-07-07"), "description": "Snap me @ swift_daily", "followers_count": 163, "friends_count": 423, "statues_count": 4077 }, "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-06T10:00:08.000Z"), "id": 684796605292228608, "text": "Definitely time for me to fallllll alllllllll the 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": 43272777, "name": "TMH", "screen_name": "DaLegendaryTati", "lang": "en", "location": "Pride Rock ✌", "create_at": date("2009-05-28"), "description": "Dreams do come true, if only we wish hard enough. You can have anything in life if you will sacrifice everything else for it.", "followers_count": 565, "friends_count": 530, "statues_count": 35025 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cranbury, NJ", "id": "010cea6f5a910a2b", "name": "Cranbury", "place_type": "city", "bounding_box": rectangle("-74.54946,40.288358 -74.476519,40.338744") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3415520, "cityName": "Cranbury" } }
+{ "create_at": datetime("2016-01-06T10:00:08.000Z"), "id": 684796605392916480, "text": "happy birthday aud! enjoy your day today @AudreyLayne1 ��❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1663078111 }}, "user": { "id": 2495304176, "name": "ss♛", "screen_name": "serena_suzansky", "lang": "en", "location": "null", "create_at": date("2014-05-14"), "description": "Matthew Collins has my heart", "followers_count": 499, "friends_count": 193, "statues_count": 2796 }, "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-01-06T10:00:08.000Z"), "id": 684796605552291840, "text": "Can't wait till this check drop ..Ima act 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": 352595425, "name": "Big Mama", "screen_name": "Zar_cane", "lang": "en", "location": "Jersey ✊ ", "create_at": date("2011-08-10"), "description": "You've gotta dance like there's nobody watching, Love like you'll never be hurt, Sing like there's nobody listening, And live like it's heaven on earth", "followers_count": 946, "friends_count": 910, "statues_count": 16912 }, "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-06T10:00:08.000Z"), "id": 684796605648642048, "text": "@BadassMemori MYDI I AM AT WORK", "in_reply_to_status": 684796394608148482, "in_reply_to_user": 3216564843, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3216564843 }}, "user": { "id": 3268559538, "name": "Misia || Jan 21", "screen_name": "branmuffinbob", "lang": "en", "location": "The Scorch", "create_at": date("2015-07-04"), "description": "#BobMorleyDefenseSquad/One time Bob Morley followed me on accident 9.14.15/Excessive Multishipper/(she/her)", "followers_count": 532, "friends_count": 404, "statues_count": 10998 }, "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-06T10:00:08.000Z"), "id": 684796605971755009, "text": "There are some good people left in the world. Even in high school. The trick is finding them.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.5357635,35.94815842"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1700079110, "name": "Pam Page", "screen_name": "pampagenc", "lang": "en", "location": "null", "create_at": date("2013-08-25"), "description": "null", "followers_count": 30, "friends_count": 38, "statues_count": 356 }, "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-01-06T10:00:08.000Z"), "id": 684796606156255232, "text": "@the_moviebob and they were doing so well too", "in_reply_to_status": 684477704557703168, "in_reply_to_user": 264050862, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 264050862 }}, "user": { "id": 1157543995, "name": "Joystik the Skeleton", "screen_name": "Joystik_thePony", "lang": "en", "location": "Pittsburgh", "create_at": date("2013-02-07"), "description": "Just a Pittsburgh Brony trying to make his way through this crazy world with good humor, snark and jetpacks. Sports Fanatic. Lover of Video Games and Comics.", "followers_count": 132, "friends_count": 330, "statues_count": 12407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bridgeville, PA", "id": "d944bf83412879e7", "name": "Bridgeville", "place_type": "city", "bounding_box": rectangle("-80.120691,40.348838 -80.087721,40.371893") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4208624, "cityName": "Bridgeville" } }
+{ "create_at": datetime("2016-01-06T10:00:08.000Z"), "id": 684796606592368640, "text": "@J_Bowers94 https://t.co/GfuGcuZTdl", "in_reply_to_status": -1, "in_reply_to_user": 255807228, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 255807228 }}, "user": { "id": 612377259, "name": "Megan Elizabeth", "screen_name": "meggs_21", "lang": "en", "location": "Lake Charles, LA", "create_at": date("2012-06-18"), "description": "McNeese State University Softball #34", "followers_count": 1003, "friends_count": 310, "statues_count": 49010 }, "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-06T10:00:08.000Z"), "id": 684796606739267584, "text": "Wind 2.0 mph SE. Barometer 30.345 in, Falling Rapidly. Temperature 39.0 °F. Rain today 0.00 in. Humidity 35%", "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": 3417 }, "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-06T10:00:08.000Z"), "id": 684796606802038784, "text": "Roaches everywhere like we forgot to take the trash out", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1589878459, "name": "Bradley Moore ", "screen_name": "_BreakingBrad_1", "lang": "en", "location": "null", "create_at": date("2013-07-12"), "description": "It's amazing I've made it this far", "followers_count": 158, "friends_count": 138, "statues_count": 4011 }, "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-06T10:00:08.000Z"), "id": 684796607443812352, "text": "This whole situation is just making it harder.��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 607116262, "name": "❁Cheree❁", "screen_name": "cherrybabe65", "lang": "en", "location": "Gilbert, AZ.", "create_at": date("2012-06-13"), "description": "null", "followers_count": 380, "friends_count": 133, "statues_count": 8187 }, "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-06T10:00:08.000Z"), "id": 684796607456391168, "text": "Got a new puppy ? Got see our experts this Saturday ! #petm0638 @walker5687 @CraigZ_petm https://t.co/kWwDD4XkGL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "petm0638" }}, "user_mentions": {{ 4181350704, 2462251038 }}, "user": { "id": 3951340513, "name": "Chantel Gary", "screen_name": "chantel_gary", "lang": "en", "location": "Lafayette, LA", "create_at": date("2015-10-19"), "description": "#petm0638 #inspiredbypets #care Customer Engagement Leader @0638 #LouisianaPetm", "followers_count": 70, "friends_count": 245, "statues_count": 128 }, "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-06T10:00:08.000Z"), "id": 684796607460585472, "text": "@LarenzTate to @Tyrese: \"O-Dog was America's worst nightmare. Jodie is just a baby boy.\" #lol", "in_reply_to_status": -1, "in_reply_to_user": 191204273, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "lol" }}, "user_mentions": {{ 191204273, 24621955 }}, "user": { "id": 2270843233, "name": "Silvestre Jaime", "screen_name": "M_E_L_68_62_75", "lang": "en", "location": "Fort Smith, AR since 4-28-97", "create_at": date("2013-12-31"), "description": "Lawn service member. Cinephile. Cat lover. Huge fan to war movies and @RealNancyAllen, @EMILIOTHEWAY, @LarenzTate, etc.", "followers_count": 1310, "friends_count": 1862, "statues_count": 7622 }, "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-06T10:00:08.000Z"), "id": 684796607485710336, "text": "Cloudy this afternoon, high 46 (8 C). Low 41 (5 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712679, "name": "OKCWX", "screen_name": "OKCWX", "lang": "en", "location": "Oklahoma City, Oklahoma", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Oklahoma City, Oklahoma", "followers_count": 186, "friends_count": 93, "statues_count": 8209 }, "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-06T10:00:08.000Z"), "id": 684796607573827585, "text": "@GhostLikeSwayz_ https://t.co/H3TXcmWKO8", "in_reply_to_status": 684794797119418368, "in_reply_to_user": 544814345, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 544814345 }}, "user": { "id": 353551043, "name": "Nightcrawler", "screen_name": "Eddddddd564", "lang": "en", "location": "RIVERSIDE ON OCCASIONS ", "create_at": date("2011-08-12"), "description": "UC Riverside! | SD 619| live | love | laugh | Suave Bros. Inc.|", "followers_count": 686, "friends_count": 422, "statues_count": 18550 }, "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-06T10:00:08.000Z"), "id": 684796607682879489, "text": "ITS KEITH DE BRUYNE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "nl", "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": 830, "friends_count": 653, "statues_count": 15674 }, "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-06T10:00:08.000Z"), "id": 684796608207138817, "text": "There is something truly truly annoying about people who buy nice cars and can't even friggen drive them!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3068473390, "name": "Shiva Sharifi", "screen_name": "AllShivaSharifi", "lang": "en", "location": "SF Bay Area", "create_at": date("2015-03-03"), "description": "AllShiva Photography ~ Creative Director, Digital Media Marketing, Motorcycle Babe, Dreamer, Dancer ~ Your Angel ~", "followers_count": 493, "friends_count": 449, "statues_count": 613 }, "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-06T10:00:08.000Z"), "id": 684796608341504001, "text": "You know you have a cool teacher when the first thing he walks in and says is \"whats up mothafuckas\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 441204769, "name": "Kłaus", "screen_name": "klausbeckk", "lang": "en", "location": "Raleigh, NC", "create_at": date("2011-12-19"), "description": "Just grindin'", "followers_count": 368, "friends_count": 177, "statues_count": 2996 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fuquay-Varina, NC", "id": "000b2b6f4a781bba", "name": "Fuquay-Varina", "place_type": "city", "bounding_box": rectangle("-78.844741,35.560069 -78.697863,35.670874") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3725300, "cityName": "Fuquay-Varina" } }
+{ "create_at": datetime("2016-01-06T10:00:09.000Z"), "id": 684796608714768384, "text": "@loqueeeee @jsc905 @JesseMalowitz I know", "in_reply_to_status": 684796546404200450, "in_reply_to_user": 2642199735, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2642199735, 196001502, 173146781 }}, "user": { "id": 500485708, "name": "Vinson Smith, Ph.D", "screen_name": "MrVincredible", "lang": "en", "location": "NY", "create_at": date("2012-02-22"), "description": "#AirForce #JohnJayCollege #Jets #ChelseaFC #NYCFC #LosKnicks #Yankees #UGADawgs", "followers_count": 1243, "friends_count": 663, "statues_count": 194926 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chester, NY", "id": "00f622ad4940616f", "name": "Chester", "place_type": "city", "bounding_box": rectangle("-74.310835,41.341301 -74.249668,41.382249") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36071, "countyName": "Orange", "cityID": 3615297, "cityName": "Chester" } }
+{ "create_at": datetime("2016-01-06T10:00:09.000Z"), "id": 684796609096331264, "text": "@Pie_SocialMedia @MomsDemand @Everytown Guns don't kill people do. Crack down on criminals with guns do not render good ppl helpless", "in_reply_to_status": 684791473599033344, "in_reply_to_user": 2191568058, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2191568058, 1017637447, 250386727 }}, "user": { "id": 70940944, "name": "Rusti McCollum", "screen_name": "RUSTIMCCOLLUM", "lang": "en", "location": "Corbett,OREGON", "create_at": date("2009-09-02"), "description": "I am many things, a complex woman for sure! :)", "followers_count": 1076, "friends_count": 2391, "statues_count": 5750 }, "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-06T10:00:09.000Z"), "id": 684796609352171520, "text": "First day of winter term ✔️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1240496389, "name": "Jesse Delgado", "screen_name": "follow_jesse", "lang": "en", "location": "SE", "create_at": date("2013-03-03"), "description": "Life's a risk carnal", "followers_count": 484, "friends_count": 290, "statues_count": 11056 }, "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-06T10:00:09.000Z"), "id": 684796609503215616, "text": "Wind 1.5 mph NE. Barometer 30.149 in, Falling. Temperature 49.7 °F. Rain today 0.00in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.11555556,29.6725"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25210383, "name": "New Braunfels WX", "screen_name": "jeffreycentexwx", "lang": "en", "location": "New Braunfels, TX", "create_at": date("2009-03-18"), "description": "Weather Station located on County Line Road in New Braunfels, TX.\r\n\r\nRealtime Weather @ http://www.nbtx-wx.com\r\nFollow me @ http://twitter.com/jeffreycentex", "followers_count": 379, "friends_count": 24, "statues_count": 160190 }, "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-01-06T10:00:09.000Z"), "id": 684796609704673281, "text": "I got Hulu, Netflix and Amazon Prime baby, fuck with a true boss ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 301699308, "name": "Mean Greeze Okerlund", "screen_name": "DeeGreezy", "lang": "en", "location": "Burning the mistletoe", "create_at": date("2011-05-19"), "description": "Legend. Flowing straight from the survivor scroll.", "followers_count": 1177, "friends_count": 1027, "statues_count": 153027 }, "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-06T10:00:09.000Z"), "id": 684796609851461634, "text": "Greetings from Kalamazoo, MI! Have a great day peeps! https://t.co/20sSfFJx6Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 236177149, "name": "Michael Vasquez", "screen_name": "Michael4jc", "lang": "en", "location": "Imlay City, MI", "create_at": date("2011-01-09"), "description": "Love life, work hard, place others first, get fit..your life is waiting for you!!!", "followers_count": 231, "friends_count": 344, "statues_count": 5039 }, "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-06T10:00:09.000Z"), "id": 684796609859723264, "text": "interns first day! here's some valuable advice. @ New York, New York https://t.co/02pk9gVOFz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0064,40.7142"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 135690877, "name": "becky", "screen_name": "bhabs37", "lang": "en", "location": "nyc ", "create_at": date("2010-04-21"), "description": "talk hockey to me • insta: bhabs37 ✌️", "followers_count": 1877, "friends_count": 396, "statues_count": 41800 }, "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-06T10:00:09.000Z"), "id": 684796609935323136, "text": "@AdamSchefter @kbarlowkstp @Vikings Interesting.", "in_reply_to_status": 684791950311047172, "in_reply_to_user": 51263592, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 51263592, 17502416, 25545388 }}, "user": { "id": 1192032888, "name": "Avery Williams", "screen_name": "averywilliams95", "lang": "en", "location": "Kentwood, Michigan. ", "create_at": date("2013-02-17"), "description": "Married,44, English/Spanish speaker.Bass player, Geography geek, confirmed insomniac, Native Houston, Texan livin' in Michigan. Proud music junkie!!!", "followers_count": 498, "friends_count": 735, "statues_count": 13359 }, "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-01-06T10:00:09.000Z"), "id": 684796610178625538, "text": "@TakeGawd meh. They can be shut down . A horrendous Baltimore team did it.", "in_reply_to_status": 684796489156182017, "in_reply_to_user": 439227673, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 439227673 }}, "user": { "id": 2475822420, "name": "Knights of Ren", "screen_name": "JacobReznik", "lang": "en", "location": "Michigan", "create_at": date("2014-05-03"), "description": "I have no problem with that", "followers_count": 60, "friends_count": 107, "statues_count": 10806 }, "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-01-06T10:00:09.000Z"), "id": 684796610295939072, "text": "Afternoon rain this afternoon, high 57 (14 C). Low 49 (9 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712488, "name": "SanAntonioWX", "screen_name": "SanAntonioWX", "lang": "en", "location": "San Antonio, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for San Antonio, Texas", "followers_count": 251, "friends_count": 93, "statues_count": 8199 }, "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-06T10:00:09.000Z"), "id": 684796610384035841, "text": "I say the dumbest things when 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": 1736045779, "name": "Mandy Lou Graham", "screen_name": "MandylouGraham", "lang": "en", "location": "Biggest State In The South", "create_at": date("2013-09-06"), "description": "The Lord works in mysterious ways. Be yourself. Songwriter•Future DallasPD officer•Trial junkie #TexasCountry #CountryMusic #AutismAcceptance #AutismAwareness", "followers_count": 1015, "friends_count": 1867, "statues_count": 18314 }, "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-06T10:00:09.000Z"), "id": 684796610677706752, "text": "'If at first u don't succeed, dust yourself off and try again' ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27164206, "name": "Aisha Nicole", "screen_name": "1_AISHANICOLE", "lang": "en", "location": "where i wanna be.....", "create_at": date("2009-03-27"), "description": "Everything happens for a reason.............", "followers_count": 1135, "friends_count": 384, "statues_count": 8431 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milton, GA", "id": "00534aed096d7799", "name": "Milton", "place_type": "city", "bounding_box": rectangle("-84.388334,34.083082 -84.244552,34.186143") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1351670, "cityName": "Milton" } }
+{ "create_at": datetime("2016-01-06T10:00:09.000Z"), "id": 684796610837000193, "text": "@PinkTropic_ @TopFlightPrez thank you! ��", "in_reply_to_status": 684565164335628289, "in_reply_to_user": 319886927, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 319886927, 274574183 }}, "user": { "id": 2792268096, "name": "Greene", "screen_name": "AllanahBrenae", "lang": "en", "location": "But God!", "create_at": date("2014-09-05"), "description": "Medical laboratory sciences major. Chemistry minor. Love enthusiast. #UNT", "followers_count": 1135, "friends_count": 815, "statues_count": 27586 }, "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-06T10:00:09.000Z"), "id": 684796610908389376, "text": "I'm at @TopsMarketS in Erie, PA https://t.co/b7XhV8EIMh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.03052225,42.11697217"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1454121961 }}, "user": { "id": 22919399, "name": "Mike Sopp", "screen_name": "HZHackenbush", "lang": "en", "location": "Erie , PA", "create_at": date("2009-03-05"), "description": "IF you like me on Twitter you will love me on Facebook.", "followers_count": 659, "friends_count": 1868, "statues_count": 10694 }, "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-06T10:00:09.000Z"), "id": 684796610996506624, "text": "@KeelyOvO can I see some ID please", "in_reply_to_status": 684795278738747392, "in_reply_to_user": 526954405, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 526954405 }}, "user": { "id": 465235740, "name": "January 6ixth", "screen_name": "2Girls1Carp", "lang": "en", "location": "bfe", "create_at": date("2012-01-15"), "description": "SFHS/DACC University", "followers_count": 634, "friends_count": 290, "statues_count": 24655 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Catlin, IL", "id": "0186df171d72b3f8", "name": "Catlin", "place_type": "city", "bounding_box": rectangle("-87.719767,40.057802 -87.658315,40.101726") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17183, "countyName": "Vermilion", "cityID": 1711774, "cityName": "Catlin" } }
+{ "create_at": datetime("2016-01-06T10:00:09.000Z"), "id": 684796611071873025, "text": "I just have this idea of a suicide bomber running into a crowd screaming and confetti and glitter shoots everywhere instead of kaboom.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1685739595, "name": "Trouserless Tigerfox", "screen_name": "tigerfoxtail", "lang": "en", "location": "Anchorage, AK", "create_at": date("2013-08-20"), "description": "the totally nsfw account of a silly tigerfox.\n\nMain: @TheAltaico\n\nthe guy Im dating: @CrevanFawxAD", "followers_count": 1640, "friends_count": 1686, "statues_count": 15668 }, "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-06T10:00:09.000Z"), "id": 684796611072004096, "text": "I want fish yams bake Mac and corn bread ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 117615159, "name": "The Ląшγεг Girl", "screen_name": "KXIVB_", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2010-02-25"), "description": "Molding The Next Generation | Temple U.♕♕ I ♥ politics", "followers_count": 1085, "friends_count": 686, "statues_count": 64966 }, "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-06T10:00:09.000Z"), "id": 684796611260592128, "text": "The CHINESE Will CONTROL ASIA \n\nAmerica Military Only thing SAVING\n#JAPAN #TAIWAN #SouthKorea #AUSTRALIA FOR NOW\n\nbut What happens in 2035", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "JAPAN", "TAIWAN", "SouthKorea", "AUSTRALIA" }}, "user": { "id": 41012342, "name": "Gary the numbers guy", "screen_name": "Numerologynow", "lang": "en", "location": "Cleveland/LAS VEGAS/Ufa,Russia", "create_at": date("2009-05-18"), "description": "33 life path - ELITE NUMEROLOGIST ASTROLOGER-SPORTS HANDICAPPER TEACHER-HISTORIAN-DAY-TRADER T / ANTI-COMMUNIST / Created #GG33", "followers_count": 7237, "friends_count": 998, "statues_count": 53714 }, "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-06T10:00:09.000Z"), "id": 684796611688415233, "text": "Esque somos Contreras @danzbrb .. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user_mentions": {{ 64124433 }}, "user": { "id": 140647979, "name": "Rosalía", "screen_name": "Rosalily_", "lang": "en", "location": "Fort Worth, Texas", "create_at": date("2010-05-05"), "description": "null", "followers_count": 426, "friends_count": 398, "statues_count": 13219 }, "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-06T10:00:09.000Z"), "id": 684796612204347392, "text": "Having no friends is like https://t.co/1zaA5hkDRX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1383133171, "name": "victoria hernandez", "screen_name": "v_h1234", "lang": "en", "location": "null", "create_at": date("2013-04-26"), "description": "God has a plan for me samuel 22:33 veni vidi vici", "followers_count": 268, "friends_count": 221, "statues_count": 6682 }, "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-06T10:00:09.000Z"), "id": 684796612384813056, "text": "Hair appt Friday tho��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2950718351, "name": "Teneille.", "screen_name": "teneillegrimes", "lang": "en", "location": "ATL ", "create_at": date("2014-12-29"), "description": "20.Tuskegee.", "followers_count": 415, "friends_count": 344, "statues_count": 6935 }, "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-01-06T10:00:09.000Z"), "id": 684796612393177088, "text": "I decided this will be the year I get my shit together and work out everyday.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 392767200, "name": "Merry Chauvin", "screen_name": "May_Chauvin", "lang": "en", "location": "Houma, LA", "create_at": date("2011-10-17"), "description": "Makeup artist. Perpetually broke. I feel very strongly about soup and cat butts. snap: @rocky991", "followers_count": 175, "friends_count": 700, "statues_count": 3216 }, "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-01-06T10:00:09.000Z"), "id": 684796612552470530, "text": "#tweetavideothatyoucantexplain https://t.co/UHF2bytHDj", "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": 259570778, "name": "øśçår", "screen_name": "Oscar_Benitez13", "lang": "en", "location": "Garden Grove ", "create_at": date("2011-03-01"), "description": "Second of all, I'm not a rapper, so stop rappin' at me.", "followers_count": 171, "friends_count": 186, "statues_count": 595 }, "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-06T10:00:10.000Z"), "id": 684796612615380993, "text": "SMOKY QUARTZ \nHelps to activate your survival instincts, grounds light force into your physical… https://t.co/MWT5UTajDR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.9651794,32.2264099"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2192478272, "name": "Celestial Rites", "screen_name": "jennifer_kraych", "lang": "en", "location": "543 N. 4th Ave Tucson Arizona ", "create_at": date("2013-11-13"), "description": "Professional tarot reader for seventeen years local business co-owner of Celestial Rites 420 East 7th Street Tucson Arizona 85705 (520) 344-4203", "followers_count": 19, "friends_count": 8, "statues_count": 626 }, "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-06T10:00:10.000Z"), "id": 684796612699226112, "text": "@perlavianney_ I know we really need to!", "in_reply_to_status": 684787097559896064, "in_reply_to_user": 2397516992, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2397516992 }}, "user": { "id": 2995087554, "name": "Allie♡", "screen_name": "MisssAlliee", "lang": "en", "location": "null", "create_at": date("2015-01-24"), "description": "Now this is a story all about how my life got flipped-turned upside down...\n ♡ R.I.P Christian ♡", "followers_count": 42, "friends_count": 167, "statues_count": 1295 }, "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-06T10:00:10.000Z"), "id": 684796612854456323, "text": "@eonline check out our #SisterhoodOfTheTravelingPants trivia quiz! https://t.co/aGK0Syn7hh", "in_reply_to_status": 684234390264803328, "in_reply_to_user": 2883841, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SisterhoodOfTheTravelingPants" }}, "user_mentions": {{ 2883841 }}, "user": { "id": 377844134, "name": "women.com", "screen_name": "womendotcom", "lang": "en", "location": "Los Angeles, California", "create_at": date("2011-09-21"), "description": "The ultimate destination for women", "followers_count": 3192, "friends_count": 1228, "statues_count": 7339 }, "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-06T10:00:10.000Z"), "id": 684796613047496704, "text": "What ever you get, get it fully involved! (at @FirehouseSubs in Colonial Heights, VA) https://t.co/6XYWQKCCuy https://t.co/pGlds4Zd5f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.39404095,37.25492917"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 22837424 }}, "user": { "id": 14703986, "name": "Matt Yesbeck", "screen_name": "Cscapes", "lang": "en", "location": "Richmond, Va", "create_at": date("2008-05-08"), "description": "I.T. Manager in RVA, follower of Christ, musician, photographer & Disney geek who loves every day of summer!", "followers_count": 157, "friends_count": 159, "statues_count": 4667 }, "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-06T10:00:10.000Z"), "id": 684796613122887681, "text": "@2laude4ya @whoaitsvanessa @Jaszhane @SimpMasterFlex_ ������������������", "in_reply_to_status": 684795728737230848, "in_reply_to_user": 1177794972, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1177794972, 547086519, 253242267, 990261242 }}, "user": { "id": 274310795, "name": "★ Swaggy C★", "screen_name": "Dub2_Breezy", "lang": "en", "location": "Hawaii ✈️ Chicago", "create_at": date("2011-03-29"), "description": "NIU student x Musician", "followers_count": 1222, "friends_count": 759, "statues_count": 24539 }, "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-06T10:00:10.000Z"), "id": 684796613215191041, "text": "Im alreadi Hesitant to start with this Macc cuz she fulla shitt too��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2748456570, "name": "Daddie'MACC", "screen_name": "DaddieNiqNiq_", "lang": "en", "location": "null", "create_at": date("2014-08-20"), "description": "✊", "followers_count": 1117, "friends_count": 1822, "statues_count": 3650 }, "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-06T10:00:10.000Z"), "id": 684796614066704384, "text": "Recently moved or changed your e-mail? Let us know! Update your info here: https://t.co/fk7RZhCr6c", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 386106810, "name": "Southern Scholarship", "screen_name": "SSFTweets", "lang": "en", "location": "null", "create_at": date("2011-10-06"), "description": "For over 50 yrs, Southern Scholarship Foundation has helped students with outstanding academic abilities, good character, & financial need attend universities.", "followers_count": 1049, "friends_count": 2013, "statues_count": 3879 }, "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-06T10:00:10.000Z"), "id": 684796614138032128, "text": "@d_rullo aren't you working ?", "in_reply_to_status": 684794360127459328, "in_reply_to_user": 199299328, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 199299328 }}, "user": { "id": 93121974, "name": "EricaRullo", "screen_name": "EricaRullo", "lang": "en", "location": "null", "create_at": date("2009-11-27"), "description": "ॐ☮☯|♡|&bitchyness ☪", "followers_count": 212, "friends_count": 128, "statues_count": 45455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forked River, NJ", "id": "00d88eb470b73bd2", "name": "Forked River", "place_type": "city", "bounding_box": rectangle("-74.223558,39.809802 -74.143,39.879125") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean", "cityID": 3424180, "cityName": "Forked River" } }
+{ "create_at": datetime("2016-01-06T10:00:10.000Z"), "id": 684796614188244993, "text": "Afternoon rain this afternoon, high 50 (10 C). Low 47 (8 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712452, "name": "FortWorthWX", "screen_name": "FortWorthWX", "lang": "en", "location": "Fort Worth, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Fort Worth, Texas", "followers_count": 217, "friends_count": 93, "statues_count": 8186 }, "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-06T10:00:10.000Z"), "id": 684796614242885632, "text": "Cold weather comfort, burger and beer ������ #billiessaloon #eatwell #goodfood @ Billie's 1890 Saloon https://t.co/wxzrdbsKvn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.06797117,40.94519922"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "billiessaloon", "eatwell", "goodfood" }}, "user": { "id": 277084828, "name": "Cat Troiano", "screen_name": "CatTroianoLI", "lang": "en", "location": "null", "create_at": date("2011-04-04"), "description": "Writer. Gourmet cook. Animal lover. Hiking enthusiast. Admirer of all things French and Italian.", "followers_count": 63, "friends_count": 141, "statues_count": 970 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Jefferson, NY", "id": "70d4a44111ba3c94", "name": "Port Jefferson", "place_type": "city", "bounding_box": rectangle("-73.082446,40.933025 -73.037902,40.966636") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3659355, "cityName": "Port Jefferson" } }
+{ "create_at": datetime("2016-01-06T10:00:10.000Z"), "id": 684796614557315072, "text": "Don't feel well", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1960014500, "name": "Jackie Arevalo", "screen_name": "jackiearevalo13", "lang": "en", "location": "null", "create_at": date("2013-10-13"), "description": "null", "followers_count": 251, "friends_count": 819, "statues_count": 2216 }, "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-06T10:00:10.000Z"), "id": 684796614662205440, "text": "My dreams are always trips.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 583311482, "name": "lariza", "screen_name": "lariza_f", "lang": "en", "location": "Minneapolis, MN", "create_at": date("2012-05-17"), "description": "null", "followers_count": 528, "friends_count": 327, "statues_count": 1595 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Robbinsdale, MN", "id": "25174d4b31d9a511", "name": "Robbinsdale", "place_type": "city", "bounding_box": rectangle("-93.356269,45.006048 -93.318293,45.041009") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2754808, "cityName": "Robbinsdale" } }
+{ "create_at": datetime("2016-01-06T10:00:10.000Z"), "id": 684796614679113728, "text": "JUST DANCE SATURDAYS\n@ BAROQUE\nThe ultimate Nightclub in Queens with the...\nSOUND SYSTEM , LIGHT… https://t.co/KpTxNqFf01", "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": 25, "friends_count": 10, "statues_count": 1458 }, "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-06T10:00:10.000Z"), "id": 684796614712684545, "text": "As we finish out the first full week of the New Year, we'd like to wish you all the best for… https://t.co/34EiaBCEou", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.41083333,41.82361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3623938582, "name": "Lore", "screen_name": "Studio_Lore", "lang": "en", "location": "Providence, RI", "create_at": date("2015-09-11"), "description": "Lore is a curated collection of hand crafted art & design objects ranging from fine art to furniture to jewelry, textiles, and apparel. Located in The Arcade.", "followers_count": 24, "friends_count": 57, "statues_count": 55 }, "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-06T10:00:10.000Z"), "id": 684796614741852160, "text": "Wtf https://t.co/mK1KxSdcqJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2419497853, "name": "$tuss.", "screen_name": "LLUXURYXQUE", "lang": "en", "location": "null", "create_at": date("2014-03-30"), "description": "Most Bitches In The World Are Males", "followers_count": 284, "friends_count": 30, "statues_count": 2026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-06T10:00:10.000Z"), "id": 684796614938996736, "text": "Rg3 is overrated B https://t.co/slnfn7uGN0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 477696706, "name": "Stanlee B", "screen_name": "ArtByLey", "lang": "en", "location": "Painting Naked wit yo BM!", "create_at": date("2012-01-29"), "description": "#UnorthodoxAbstractArtist #ArtistWithSubstance follow my Ig @stanlee_", "followers_count": 83, "friends_count": 196, "statues_count": 1062 }, "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-01-06T10:00:10.000Z"), "id": 684796615010336769, "text": "It's been a nice little switch up from the sweet tea I am so use to. And I had my first cup of coffee of the new year today!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20421475, "name": "Ware is the Vodka?!?", "screen_name": "sevibabyyy", "lang": "en", "location": "Dallas, Texas", "create_at": date("2009-02-08"), "description": "Sevi | Influencer | Dallas Lifestyle Blogger | Crime fighter by day, Vodka drinker by night!", "followers_count": 976, "friends_count": 1385, "statues_count": 10348 }, "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-06T10:00:10.000Z"), "id": 684796615178125313, "text": "Just about ev... Nevermind https://t.co/yPoBqHLHRT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 257276567, "name": "Optimus, K-9", "screen_name": "_IAintShitSon", "lang": "en", "location": "Kanto Region 757", "create_at": date("2011-02-24"), "description": "You niggas can't be this fucking stupid. #DogHiveCEO", "followers_count": 5004, "friends_count": 1950, "statues_count": 200774 }, "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-06T10:00:10.000Z"), "id": 684796615215845376, "text": "@Manderslicious What was the matter with it?", "in_reply_to_status": 684796450996269056, "in_reply_to_user": 2414876328, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2414876328 }}, "user": { "id": 426089231, "name": "Annique Fenlason", "screen_name": "montanamommie", "lang": "en", "location": "Montana", "create_at": date("2011-12-01"), "description": "I'm a mom, I'm single, and I'm a born again Christian!!", "followers_count": 175, "friends_count": 210, "statues_count": 6803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Four Corners, MT", "id": "0080d1bd31076720", "name": "Four Corners", "place_type": "city", "bounding_box": rectangle("-111.215646,45.638329 -111.165012,45.689635") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30031, "countyName": "Gallatin", "cityID": 3028862, "cityName": "Four Corners" } }
+{ "create_at": datetime("2016-01-06T10:00:10.000Z"), "id": 684796615433916416, "text": "I'm not angel myself but I know what I can bring to the table and what i would/could change about me", "in_reply_to_status": 684793972712013824, "in_reply_to_user": 427606931, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "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": 2332, "friends_count": 855, "statues_count": 69791 }, "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-06T10:00:10.000Z"), "id": 684796615719190530, "text": "you you you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3916438453, "name": "☮R☮", "screen_name": "rheynaa___", "lang": "en", "location": "Vernon, TX", "create_at": date("2015-10-16"), "description": "null", "followers_count": 214, "friends_count": 156, "statues_count": 637 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vernon, TX", "id": "2fb1a91a3ceee124", "name": "Vernon", "place_type": "city", "bounding_box": rectangle("-99.332828,34.121343 -99.263514,34.167119") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48487, "countyName": "Wilbarger", "cityID": 4875308, "cityName": "Vernon" } }
+{ "create_at": datetime("2016-01-06T10:00:10.000Z"), "id": 684796615744450564, "text": "Work work work, no play ��������#workflow #cutconcrete #applecoringandsawing #ACS… https://t.co/IJowvYoNGS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.60320387,40.35031476"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "workflow", "cutconcrete", "applecoringandsawing", "ACS" }}, "user": { "id": 3192391489, "name": "John Yendrick II", "screen_name": "burly_j", "lang": "en", "location": "null", "create_at": date("2015-05-11"), "description": "Burly bearded bastard. I brew beer; ride motorcycles; beard; and apparently tweet.", "followers_count": 50, "friends_count": 95, "statues_count": 125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plainsboro, NJ", "id": "01448c9d3cdcbe70", "name": "Plainsboro", "place_type": "city", "bounding_box": rectangle("-74.630416,40.299558 -74.565011,40.365216") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34021, "countyName": "Mercer" } }
+{ "create_at": datetime("2016-01-06T10:00:10.000Z"), "id": 684796616033697792, "text": "@Vsilly @PotentFlynn believe me I'm over not being able to use water for an hour of the day", "in_reply_to_status": 684795996790915072, "in_reply_to_user": 173503857, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 173503857, 164389456 }}, "user": { "id": 125449843, "name": "Danny Pollack", "screen_name": "dpfromdc", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-03-22"), "description": "It's all happening", "followers_count": 860, "friends_count": 823, "statues_count": 10007 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:10.000Z"), "id": 684796616465747969, "text": "That jam band that only jam band fans care about", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 187343681, "name": "♡ Li'l Funyuns ♡", "screen_name": "radrenard", "lang": "en", "location": "Roseville, MI", "create_at": date("2010-09-05"), "description": "as if", "followers_count": 418, "friends_count": 310, "statues_count": 42301 }, "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-06T10:00:11.000Z"), "id": 684796616880988160, "text": "Afternoon rain this afternoon, high 49 (9 C). Low 47 (8 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712424, "name": "DallasWX", "screen_name": "DallasWX", "lang": "en", "location": "Dallas, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Dallas, Texas", "followers_count": 194, "friends_count": 93, "statues_count": 8163 }, "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-06T10:00:11.000Z"), "id": 684796616948121600, "text": "Love the weather bih", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 116292764, "name": "black_tito", "screen_name": "TitoBlackent", "lang": "en", "location": "N.O. , Louisana", "create_at": date("2010-02-21"), "description": "#titoblackthebarber and #titoblackent Working hard and staying humble . Barber & Promoter", "followers_count": 749, "friends_count": 1456, "statues_count": 4578 }, "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-06T10:00:11.000Z"), "id": 684796616998543360, "text": "Feeling like Summer Baby https://t.co/LrfrO0U3Th", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 543264178, "name": "VondaKayHolder News", "screen_name": "BlacangelNews", "lang": "en", "location": "Atlanta,GA", "create_at": date("2012-04-02"), "description": "@Xmas_blacangel @VondaKayAngel @VondaKayHolder @LoveBlacangel @VondaBlacangel @BlacangelVonda @VondaKay_Marley @VondaKayPalmer @JQ340_TCMG", "followers_count": 229, "friends_count": 1662, "statues_count": 119582 }, "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-06T10:00:11.000Z"), "id": 684796617191370752, "text": ".@DennisPrager U R so dishonest! 2 contrast @TedCruz & \"the poison pill\" amendment \"back then\" w/ Rubio's position NOW on immigration!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17091571, 23022687 }}, "user": { "id": 55348286, "name": "Hawkeye CruzCrew", "screen_name": "inhuggermugger", "lang": "en", "location": "Jerusalem!", "create_at": date("2009-07-09"), "description": "Defend Israel Defeat IslamoNazism; History & Hawkeye's View Daily; Music @RomanceNSorrow You-tube: http://goo.gl/XSKXP0 FB Fan Page http://on.fb.me/14Hj3Dv", "followers_count": 17121, "friends_count": 14115, "statues_count": 219984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:11.000Z"), "id": 684796617237499909, "text": "Rain Rain Rain en el sur de Calif", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 188108412, "name": "Efren Carvajal", "screen_name": "llgemini2010", "lang": "en", "location": "usa", "create_at": date("2010-09-07"), "description": "null", "followers_count": 125, "friends_count": 674, "statues_count": 162 }, "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-06T10:00:11.000Z"), "id": 684796617455579136, "text": "Omg....something terrible has happened can u guys pray 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": 2578969358, "name": "Leia Organa", "screen_name": "giannasty", "lang": "en", "location": "Carson, CA", "create_at": date("2014-06-20"), "description": "♏️✨• probably watching that 70s show", "followers_count": 703, "friends_count": 263, "statues_count": 40947 }, "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-06T10:00:11.000Z"), "id": 684796618495885314, "text": "there was a shooting on G street & vineyard .. I live on 4th & vineyard .. how crazy .. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 88740105, "name": "Big Neech", "screen_name": "_Nichita", "lang": "en", "location": "null", "create_at": date("2009-11-09"), "description": "MUA// June 2016✨", "followers_count": 2424, "friends_count": 2173, "statues_count": 114263 }, "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-06T10:00:11.000Z"), "id": 684796618600677377, "text": "Free food tastes 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": 628981940, "name": "Tyler Musil™", "screen_name": "tylermusil52", "lang": "en", "location": "623, AZ", "create_at": date("2012-07-06"), "description": "LNH BW134 CJD | #Packernation | Snap: tylermusil", "followers_count": 1047, "friends_count": 426, "statues_count": 11253 }, "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-06T10:00:11.000Z"), "id": 684796618688872449, "text": "Of course I peep shit, but I ain't gone make u famous", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 131350197, "name": "January 7th™", "screen_name": "TooReal_DotCom", "lang": "en", "location": "Florida, USA", "create_at": date("2010-04-09"), "description": "Juice™.", "followers_count": 608, "friends_count": 509, "statues_count": 14073 }, "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-01-06T10:00:11.000Z"), "id": 684796618718117888, "text": "My pets are my everything. Unfortunately, yesterday I lost p", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 527012788, "name": "Norms", "screen_name": "_Norma12345", "lang": "en", "location": "Albuquerque", "create_at": date("2012-03-16"), "description": "love love love", "followers_count": 420, "friends_count": 316, "statues_count": 8984 }, "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-06T10:00:11.000Z"), "id": 684796619431227393, "text": "Yassss ������ https://t.co/70iTUrfXgX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3309195902, "name": "Lilia Whittington", "screen_name": "LiliaW18", "lang": "en", "location": "Knoxville, TN", "create_at": date("2015-08-07"), "description": "|Swimming||Nick||Jesus|", "followers_count": 83, "friends_count": 272, "statues_count": 301 }, "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-06T10:00:11.000Z"), "id": 684796619573870593, "text": "People keep complaining about their personal life, move on. Niggas & bitches come & go. It's LIFE shit happens ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 536438969, "name": "❣", "screen_name": "__forevaSHEY", "lang": "en", "location": "null", "create_at": date("2012-03-25"), "description": "#umes19 #SlickD_Banging✨#YoungNY_Banging", "followers_count": 997, "friends_count": 971, "statues_count": 42823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lochearn, MD", "id": "35b58c251fb1789f", "name": "Lochearn", "place_type": "city", "bounding_box": rectangle("-76.748415,39.319362 -76.711027,39.377081") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2447450, "cityName": "Lochearn" } }
+{ "create_at": datetime("2016-01-06T10:00:11.000Z"), "id": 684796619703713792, "text": "Afternoon rain this afternoon, high 55 (13 C). Low 49 (9 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712472, "name": "AustinWX", "screen_name": "AustinWX", "lang": "en", "location": "Austin, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Austin, Texas", "followers_count": 1011, "friends_count": 1366, "statues_count": 8024 }, "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-06T10:00:11.000Z"), "id": 684796619770867713, "text": "@squarepanda #ces2016 we are on!!! @ Sands Convention Center At The Venitian, Las Vegas https://t.co/m2kHO372VA", "in_reply_to_status": -1, "in_reply_to_user": 4362629353, "favorite_count": 0, "coordinate": point("-115.16611281,36.12214647"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ces2016" }}, "user_mentions": {{ 4362629353 }}, "user": { "id": 15110777, "name": "ahmay98", "screen_name": "ahmay98", "lang": "en", "location": "null", "create_at": date("2008-06-13"), "description": "null", "followers_count": 21, "friends_count": 74, "statues_count": 107 }, "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-06T10:00:11.000Z"), "id": 684796619963760642, "text": "@darthcaligula us https://t.co/xELCX5H2wA", "in_reply_to_status": -1, "in_reply_to_user": 1416307218, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1416307218 }}, "user": { "id": 158613950, "name": "Beth❄", "screen_name": "Littlebits0523", "lang": "en", "location": "Kansas", "create_at": date("2010-06-22"), "description": "18 | BUCO | I like to pretend I'm funny", "followers_count": 154, "friends_count": 190, "statues_count": 4483 }, "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-01-06T10:00:11.000Z"), "id": 684796620119117824, "text": "She's very well organized she has to be. Everything in order. @nochanceofrain @msgoody2shoes21 @finndango @Kateplusmy8 @TLC", "in_reply_to_status": 684794938056249344, "in_reply_to_user": 2869230631, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2869230631, 183161413, 283323982, 268318618, 21313855 }}, "user": { "id": 572363841, "name": "Andrea lee johnson", "screen_name": "JohnsonAndra", "lang": "en", "location": "null", "create_at": date("2012-05-05"), "description": "null", "followers_count": 14, "friends_count": 25, "statues_count": 3001 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwood, OH", "id": "e18891c1461c61a9", "name": "Norwood", "place_type": "city", "bounding_box": rectangle("-84.473831,39.144696 -84.433618,39.178205") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3957386, "cityName": "Norwood" } }
+{ "create_at": datetime("2016-01-06T10:00:11.000Z"), "id": 684796620613943297, "text": "@TODAYshow @MLauer @katiecouric Nice hair, Matt. We all miss the chemistry between the two of them.", "in_reply_to_status": 684795921352110080, "in_reply_to_user": 7744592, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 7744592, 607289206, 18812301 }}, "user": { "id": 15667716, "name": "Brandon Miller", "screen_name": "BrandonToledo", "lang": "en", "location": "Toledo OH", "create_at": date("2008-07-30"), "description": "Game Show Nut, #Mojoholic, Dancer, Church Guy, PS4, HUGE WWE fan and fan of @LilianGarcia, @StephMcmahon, @Tigers, @Yankees, and @DetroitRedWings :-)", "followers_count": 820, "friends_count": 1980, "statues_count": 154289 }, "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-06T10:00:12.000Z"), "id": 684796621129822208, "text": "@adriangobeli https://t.co/6qgsocQDbB", "in_reply_to_status": 684796059663519744, "in_reply_to_user": 323906212, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 323906212 }}, "user": { "id": 473267873, "name": "Johnny Gustafson", "screen_name": "sonofthegustaf", "lang": "en", "location": "sc: sonofgustaf", "create_at": date("2012-01-24"), "description": "Hi, I'm Rod, and I like to party. - Kevin. Domingo Beisbol Academy honors graduate.", "followers_count": 687, "friends_count": 747, "statues_count": 5313 }, "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-01-06T10:00:12.000Z"), "id": 684796621196898304, "text": "#WCW cause why not? �� @NinaFlores4 https://t.co/vNRaaAmTLb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WCW" }}, "user_mentions": {{ 403777412 }}, "user": { "id": 570579840, "name": "Drew", "screen_name": "_Garzaaaa", "lang": "en", "location": "North Richland Hills ,Texas", "create_at": date("2012-05-03"), "description": "Andrew. Baseball. | LHP at Birdville HS |Senior| C/o 2016| Proverbs 21:5 (זזז) | Future M.D", "followers_count": 1108, "friends_count": 353, "statues_count": 15250 }, "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-06T10:00:12.000Z"), "id": 684796621306081280, "text": "@schneidan @OldSchoolLab @ILFORDPhoto @emulsivefilm didn't think of that! I just had a sob in the dark.", "in_reply_to_status": 684794998353559552, "in_reply_to_user": 66594996, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 66594996, 322102425, 59136384, 3178239222 }}, "user": { "id": 1295559284, "name": "erik gould", "screen_name": "ClickErik", "lang": "en", "location": "Pawtucket RI USA", "create_at": date("2013-03-24"), "description": "I will never prostitute my work, except for money.", "followers_count": 298, "friends_count": 599, "statues_count": 3217 }, "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-06T10:00:12.000Z"), "id": 684796621352235008, "text": "Do you boo ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2602597279, "name": "Kel-c-ah", "screen_name": "kelsiesimpson23", "lang": "en", "location": "null", "create_at": date("2014-07-03"), "description": "if no one loves you I will ❣ //Andrew Ruiz", "followers_count": 482, "friends_count": 482, "statues_count": 5150 }, "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-06T10:00:12.000Z"), "id": 684796621536792576, "text": "First wcw of the year goes to my beautiful biff ilysm @reynolds_tayler ���� https://t.co/CLwlJ6kdGe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 717346454 }}, "user": { "id": 1143742808, "name": "Jessie", "screen_name": "jessiee_erin", "lang": "en", "location": "3.8.12❤️", "create_at": date("2013-02-02"), "description": "Life is worth living✨", "followers_count": 594, "friends_count": 480, "statues_count": 13816 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Belleville, MI", "id": "01ec07b14766a84d", "name": "Belleville", "place_type": "city", "bounding_box": rectangle("-83.543444,42.174339 -83.435055,42.216387") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2607020, "cityName": "Belleville" } }
+{ "create_at": datetime("2016-01-06T10:00:12.000Z"), "id": 684796621616463872, "text": "@Versuhtyle yeah... you picked a god day for your return to the Twitter machine.. Welcome back stud", "in_reply_to_status": 684796154182287360, "in_reply_to_user": 588604833, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 588604833 }}, "user": { "id": 40949136, "name": "Benny Ricciardi", "screen_name": "bennyr11", "lang": "en", "location": "Jersey strong", "create_at": date("2009-05-18"), "description": "Webcast host for @Rotocurve, podcast @Rotowire, BRich11 #DFS. Writer @DKPlaybook @Rotocurve and @Rotowire. 2014 winner #FSWA Baseball Article of the Year, Web", "followers_count": 3069, "friends_count": 407, "statues_count": 10277 }, "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-06T10:00:12.000Z"), "id": 684796622040096769, "text": "I think I just discovered that I do have a heart, that was the cutest friggin thing 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": 380711524, "name": "Cynthia Barnhart", "screen_name": "cindiella", "lang": "en", "location": "null", "create_at": date("2011-09-26"), "description": "that's the thing about pain, it demands to be felt. #endowarrior", "followers_count": 309, "friends_count": 175, "statues_count": 25722 }, "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-06T10:00:12.000Z"), "id": 684796622232928256, "text": "Showers this afternoon, high 62 (17 C). Low 56 (13 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712388, "name": "HoustonWX", "screen_name": "HoustonWX", "lang": "en", "location": "Houston, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Houston, Texas", "followers_count": 1816, "friends_count": 77, "statues_count": 8033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-06T10:00:12.000Z"), "id": 684796622342086656, "text": "@angryli0n @KitemanX @polemicablog accusing \"strawmen\" on Twitter is actually using straw at this point�� btw calling me sweetheart-nice try", "in_reply_to_status": 684795742494552065, "in_reply_to_user": 27890940, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27890940, 370038842, 1688873858 }}, "user": { "id": 11198972, "name": "kylo stimpy", "screen_name": "lemoncayke", "lang": "en", "location": "nyc", "create_at": date("2007-12-15"), "description": "insomnia, neurobio, equality, cats, tacos, and haunted houses. If you're a garbage person you'll get ONE insult from me and then a block #BLM #IStandWithPP", "followers_count": 580, "friends_count": 384, "statues_count": 11205 }, "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-06T10:00:12.000Z"), "id": 684796622841200641, "text": "@gracehelbig Can you please tweet for a fan who urgently needs a handicapped vehicle? https://t.co/trNsxhHj1n #VanForSean", "in_reply_to_status": 684796545791741952, "in_reply_to_user": 21502768, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VanForSean" }}, "user_mentions": {{ 21502768 }}, "user": { "id": 1651437565, "name": "SeanSmith", "screen_name": "SSmith1138", "lang": "en", "location": "null", "create_at": date("2013-08-06"), "description": "null", "followers_count": 9082, "friends_count": 9780, "statues_count": 6676 }, "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-06T10:00:12.000Z"), "id": 684796623134789632, "text": "@britt_underwood You're so nice and I love talking to you even if it's just through a tweet :) maybe you can follow me sometime :) ILY! ❤ ��", "in_reply_to_status": -1, "in_reply_to_user": 89626688, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 89626688 }}, "user": { "id": 2725977550, "name": "♡Katelyn♡", "screen_name": "_KatelynEliz_xo", "lang": "en", "location": "Pennsylvania, USA ", "create_at": date("2014-07-26"), "description": "my personal twitter■ Cimorelli 7/7 ■ Keep your head up, Keep your heart strong. ❤", "followers_count": 289, "friends_count": 251, "statues_count": 4521 }, "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-06T10:00:12.000Z"), "id": 684796623399030785, "text": "@FunniestVines @CaseyOliphant", "in_reply_to_status": 678768507581046788, "in_reply_to_user": 585937668, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 585937668, 1323209419 }}, "user": { "id": 263879147, "name": "Jenna LaRose", "screen_name": "jen_larose", "lang": "en", "location": "null", "create_at": date("2011-03-10"), "description": "null", "followers_count": 510, "friends_count": 445, "statues_count": 4158 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lockport, NY", "id": "45009687ba062971", "name": "Lockport", "place_type": "city", "bounding_box": rectangle("-78.744599,43.138056 -78.618744,43.205149") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36063, "countyName": "Niagara", "cityID": 3643082, "cityName": "Lockport" } }
+{ "create_at": datetime("2016-01-06T10:00:12.000Z"), "id": 684796623499575296, "text": "Lol, same .. https://t.co/hiPEIk483b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 625235883, "name": "michaela. ✨", "screen_name": "SheSo_Eylau", "lang": "en", "location": "Roszay ❤️✨", "create_at": date("2012-07-02"), "description": "Understand The Game Or Be A Victim. ✨ Rest Easy Big Ced. ❤️", "followers_count": 1725, "friends_count": 1527, "statues_count": 41429 }, "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.355819 -93.968792,33.507099") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48037, "countyName": "Bowie", "cityID": 4872368, "cityName": "Texarkana" } }
+{ "create_at": datetime("2016-01-06T10:00:12.000Z"), "id": 684796623512272896, "text": "@RobertPinsky Am presenting \"ABC\" to my poetry group today. #tourdeforce What is your favorite creation?", "in_reply_to_status": -1, "in_reply_to_user": 111066459, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "tourdeforce" }}, "user_mentions": {{ 111066459 }}, "user": { "id": 2418046156, "name": "ReadingJudith", "screen_name": "ReadingJudith", "lang": "en", "location": "Bristol, RI", "create_at": date("2014-03-17"), "description": "Lover of reading, writing, travel and Rhode Island. Interested in everything. BA English Lit. Retired ICU RN, CCRN.", "followers_count": 358, "friends_count": 683, "statues_count": 6848 }, "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-06T10:00:12.000Z"), "id": 684796623524737025, "text": "@Crislex @JustinWNS played HC TDM yesterday and no bullshit there was like 20 straight seconds of silence cause everyone was camping.", "in_reply_to_status": 684790256575639552, "in_reply_to_user": 20738274, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20738274, 1345648788 }}, "user": { "id": 2235680174, "name": "Asstiel.", "screen_name": "NotTylerHaynes", "lang": "en", "location": "Red Oak, Texas", "create_at": date("2013-12-07"), "description": "I'm 20 not 12; Professional Loser at Life, I can't grow facial hair, I'm a lot cooler online.", "followers_count": 299, "friends_count": 247, "statues_count": 14022 }, "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-06T10:00:12.000Z"), "id": 684796623562526722, "text": "@yungxpharaoh_ yes you know I'm down for that !!!", "in_reply_to_status": 684794311792279552, "in_reply_to_user": 1537964262, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1537964262 }}, "user": { "id": 605630118, "name": "Colt Trapp", "screen_name": "colttrapp120", "lang": "en", "location": "@lovelylei_13 ", "create_at": date("2012-06-11"), "description": "null", "followers_count": 1279, "friends_count": 519, "statues_count": 5243 }, "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-06T10:00:12.000Z"), "id": 684796623868657664, "text": "#CustomerService alert: Client Service Manager - Stock Plan Services | Charles Schwab | #LoneTree, CO https://t.co/2MY8o6dpyb #Veterans", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.8963682,39.5360997"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CustomerService", "LoneTree", "Veterans" }}, "user": { "id": 71966627, "name": "CO Cust. Srv. Jobs", "screen_name": "tmj_CO_cstsrv", "lang": "en", "location": "Colorado", "create_at": date("2009-09-05"), "description": "Follow this account for geo-targeted Customer Service job tweets in Colorado Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 331, "friends_count": 274, "statues_count": 171 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carriage Club, CO", "id": "2d9dc2a8b478da71", "name": "Carriage Club", "place_type": "city", "bounding_box": rectangle("-104.904327,39.521709 -104.885217,39.53651") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8035, "countyName": "Douglas", "cityID": 845955, "cityName": "Lone Tree" } }
+{ "create_at": datetime("2016-01-06T10:00:12.000Z"), "id": 684796623944286212, "text": "Barberton Oh Temp:33.5°F Wind:2 mph Dir:S Baro:Falling Rain2day:0.00in Hum:51% UV:0.7 @ 13:00 01/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": 235, "friends_count": 228, "statues_count": 113717 }, "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-01-06T10:00:12.000Z"), "id": 684796624044855296, "text": "Thanks Brit! https://t.co/kkuZgek4mA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 394609792, "name": "Trevor Hartley", "screen_name": "T_Nasty1point0", "lang": "en", "location": "wherevers not gonna get me hit", "create_at": date("2011-10-20"), "description": "just an average man, with exceptional hair. ΦΣK 2567 DAMN PROUD!", "followers_count": 338, "friends_count": 397, "statues_count": 8679 }, "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-06T10:00:12.000Z"), "id": 684796624187473920, "text": "I just looked at my tweets and I realized I spelled \"weather\" wrong. I apologize. It was 6:40am and my eyes were still stuck together.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 601541009, "name": "T.", "screen_name": "tcarvalhoo_", "lang": "en", "location": "nj/ny", "create_at": date("2012-06-06"), "description": "Hi, I'm foreign (not really).", "followers_count": 334, "friends_count": 264, "statues_count": 14700 }, "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-06T10:00:12.000Z"), "id": 684796624313430017, "text": "Damnnnnnnn lmao!!!���� https://t.co/ugcc8KFevj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 881190498, "name": "Princess Naivy ♎️", "screen_name": "NaivyArellano", "lang": "en", "location": "815", "create_at": date("2012-10-14"), "description": "rkfd. you got money, you got me. 10/05", "followers_count": 504, "friends_count": 255, "statues_count": 21284 }, "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-06T10:00:12.000Z"), "id": 684796624338485248, "text": "Pero que fué lo que pasóó?? Jaja igual ya sabemos quién gana! �� https://t.co/TZQofAesNK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 178894712, "name": "LETTY_PINEDA", "screen_name": "blancalettyp", "lang": "en", "location": "Miami, FL", "create_at": date("2010-08-15"), "description": "El futbol es mi pasión, Merengue y Americanista de ....❤ Twits 90% de futbol.⚽ #RazaAdicta #SiempreEsperandoElPróximoPartido ⚽️", "followers_count": 1819, "friends_count": 399, "statues_count": 37280 }, "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-06T10:00:12.000Z"), "id": 684796624434933760, "text": "@Metcalf_Ashley do it!!", "in_reply_to_status": 684792737229443072, "in_reply_to_user": 555513305, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 555513305 }}, "user": { "id": 2910277878, "name": "Diana Gulinyan", "screen_name": "dianagulinyann", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-11-25"), "description": "2sassy4u", "followers_count": 229, "friends_count": 140, "statues_count": 2000 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:12.000Z"), "id": 684796624892067840, "text": "@MarcACaputo lame excuse! Stop paying him!", "in_reply_to_status": 684790537677893632, "in_reply_to_user": 227373977, "favorite_count": 0, "coordinate": point("-81.66799094,26.0422531"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 227373977 }}, "user": { "id": 16362083, "name": "Betsy Lebeau", "screen_name": "betsylebeau", "lang": "en", "location": "Fairhaven, Massachusetts", "create_at": date("2008-09-19"), "description": "Retired, living in Fairhaven, Massachusetts. Own winter home in Naples, FL. Widow who loves antiquing, dining out and fashion.", "followers_count": 44, "friends_count": 261, "statues_count": 2658 }, "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": 12021, "countyName": "Collier" } }
+{ "create_at": datetime("2016-01-06T10:00:12.000Z"), "id": 684796624980197376, "text": "Cloudy this afternoon, high 58 (14 C). Low 42 (6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575081, "name": "San Angelo Weather", "screen_name": "SanAngeloWX", "lang": "en", "location": "San Angelo, TX", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for San Angelo, TX", "followers_count": 1944, "friends_count": 92, "statues_count": 7923 }, "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-06T10:00:13.000Z"), "id": 684796625244389376, "text": "Is it weird that I'm still caught up on ya ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4634309287, "name": "RAMBO Steve", "screen_name": "OgBrahh", "lang": "en", "location": "Houston, TX", "create_at": date("2015-12-22"), "description": "For God I Play ❤️ Soon Enough I'll Play For LSU", "followers_count": 16, "friends_count": 26, "statues_count": 180 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-06T10:00:13.000Z"), "id": 684796625450065920, "text": "I need new chucks", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 523794435, "name": "Chelsea W.", "screen_name": "papah_airplanes", "lang": "en", "location": "Boston, MA", "create_at": date("2012-03-13"), "description": "Chelsea. 22.", "followers_count": 242, "friends_count": 244, "statues_count": 14081 }, "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-06T10:00:13.000Z"), "id": 684796625936605185, "text": "I cannot wait for #TheGetDown to come out on @netflix! https://t.co/K0M9gMxQxv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TheGetDown" }}, "user_mentions": {{ 16573941 }}, "user": { "id": 156357340, "name": "Caroline Trujillo", "screen_name": "ctru16", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-06-16"), "description": "Believer. @TexasTech alumna. @GOLINglobal YoPro. Slow clap enthusiast. Let the music speak. Putting the E into extrovert. All thoughts are my own.", "followers_count": 734, "friends_count": 1120, "statues_count": 5565 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:13.000Z"), "id": 684796625999532033, "text": "@JolieBomb Niggas still do that?", "in_reply_to_status": 684796286969589761, "in_reply_to_user": 539436753, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 539436753 }}, "user": { "id": 88099075, "name": "Tay Fields", "screen_name": "ChillCosby__", "lang": "en", "location": "North Philly #Somerset ", "create_at": date("2009-11-06"), "description": "Portrait Artist x Photographer. #NorthPhilly. @chillyano on Instagram", "followers_count": 903, "friends_count": 2057, "statues_count": 49055 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willow Grove, PA", "id": "b473d3b0c8ee5cf5", "name": "Willow Grove", "place_type": "city", "bounding_box": rectangle("-75.143114,40.129922 -75.091487,40.171482") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42091, "countyName": "Montgomery", "cityID": 4285408, "cityName": "Willow Grove" } }
+{ "create_at": datetime("2016-01-06T10:00:13.000Z"), "id": 684796626326712320, "text": "Bitches Weave Be BOGUS ASL������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2342542920, "name": "✨Mic'sKeeper", "screen_name": "LoveParisAliyah", "lang": "en", "location": "Chicago, IL", "create_at": date("2014-02-13"), "description": "#LLMickey | 18 | ZachsWife ❤️", "followers_count": 299, "friends_count": 242, "statues_count": 27135 }, "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-06T10:00:13.000Z"), "id": 684796626406223872, "text": "This lady's car is decked out with Jesus stickers but she's driving like Satan", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1708537974, "name": "MANUAL", "screen_name": "MMANNUUELL", "lang": "en", "location": "null", "create_at": date("2013-08-28"), "description": "Hebrews 13:8 ☕️", "followers_count": 223, "friends_count": 212, "statues_count": 3035 }, "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-06T10:00:13.000Z"), "id": 684796626469138432, "text": "Today sucks", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2953190462, "name": "Aiyana♡", "screen_name": "aiyananichole_", "lang": "en", "location": "null", "create_at": date("2014-12-30"), "description": "Aquarius", "followers_count": 135, "friends_count": 231, "statues_count": 1849 }, "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-06T10:00:13.000Z"), "id": 684796626490097664, "text": "Can you recommend anyone for this #job? Retail Sales Associate - https://t.co/VwFfmdNqVE #LAHAINA, HI #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-156.6817594,20.8866199"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "LAHAINA", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 38082277, "name": "TMJ-MAU Jobs", "screen_name": "tmj_mau_jobs", "lang": "en", "location": "Maui, HI", "create_at": date("2009-05-05"), "description": "Follow this account for geo-targeted Other job tweets in Maui, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 547, "friends_count": 535, "statues_count": 15 }, "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-01-06T10:00:13.000Z"), "id": 684796626951475200, "text": "#Retail #Job in #Wise, VA: Store Team Leader at Tractor Supply Company https://t.co/zikZSeR4mv #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.5967537,36.9715586"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "Job", "Wise", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 59694665, "name": "TMJ-VA Retail Jobs", "screen_name": "tmj_va_retail", "lang": "en", "location": "Virginia", "create_at": date("2009-07-23"), "description": "Follow this account for geo-targeted Retail job tweets in Virginia Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 376, "friends_count": 304, "statues_count": 744 }, "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": 51195, "countyName": "Wise", "cityID": 5187072, "cityName": "Wise" } }
+{ "create_at": datetime("2016-01-06T10:00:13.000Z"), "id": 684796627119255552, "text": "Be sure to click the \"Radar\" tab if you are still stuck on \"Satellite.\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 250248914, "name": "Robert Haviland", "screen_name": "LambysPoet", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-02-10"), "description": "Personal poet to the world's most beautiful and talented woman, Brit. Poetry Book http://tinyurl.com/q59vsyd", "followers_count": 304, "friends_count": 446, "statues_count": 187640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:13.000Z"), "id": 684796627253620736, "text": "#atdcare The NAC CARE team is all online and ready for your calls or tweets. @atdchapters, do you have questions about CARE?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "atdcare" }}, "user_mentions": {{ 128617092 }}, "user": { "id": 486098649, "name": "Jeff Gehris", "screen_name": "JGehris", "lang": "en", "location": "Hampton, VA", "create_at": date("2012-02-07"), "description": "Recruiting and developing the employees of Langley FCU to ensure strong professional and personal growth.", "followers_count": 100, "friends_count": 180, "statues_count": 405 }, "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-06T10:00:13.000Z"), "id": 684796627270250496, "text": "Fighting for what's minds and what I deserve willing to fight the world if I have too u can't 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": 555624469, "name": "QueenBash_2swayze", "screen_name": "QueenBash2Live", "lang": "en", "location": "Virginia, beach virginia", "create_at": date("2012-04-16"), "description": "i am smart ,ambitious and the life of the party i enjoy life and keep god first in everything i do am just a country girl from Alabama and a rasta", "followers_count": 332, "friends_count": 1176, "statues_count": 1542 }, "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-06T10:00:13.000Z"), "id": 684796627291353088, "text": "@ChelseaHouska You are my favorite��. Are you coming back for another season??", "in_reply_to_status": -1, "in_reply_to_user": 202495093, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 202495093 }}, "user": { "id": 904574587, "name": "Jenni", "screen_name": "jenn02242", "lang": "en", "location": "null", "create_at": date("2012-10-25"), "description": "A smile is the prettiest thing you can wear", "followers_count": 16, "friends_count": 125, "statues_count": 169 }, "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-06T10:00:13.000Z"), "id": 684796627458981889, "text": "S/o @DigitalFeedNYC for the support! Watch Michigan Mob Official ScarfoTesta Video exclusively on @thisis50 & Share https://t.co/o6vNmozpag", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1674578383, 15781432 }}, "user": { "id": 1158853686, "name": "MichiganMobxBTG", "screen_name": "BTG_FAM", "lang": "en", "location": "thakiddtv@yahoo.com", "create_at": date("2013-02-07"), "description": "BTG x MindPower x @StainlessDjs http://Mtv.com/artists/btg", "followers_count": 1468, "friends_count": 3592, "statues_count": 1457 }, "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-06T10:00:13.000Z"), "id": 684796627576426497, "text": "@savannah5_25_12 See, I don't do this stuff because I hate you���� https://t.co/NQgSbjueIn", "in_reply_to_status": -1, "in_reply_to_user": 637212238, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 637212238 }}, "user": { "id": 588813011, "name": "Clark Kent", "screen_name": "Cole_Pendleton", "lang": "en", "location": "Savannah Michonski❤️", "create_at": date("2012-05-23"), "description": "San Diego ||18|| Sonoma State University", "followers_count": 523, "friends_count": 801, "statues_count": 4284 }, "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-06T10:00:13.000Z"), "id": 684796627828146177, "text": "Sunny this afternoon, high 65 (18 C). Low 49 (9 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575147, "name": "Baton Rouge Weather", "screen_name": "BatonRougeWX", "lang": "en", "location": "Baton Rouge, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Baton Rouge, LA", "followers_count": 2874, "friends_count": 92, "statues_count": 7876 }, "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-06T10:00:13.000Z"), "id": 684796627974885376, "text": "I know u think I'm smart... But come on anyone could've figured that out #Dummies", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Dummies" }}, "user": { "id": 410205812, "name": "Marina Sinishtaj", "screen_name": "MarinaSinishtaj", "lang": "en", "location": "null", "create_at": date("2011-11-11"), "description": "For it was not into my ear you whispered, but into my heart. It was not my lips you kissed, but my soul.", "followers_count": 358, "friends_count": 293, "statues_count": 4501 }, "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-01-06T10:00:13.000Z"), "id": 684796628251840513, "text": "Just like in Florida...I'm not sure they know its 19 degrees out… https://t.co/BR95ubFaB9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.50261004,40.83604982"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20069973, "name": "Michael J Fox", "screen_name": "mjfoxy12", "lang": "en", "location": "Orlando, FL", "create_at": date("2009-02-04"), "description": "Work at Florida Hospital as the Assistant Director of Sports Partnerships", "followers_count": 294, "friends_count": 282, "statues_count": 498 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morris Plains, NJ", "id": "01b1ca90360bd9f4", "name": "Morris Plains", "place_type": "city", "bounding_box": rectangle("-74.533584,40.816841 -74.460175,40.871364") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34027, "countyName": "Morris", "cityID": 3448210, "cityName": "Morris Plains" } }
+{ "create_at": datetime("2016-01-06T10:00:13.000Z"), "id": 684796628675461121, "text": "Cold af waiting on this nut ass bus ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2902471703, "name": "StoneAge Lexx", "screen_name": "KeepCalmItsLexx", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2014-12-02"), "description": "#StoneAge If You Cant Be Safe .. Be Fuckin Deadly | iRap & Sh*t | Send Instrumentals & Bookings To Remainambitious@gmail.com", "followers_count": 1661, "friends_count": 1107, "statues_count": 20356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flourtown, PA", "id": "c1e5bddd6cb22b02", "name": "Flourtown", "place_type": "city", "bounding_box": rectangle("-75.21578,40.086143 -75.193722,40.120208") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42091, "countyName": "Montgomery", "cityID": 4226376, "cityName": "Flourtown" } }
+{ "create_at": datetime("2016-01-06T10:00:13.000Z"), "id": 684796628860071936, "text": "the 118th st station always smells like pizza grease tbh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 779117629, "name": "mak", "screen_name": "makbethke", "lang": "en", "location": "nyc", "create_at": date("2012-08-24"), "description": "morbid goth teen, known lover of memes, believer in genghis swan", "followers_count": 109, "friends_count": 350, "statues_count": 3104 }, "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-06T10:00:14.000Z"), "id": 684796629539536897, "text": "@reipuerto that’s a 3+ year old phone", "in_reply_to_status": 684796456780312577, "in_reply_to_user": 42293695, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 42293695 }}, "user": { "id": 20339205, "name": "HoHo", "screen_name": "HoHo23", "lang": "en", "location": "Miami Beach", "create_at": date("2009-02-07"), "description": "Beer Lover/Blogger, video game enthusiast, fixed bike riding fool. Lets have some beers! Staticx2552 on XBL, PSN and WiiU #MUFC", "followers_count": 676, "friends_count": 320, "statues_count": 37560 }, "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-06T10:00:14.000Z"), "id": 684796629551976448, "text": "The right way to do it https://t.co/zY0S2ECS77", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1534146578, "name": "Buff Swanson", "screen_name": "CodeineKurt", "lang": "en", "location": "DALLAS NIGGA", "create_at": date("2013-06-20"), "description": "I was addicted to the hokie pokie then I turned myself around. 2xSkrr. good things take time. We ain't bout Killen We bout Chillen -JROC. Codeine Kurt is Me", "followers_count": 510, "friends_count": 535, "statues_count": 7204 }, "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-06T10:00:14.000Z"), "id": 684796630449688576, "text": "#lunabessa #trionoir #daddynoir #day42 The whole fam damily ! Look at Luna's expression.… https://t.co/VsDM0JsR77", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.6974716,40.5858192"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "lunabessa", "trionoir", "daddynoir", "day42" }}, "user": { "id": 14807077, "name": "Benoir", "screen_name": "musicnoir", "lang": "en", "location": "Long Beach NY", "create_at": date("2008-05-16"), "description": "I am a music artist, producer, educator, and seeker of knowledge.... Peace", "followers_count": 327, "friends_count": 277, "statues_count": 3632 }, "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-06T10:00:14.000Z"), "id": 684796630575390720, "text": "Partly cloudy this afternoon, high 61 (16 C). Low 54 (12 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575204, "name": "New Orleans Weather", "screen_name": "NewOrleansWX", "lang": "en", "location": "New Orleans, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for New Orleans, LA", "followers_count": 2899, "friends_count": 92, "statues_count": 7894 }, "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-06T10:00:14.000Z"), "id": 684796630613229568, "text": "PeaCoat and Sweater Season. https://t.co/NV8Fyp8GQB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 103747040, "name": "Bordeaux.", "screen_name": "Jay_Bordeaux", "lang": "en", "location": "Richmond/Cleveland ", "create_at": date("2010-01-10"), "description": "(Artist)\n Bordeauxartwork@gmail.com", "followers_count": 1047, "friends_count": 743, "statues_count": 83711 }, "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-06T10:00:14.000Z"), "id": 684796630801891328, "text": "@lantzv44 Ohhh haha", "in_reply_to_status": 684796495434919936, "in_reply_to_user": 2352035670, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 2352035670 }}, "user": { "id": 3277995444, "name": "Abby Neira الوصيفة", "screen_name": "neira_abby", "lang": "en", "location": "Bakersfield, CA", "create_at": date("2015-07-12"), "description": "A queen knows how to build her empire with the same stones that were thrown at her", "followers_count": 137, "friends_count": 503, "statues_count": 670 }, "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-06T10:00:14.000Z"), "id": 684796631129190400, "text": "Also kettle cooked chips are so hard...like damn why this crunch 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": 3271891908, "name": "AG", "screen_name": "AG_Magicz", "lang": "en", "location": "null", "create_at": date("2015-07-08"), "description": "My twitter of thoughts, ideas & RTs. ✌", "followers_count": 15, "friends_count": 72, "statues_count": 829 }, "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-06T10:00:14.000Z"), "id": 684796631175290880, "text": "@Tde_Richie @adventxres nah no food period", "in_reply_to_status": 684796561876881408, "in_reply_to_user": 333224539, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 333224539, 1329231578 }}, "user": { "id": 3029221168, "name": "☹", "screen_name": "DTHam2", "lang": "en", "location": "Norman, OK", "create_at": date("2015-02-10"), "description": "Vikings, Sooners, Thunder.", "followers_count": 486, "friends_count": 409, "statues_count": 25698 }, "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-06T10:00:14.000Z"), "id": 684796631338868736, "text": "Now if we talking body. You gotta perfect one so put it 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": 65053809, "name": "CRod", "screen_name": "TheOfficiaICRod", "lang": "en", "location": "Lincoln Park, MI", "create_at": date("2009-08-12"), "description": "#OvOXO Nov 10 1996.\nGym.\nShoes.\nXbox.\nI like big butts and pretty smiles.\nSomewhere keeping real with myself and nobody else.", "followers_count": 1032, "friends_count": 715, "statues_count": 45020 }, "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-01-06T10:00:14.000Z"), "id": 684796631351488512, "text": "My eyebrows 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": 247579361, "name": "shanti ❣", "screen_name": "_shanticashee", "lang": "en", "location": "ya daddy house", "create_at": date("2011-02-04"), "description": "dnh 12:16 ❤️ th 3:27 ❤️ rip Aunt Nichole 10:17 ❤️ #choleworld #4everNichole ~ at peace with myself", "followers_count": 935, "friends_count": 799, "statues_count": 52914 }, "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-06T10:00:14.000Z"), "id": 684796631666044930, "text": "Dumped :(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 244313450, "name": "Guala Bear", "screen_name": "tedcadillac", "lang": "en", "location": "Columbus", "create_at": date("2011-01-28"), "description": "http://instagram.com/tedcadillac -- ericmaxrodriguez@gmail.com", "followers_count": 992, "friends_count": 383, "statues_count": 7952 }, "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-06T10:00:14.000Z"), "id": 684796631791738880, "text": "[Wed Jan 6th, 12:00pm] Temperature: 64.22 F, 17.9 C; Humidity: 41.9%; Light: 10.25%. https://t.co/UI5qWnQ21d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 269430133, "name": "Shoemaker Sandbox", "screen_name": "shoe_sandbox", "lang": "en", "location": "Earth (for now). ", "create_at": date("2011-03-20"), "description": "I am a bot. Developer account for @shoemaker.", "followers_count": 7, "friends_count": 1, "statues_count": 24523 }, "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-06T10:00:14.000Z"), "id": 684796631888211968, "text": "@KarlRove is such a turd. @realDonaldTrump for president. @NordStormRising \nCombat veterans for #Trump2016", "in_reply_to_status": -1, "in_reply_to_user": 18791763, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump2016" }}, "user_mentions": {{ 18791763, 25073877, 3596501833 }}, "user": { "id": 3254808686, "name": "Jacob Wood", "screen_name": "JacobWo83406074", "lang": "en", "location": "america", "create_at": date("2015-06-24"), "description": "Well i wanted to be a Rockstar but ended up in the US ARMY, Afghanistan 2012 and Kuwait 2015. Huge Van Halen fan, Love all death metal too", "followers_count": 62, "friends_count": 207, "statues_count": 314 }, "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-01-06T10:00:14.000Z"), "id": 684796632022401026, "text": "My cooperating teacher gave me my own teachers desk. :)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27137309, "name": "Hannah", "screen_name": "hannahingraham", "lang": "en", "location": "Woodland & Ellensburg, WA ", "create_at": date("2009-03-27"), "description": "cwu & gage martell", "followers_count": 245, "friends_count": 94, "statues_count": 9226 }, "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-06T10:00:14.000Z"), "id": 684796632265666560, "text": "Wind 9.0 mph S. Barometer 1025.50 mb, Falling. Temperature 41.1 °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": 25, "friends_count": 124, "statues_count": 11650 }, "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-06T10:00:14.000Z"), "id": 684796632458772480, "text": "Miss Texas 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": 767904524, "name": "Gennesis", "screen_name": "gennesis_torres", "lang": "en", "location": "Laredo, TX", "create_at": date("2012-08-19"), "description": "Invest in yourself don't ever stop working on yourself", "followers_count": 1078, "friends_count": 595, "statues_count": 25154 }, "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.639696") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55059, "countyName": "Kenosha", "cityID": 5539225, "cityName": "Kenosha" } }
+{ "create_at": datetime("2016-01-06T10:00:14.000Z"), "id": 684796632622170112, "text": "The forecast for tonight is mostly cloudy with a low of 32°F. #ThreeKingsDay #Epiphany", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.5333,41.5167"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ThreeKingsDay", "Epiphany" }}, "user": { "id": 42666285, "name": "Quad Cities Weather", "screen_name": "qcweather", "lang": "en", "location": "Quad Cities", "create_at": date("2009-05-26"), "description": "Is it going to be warm, wet or cold in the Quad Cities today? You can also find us on Facebook.", "followers_count": 741, "friends_count": 0, "statues_count": 11501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rock Island Arsenal, IL", "id": "8f966c3da836a196", "name": "Rock Island Arsenal", "place_type": "city", "bounding_box": rectangle("-90.570223,41.509992 -90.511303,41.523635") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17161, "countyName": "Rock Island", "cityID": 1765104, "cityName": "Rock Island Arsenal" } }
+{ "create_at": datetime("2016-01-06T10:00:14.000Z"), "id": 684796632676827136, "text": "$500 mill isn't even $500 million mcchickens anymore", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 146926463, "name": "Anthony Marciano", "screen_name": "A_Marciano691", "lang": "en", "location": "Queens", "create_at": date("2010-05-22"), "description": "TKE theta sigma", "followers_count": 464, "friends_count": 972, "statues_count": 9124 }, "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-06T10:00:14.000Z"), "id": 684796633066807296, "text": "Interested in a #Healthcare #job near #Honolulu, HI? This could be a great fit: https://t.co/uuqxndUpNT #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "Honolulu", "Hiring", "CareerArc" }}, "user": { "id": 23526678, "name": "TMJ- HON Health Jobs", "screen_name": "tmj_hon_health", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Healthcare job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 407, "friends_count": 304, "statues_count": 192 }, "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-06T10:00:14.000Z"), "id": 684796633091985408, "text": "@30SecondJams @Krislovee12", "in_reply_to_status": 684793807276240896, "in_reply_to_user": 2896309515, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2896309515, 2900271565 }}, "user": { "id": 284301389, "name": "Anaisa Tamayo", "screen_name": "Anaisaa_", "lang": "en", "location": "null", "create_at": date("2011-04-18"), "description": "i always return my shopping cart", "followers_count": 498, "friends_count": 652, "statues_count": 7447 }, "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-06T10:00:14.000Z"), "id": 684796633301659648, "text": "Want to work at Starbucks? We're #hiring in ! Click for details: https://t.co/HMXke1iBFV #Hospitality #Veterans #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.7973653,21.3996475"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Hospitality", "Veterans", "Job", "Jobs", "CareerArc" }}, "user": { "id": 23897767, "name": "TMJ-HON Hosp. Jobs", "screen_name": "tmj_hon_hrta", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-11"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 411, "friends_count": 292, "statues_count": 241 }, "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-01-06T10:00:14.000Z"), "id": 684796633414934528, "text": "Cloudy this afternoon, high 56 (13 C). Low 47 (8 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575255, "name": "Shreveport Weather", "screen_name": "ShreveportWX", "lang": "en", "location": "Shreveport, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Shreveport, LA", "followers_count": 700, "friends_count": 92, "statues_count": 7952 }, "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-06T10:00:15.000Z"), "id": 684796633612062720, "text": "�� dad https://t.co/0Gvd3nqdZ5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2356068851, "name": "kally", "screen_name": "kaaliajay", "lang": "en", "location": "76$outh", "create_at": date("2014-02-20"), "description": "@bvsedzach", "followers_count": 954, "friends_count": 548, "statues_count": 41628 }, "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-01-06T10:00:15.000Z"), "id": 684796633649790976, "text": "https://t.co/kGe5vplldB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2463705139, "name": "Adan Sandoval", "screen_name": "AidanSandoval", "lang": "en", "location": "West Coast", "create_at": date("2014-04-25"), "description": "Call me (Ay-Den) • YouNower • I'm a Kardashian btw ✨", "followers_count": 1702, "friends_count": 843, "statues_count": 14055 }, "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-06T10:00:15.000Z"), "id": 684796633725321216, "text": "Want to work at Central Peninsula Hospital? We're #hiring in #Soldotna, AK! Click for details: https://t.co/xMcxdmDGIz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-151.0583332,60.4877778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Soldotna" }}, "user": { "id": 3278999924, "name": "CPH Jobs", "screen_name": "CPHJobs", "lang": "en", "location": "Soldotna, AK", "create_at": date("2015-07-13"), "description": "At CPH every employee is considered a caregiver, an integral part of our health care team. Follow us to learn more!", "followers_count": 26, "friends_count": 77, "statues_count": 50 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Soldotna, AK", "id": "350518f7c1ac93ff", "name": "Soldotna", "place_type": "city", "bounding_box": rectangle("-151.193024,60.458405 -151.048563,60.511008") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2122, "countyName": "Kenai Peninsula", "cityID": 271640, "cityName": "Soldotna" } }
+{ "create_at": datetime("2016-01-06T10:00:15.000Z"), "id": 684796633767256065, "text": "@MatherDonald @David_Engelson @nhdogmom @RandallCHerring @Trumps1stMember @JoeNBC @morningmika @Morning_Joe Yeah they blow him daily.", "in_reply_to_status": 684792028903923713, "in_reply_to_user": 3394129522, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3394129522, 222509602, 21043723, 2187579630, 3798307395, 21619519, 18227519, 254117355 }}, "user": { "id": 21932745, "name": "Julie Harris", "screen_name": "sabela13", "lang": "en", "location": "California ", "create_at": date("2009-02-25"), "description": "Proud liberal, athiest, loves animals and the planet we all share & GOP, it's called the DEMOCRATIC party.", "followers_count": 1101, "friends_count": 1100, "statues_count": 36633 }, "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-01-06T10:00:15.000Z"), "id": 684796633901481984, "text": "Mi (D) ������������ https://t.co/uGfj9yGEpm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 416581308, "name": "Chanty", "screen_name": "ChantalBlanco1", "lang": "en", "location": "Des Moines, IA", "create_at": date("2011-11-19"), "description": "Class of 2016 .\r\nIf you Unfollow me it's b/c I'm great and you're not. Sorry.", "followers_count": 644, "friends_count": 277, "statues_count": 9552 }, "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-06T10:00:15.000Z"), "id": 684796634014826496, "text": "Situations like this is why i don't have faith in anything.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 274321330, "name": "J", "screen_name": "Jordanmurph_", "lang": "en", "location": "null", "create_at": date("2011-03-29"), "description": "just trying to get my life together", "followers_count": 1197, "friends_count": 666, "statues_count": 34072 }, "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-01-06T10:00:15.000Z"), "id": 684796634312511488, "text": "I doubt that last one was original but I had to pitch it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 253116149, "name": "Muhammad Dalal", "screen_name": "mdalal91", "lang": "en", "location": "St. Louis", "create_at": date("2011-02-16"), "description": "That guy’s just a mess. It’s like God spilled a person. #LongLiveTheNote", "followers_count": 315, "friends_count": 550, "statues_count": 14664 }, "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-01-06T10:00:15.000Z"), "id": 684796634375434240, "text": "@sarahwilkes it's coo https://t.co/LcKNYEveuD", "in_reply_to_status": 684795348859006976, "in_reply_to_user": 13179152, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 13179152 }}, "user": { "id": 259148967, "name": "Andrew Pelletier", "screen_name": "andrewpelletier", "lang": "en", "location": "Chicago", "create_at": date("2011-02-28"), "description": "recovering altar boy. soul singer and spongwriter of @minorcharactrs", "followers_count": 274, "friends_count": 932, "statues_count": 2264 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roselle, IL", "id": "5e80ee08ea361155", "name": "Roselle", "place_type": "city", "bounding_box": rectangle("-88.124996,41.949557 -88.031515,42.000647") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1765806, "cityName": "Roselle" } }
+{ "create_at": datetime("2016-01-06T10:00:15.000Z"), "id": 684796634790670336, "text": "Hahahaha yeah talking to her is a waste of time lmfao https://t.co/tTUgH9YjQB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4179903193, "name": "Enidraj, B.", "screen_name": "bjardine_", "lang": "en", "location": "Vancouver, WA", "create_at": date("2015-11-09"), "description": "null", "followers_count": 30, "friends_count": 128, "statues_count": 1676 }, "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-06T10:00:15.000Z"), "id": 684796635021312001, "text": "Oh man, another hilarious video w/ @icecube @KevinHart4real & @ConanOBrien -> https://t.co/NClDm5BBYk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19671129, 23151437, 115485051 }}, "user": { "id": 7779312, "name": "Chuck Reynolds", "screen_name": "ChuckReynolds", "lang": "en", "location": "San Francisco", "create_at": date("2007-07-27"), "description": "I am good at the internet", "followers_count": 5367, "friends_count": 467, "statues_count": 69612 }, "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-06T10:00:15.000Z"), "id": 684796635042476032, "text": "Alan I think your o.g fallowed a brother on Twitter lol @alan22diaz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2332813813 }}, "user": { "id": 716666792, "name": "Anthony Carbajal", "screen_name": "Ac2114", "lang": "en", "location": "null", "create_at": date("2012-07-25"), "description": "23 | Chicago | BullsNation | Snapchat: ac2114", "followers_count": 212, "friends_count": 194, "statues_count": 2196 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntley, IL", "id": "40801c91ed89c83c", "name": "Huntley", "place_type": "city", "bounding_box": rectangle("-88.481796,42.120368 -88.388485,42.204813") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17111, "countyName": "McHenry", "cityID": 1736750, "cityName": "Huntley" } }
+{ "create_at": datetime("2016-01-06T10:00:15.000Z"), "id": 684796635281395712, "text": "My physics class sucks more than I anticipated", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 557930238, "name": "Beverly Johnson", "screen_name": "beveyhills", "lang": "en", "location": "Vancouver, WA", "create_at": date("2012-04-19"), "description": "at ease", "followers_count": 299, "friends_count": 88, "statues_count": 8523 }, "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-06T10:00:15.000Z"), "id": 684796635419906048, "text": "Power hour about to be lit. I feel it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1582128474, "name": "Matthew J. Joseph", "screen_name": "urban_ikon", "lang": "en", "location": "Queens, NY", "create_at": date("2013-07-09"), "description": "Men suffer in silence || OakwoodUniversity", "followers_count": 440, "friends_count": 185, "statues_count": 7825 }, "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-06T10:00:15.000Z"), "id": 684796635524669441, "text": "@logantheecool lmao right ������", "in_reply_to_status": 684795150401449985, "in_reply_to_user": 30576832, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 30576832 }}, "user": { "id": 239075962, "name": "Gianni Gold", "screen_name": "IYAMGianni", "lang": "en", "location": "null", "create_at": date("2011-01-16"), "description": "null", "followers_count": 1034, "friends_count": 595, "statues_count": 31133 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jennings, MO", "id": "0a58f44e2d264da4", "name": "Jennings", "place_type": "city", "bounding_box": rectangle("-90.292312,38.70187 -90.239333,38.741342") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2937178, "cityName": "Jennings" } }
+{ "create_at": datetime("2016-01-06T10:00:15.000Z"), "id": 684796635940040705, "text": "Every time the baseline in @Palehound's, \"Pet Carrot\" comes in I lay face down in the street and pray for death bc I will nvr shred like tht", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1705348908 }}, "user": { "id": 2954322081, "name": "PWR BTTM", "screen_name": "PWRBTTMBAND", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2014-12-31"), "description": "the 11th hardest working band in NYC. tweets by ben hopkins. management: jeanette@saltymgmt.com booking: josh@billions.com", "followers_count": 2951, "friends_count": 1220, "statues_count": 4590 }, "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-06T10:00:15.000Z"), "id": 684796636166393856, "text": "Huge crowds to get into #CES2016 - tightened security very visible. https://t.co/Zj8HkNiQvn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CES2016" }}, "user": { "id": 15630634, "name": "Ben Wood", "screen_name": "benwood", "lang": "en", "location": "London, UK", "create_at": date("2008-07-28"), "description": "Mobile / Wireless Industry Analyst for CCS Insight (devices, wearables, s/w, apps, services and more) +44 7767 886666 / ben.wood@ccsinsight.com", "followers_count": 6891, "friends_count": 899, "statues_count": 6147 }, "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-06T10:00:15.000Z"), "id": 684796636233625600, "text": "WHY TF DO SOUTHERN KEEP PUTTING ME IN FUCKING JONES !!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2563829240, "name": "∂єєє ' тняєєzу 3", "screen_name": "deee__3", "lang": "en", "location": "єνєяуωнєяє&вα¢к!", "create_at": date("2014-06-12"), "description": "ѕσυтнєяи υиινєяѕιту #¢нσρρα¢ιту", "followers_count": 1349, "friends_count": 1656, "statues_count": 39004 }, "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-06T10:00:15.000Z"), "id": 684796636250247168, "text": "Mostly cloudy this afternoon, high 65 (18 C). Low 52 (11 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575312, "name": "Lake Charles Weather", "screen_name": "LakeCharlesWX", "lang": "en", "location": "Lake Charles, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Lake Charles, LA", "followers_count": 541, "friends_count": 92, "statues_count": 7936 }, "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-06T10:00:15.000Z"), "id": 684796636632117249, "text": "@MaryColon hah! That's so good! Luckily it's a tad different : ) we'll have to wait and see how BOTH stories play out!", "in_reply_to_status": 684794814810984449, "in_reply_to_user": 37185695, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 37185695 }}, "user": { "id": 515764082, "name": "Skipp Sudduth", "screen_name": "Skippsudduth", "lang": "en", "location": "Sullivan County, NY", "create_at": date("2012-03-05"), "description": "Actor/Director/Writer PROTECT LIFE, listen and be kind. Shun Ignorance, save this PLANET!", "followers_count": 731, "friends_count": 101, "statues_count": 3728 }, "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-06T10:00:15.000Z"), "id": 684796636975894528, "text": "We're #hiring! Read about our latest #job opening here: Retail Cosmetics Sales - Lead Licensed Esthetician,... - https://t.co/SKB6a0iFpF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9895261,40.7507917"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "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": 676, "friends_count": 0, "statues_count": 8368 }, "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-06T10:00:15.000Z"), "id": 684796637038919680, "text": "@srsounds Can you play the Skipper Dan song? Always makes my day :) thanks!", "in_reply_to_status": -1, "in_reply_to_user": 18662387, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18662387 }}, "user": { "id": 351629937, "name": "Tyler Strickland", "screen_name": "TheMNSportsMan", "lang": "en", "location": "Minneapolis, Minnesota", "create_at": date("2011-08-09"), "description": "I am avid sports fan and an aspiring sports writer in the great state of Minnesota.", "followers_count": 166, "friends_count": 802, "statues_count": 1975 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hunters Creek, FL", "id": "0a7eb306405f5129", "name": "Hunters Creek", "place_type": "city", "bounding_box": rectangle("-81.447294,28.347846 -81.404031,28.375674") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1232967, "cityName": "Hunters Creek" } }
+{ "create_at": datetime("2016-01-06T10:00:15.000Z"), "id": 684796637068312577, "text": "My brother and her overslept and missed the bus and I had to drive them so they were making the noises", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 300902220, "name": "Hala Sura", "screen_name": "halasura", "lang": "en", "location": "University of Tennessee, Knox", "create_at": date("2011-05-18"), "description": "Kurdistan", "followers_count": 1098, "friends_count": 817, "statues_count": 17908 }, "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-01-06T10:00:15.000Z"), "id": 684796637080858625, "text": "@rockstarfelix knows #itsnothype in black anaconda @heydayfootwear… https://t.co/5o2yqiunyj", "in_reply_to_status": -1, "in_reply_to_user": 40328849, "favorite_count": 0, "coordinate": point("-71.45410277,42.31633287"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "itsnothype" }}, "user_mentions": {{ 40328849, 18551115 }}, "user": { "id": 18551115, "name": "HeydayFootwear.com", "screen_name": "HeydayFootwear", "lang": "en", "location": "iPad 42.300510,-71.434608", "create_at": date("2009-01-02"), "description": "#itsnothype, it's #HeydayFootwear, the design driven sneaker brand for fitness fanatics, hip hop dancers + tastemakers Darin@HeydayFootwear.com (347)-443-9329", "followers_count": 5079, "friends_count": 558, "statues_count": 23954 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Framingham, MA", "id": "dbd1411661903bb0", "name": "Framingham", "place_type": "city", "bounding_box": rectangle("-71.496927,42.257263 -71.374193,42.352931") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2524960, "cityName": "Framingham" } }
+{ "create_at": datetime("2016-01-06T10:00:15.000Z"), "id": 684796637084958722, "text": "Happy birthday!!@hannatruong", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 346358141, "name": "Hawk_7⃣6⃣", "screen_name": "mbritton_72", "lang": "en", "location": "Pearland, Tx✈Joilet, Illinois", "create_at": date("2011-07-31"), "description": "University of St.Francis '18 #76", "followers_count": 897, "friends_count": 1008, "statues_count": 31972 }, "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-06T10:00:16.000Z"), "id": 684796637932351489, "text": "Today is the last day to register online for #ABANewOrleans. Starting January 7, you will have to register on site. https://t.co/fYB8I5OtXh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ABANewOrleans" }}, "user": { "id": 1160433078, "name": "ABA International", "screen_name": "ABAIEvents", "lang": "en", "location": "null", "create_at": date("2013-02-08"), "description": "Nonprofit scientific membership organization promoting #BehaviorAnalysis. RT's, follows are not endorsements.", "followers_count": 3370, "friends_count": 738, "statues_count": 2684 }, "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-06T10:00:16.000Z"), "id": 684796638104301568, "text": "Alley oop Swish", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 357357060, "name": "#FreeGucci", "screen_name": "ZoomLA24", "lang": "en", "location": "Compton, CA ✈️ Richmond, VA", "create_at": date("2011-08-18"), "description": "R.I.P Dad ❤️ Same Phone, Fuck You Want?. . . .#WVU17", "followers_count": 12941, "friends_count": 623, "statues_count": 42680 }, "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-06T10:00:16.000Z"), "id": 684796638339047424, "text": "Inject this into the conversation. https://t.co/nmUVNTvxaJ", "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": 541, "friends_count": 2000, "statues_count": 4085 }, "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-06T10:00:16.000Z"), "id": 684796638368411649, "text": "This #Marketing #job might be a great fit for you: Club Supervisor Full Time - https://t.co/cETdy2uDLT #Honolulu, HI #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Marketing", "job", "Honolulu", "Hiring", "CareerArc" }}, "user": { "id": 23493115, "name": "TMJ-HON Advert Jobs", "screen_name": "tmj_hon_adv", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Marketing/Ad/PR job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 290, "friends_count": 193, "statues_count": 9 }, "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-06T10:00:16.000Z"), "id": 684796638477418496, "text": "@BeeuTayful ����", "in_reply_to_status": 684795340097204224, "in_reply_to_user": 3434645362, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3434645362 }}, "user": { "id": 2364262917, "name": "Kyla Nee Bre", "screen_name": "SassySoda_03", "lang": "en", "location": "philly,philly❤️", "create_at": date("2014-02-24"), "description": "Living and breathing for my children ☺️ one option lets go up", "followers_count": 462, "friends_count": 403, "statues_count": 8377 }, "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-06T10:00:16.000Z"), "id": 684796638536183809, "text": "@Rusty76581107 rustyy", "in_reply_to_status": -1, "in_reply_to_user": 1915419396, "favorite_count": 0, "retweet_count": 0, "lang": "fi", "is_retweet": false, "user_mentions": {{ 1915419396 }}, "user": { "id": 4009024400, "name": "Mireya♡", "screen_name": "mireyamynigga", "lang": "en", "location": "Tucson", "create_at": date("2015-10-24"), "description": "Namaste & roll that J.♡||chs'17", "followers_count": 341, "friends_count": 512, "statues_count": 1830 }, "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-06T10:00:16.000Z"), "id": 684796638720729088, "text": "#ProjectMgmt #Job alert: Pavement & Equipment Supervisor | ManTech | #ColoradoSprings, CO https://t.co/1wubYUcSlV #Veterans #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.8213634,38.8338816"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "ProjectMgmt", "Job", "ColoradoSprings", "Veterans", "Jobs", "Hiring" }}, "user": { "id": 395500294, "name": "CO Veteran Jobs", "screen_name": "tmj_co_vets", "lang": "en", "location": "Colorado Non-Metro", "create_at": date("2011-10-21"), "description": "Follow this account for geo-targeted Veteran job tweets in Colorado Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 148, "friends_count": 116, "statues_count": 61 }, "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-06T10:00:16.000Z"), "id": 684796638775279616, "text": "We're #hiring! Click to apply: Retail Store Shift Supervisor - https://t.co/sV1Z4lmpup #Retail #Norwalk, CA #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.081733,33.9022367"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Retail", "Norwalk", "Job", "Jobs", "CareerArc" }}, "user": { "id": 28709897, "name": "TMJ-CAO Retail Jobs", "screen_name": "tmj_cao_retail", "lang": "en", "location": "Orange County, CA", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Orange County, CA. Need help? Tweet us at @CareerArc!", "followers_count": 470, "friends_count": 313, "statues_count": 1088 }, "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-06T10:00:16.000Z"), "id": 684796638922067968, "text": "@drummerguy22 we should start a vikings podcast near draft time", "in_reply_to_status": -1, "in_reply_to_user": 36537001, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 36537001 }}, "user": { "id": 17332769, "name": "Snownami Severtson", "screen_name": "kylesevertson", "lang": "en", "location": "Minneapolis, MN", "create_at": date("2008-11-11"), "description": "null", "followers_count": 240, "friends_count": 778, "statues_count": 12139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edina, MN", "id": "fe577a7e304136b4", "name": "Edina", "place_type": "city", "bounding_box": rectangle("-93.402474,44.85982 -93.318726,44.930624") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2718188, "cityName": "Edina" } }
+{ "create_at": datetime("2016-01-06T10:00:16.000Z"), "id": 684796638955618304, "text": "Temp 40.7°F Rising, Pressure 30.310in Falling, Dew point 27.5°, Wind SSE 3mph, Rain today 0.00in https://t.co/gPtnr2dDtV #STLwx #MOwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.35555556,38.47694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "STLwx", "MOwx" }}, "user": { "id": 1191304230, "name": "Oakville Weather", "screen_name": "OakvilleWx", "lang": "en", "location": "Oakville, Missouri, USA", "create_at": date("2013-02-17"), "description": "Current conditions on the hour brought to you by @DaveStLou", "followers_count": 105, "friends_count": 80, "statues_count": 24657 }, "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-01-06T10:00:16.000Z"), "id": 684796639001710592, "text": "@FashionStyleTwt https://t.co/aGK0Syn7hh", "in_reply_to_status": 684214503140016128, "in_reply_to_user": 3760687289, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3760687289 }}, "user": { "id": 377844134, "name": "women.com", "screen_name": "womendotcom", "lang": "en", "location": "Los Angeles, California", "create_at": date("2011-09-21"), "description": "The ultimate destination for women", "followers_count": 3192, "friends_count": 1228, "statues_count": 7340 }, "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-06T10:00:16.000Z"), "id": 684796639173738496, "text": "Partly cloudy this afternoon, high 64 (18 C). Low 52 (11 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575370, "name": "Lafayette Weather", "screen_name": "LafayetteWX", "lang": "en", "location": "Lafayette, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Lafayette, LA", "followers_count": 469, "friends_count": 92, "statues_count": 7975 }, "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-06T10:00:16.000Z"), "id": 684796639232430080, "text": "\"Damn ma what's good\" https://t.co/LUFFOnVitO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46042642, "name": "21st bday alcoholism", "screen_name": "kodycoats", "lang": "en", "location": "fascination street", "create_at": date("2009-06-09"), "description": "Inside, as you sleep, I destroy the world. Damned to be, 95 'til infinity. ☭", "followers_count": 1477, "friends_count": 700, "statues_count": 127040 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roseland, CA", "id": "b5c2073756f0f88f", "name": "Roseland", "place_type": "city", "bounding_box": rectangle("-122.743509,38.4124 -122.713599,38.436007") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma", "cityID": 662868, "cityName": "Roseland" } }
+{ "create_at": datetime("2016-01-06T10:00:16.000Z"), "id": 684796639278567424, "text": "Jason, Jayden, Lauren, Annabelle speaking to future @WAISSBISD Students #ChooseSBISD @SBISD \nhttps://t.co/NIsO6wXvWt https://t.co/AeZWeNxFFG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ChooseSBISD" }}, "user_mentions": {{ 190094743, 36954582 }}, "user": { "id": 190094743, "name": "Westchester Academy", "screen_name": "WAISSBISD", "lang": "en", "location": "Houston, TX", "create_at": date("2010-09-12"), "description": "Westchester Academy for International Studies is an IB World School authorized in the MYP, CP and DP Programme.", "followers_count": 549, "friends_count": 448, "statues_count": 4907 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-06T10:00:16.000Z"), "id": 684796640058802176, "text": "I'm happy not gonna lie https://t.co/mtDns3RoCO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 818384606, "name": "Ken", "screen_name": "brownkennedy97", "lang": "en", "location": "null", "create_at": date("2012-09-11"), "description": "✡ jew with a VW✡♡•6.2.14•♡ snap~kllb101", "followers_count": 236, "friends_count": 319, "statues_count": 3491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edgewood, FL", "id": "b8ede6f302482b6c", "name": "Edgewood", "place_type": "city", "bounding_box": rectangle("-81.393182,28.475202 -81.36171,28.500213") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1219900, "cityName": "Edgewood" } }
+{ "create_at": datetime("2016-01-06T10:00:16.000Z"), "id": 684796640146919426, "text": "Listen to Straight Brain by k.suse #np on #SoundCloud\nhttps://t.co/cxGvF2NqfB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "np", "SoundCloud" }}, "user": { "id": 315451034, "name": "k.suse saves", "screen_name": "ikanive666", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2011-06-11"), "description": "jus your ordinary amazing super human...", "followers_count": 217, "friends_count": 624, "statues_count": 1370 }, "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-06T10:00:16.000Z"), "id": 684796640159379456, "text": "#SantaClara, California #Engineering #Job: Sr.Imaging Scientist at Intel Corporation https://t.co/Y53URAcNsp #Intel #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.9552356,37.3541079"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SantaClara", "Engineering", "Job", "Intel", "Jobs", "Hiring" }}, "user": { "id": 23363279, "name": "San Jose Eng. Jobs", "screen_name": "tmj_sjc_eng", "lang": "en", "location": "San Jose, CA", "create_at": date("2009-03-08"), "description": "Follow this account for geo-targeted Engineering job tweets in San Jose, CA. Need help? Tweet us at @CareerArc!", "followers_count": 524, "friends_count": 245, "statues_count": 497 }, "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-06T10:00:16.000Z"), "id": 684796640318885890, "text": "Wholesale shits https://t.co/17cN478JtE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2354914670, "name": "DZNTZ", "screen_name": "GanjaManMark", "lang": "en", "location": "null", "create_at": date("2014-02-21"), "description": "#ETG #VerbalAssault #HHH #RawEmpire #AunthenticMobb gang gagang gang gang", "followers_count": 389, "friends_count": 147, "statues_count": 12268 }, "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-06T10:00:16.000Z"), "id": 684796640507478016, "text": "This #SkilledTrade #job might be a great fit for you: Electrician General Interest - https://t.co/7wnDAy3QR7 #electrician #Decker, MT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-106.864257,45.0119224"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SkilledTrade", "job", "electrician", "Decker" }}, "user": { "id": 3011572896, "name": "Cloud Peak Jobs", "screen_name": "CloudPeakJobs", "lang": "en", "location": "null", "create_at": date("2015-02-06"), "description": "Cloud Peak Energy aims to be a leading energy producer operating in a safe, responsible and caring manner. Check here for current job openings.", "followers_count": 57, "friends_count": 270, "statues_count": 17 }, "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": 30003, "countyName": "Big Horn" } }
+{ "create_at": datetime("2016-01-06T10:00:16.000Z"), "id": 684796640780156928, "text": "You can never find healing and hope for the things you keep hidden!- @joshmbourgeois", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19944733 }}, "user": { "id": 2367540854, "name": "Rhett Griffin", "screen_name": "rpgriff1970", "lang": "en", "location": "null", "create_at": date("2014-03-01"), "description": "Devoted follower of Jesus Christ.Married to a wonderful woman of God and my hero Dawn.All about increasing the Kingdom.", "followers_count": 138, "friends_count": 260, "statues_count": 2414 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Evergreen, MT", "id": "5254f31b2b0d380f", "name": "Evergreen", "place_type": "city", "bounding_box": rectangle("-114.30259,48.195749 -114.243056,48.269329") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30029, "countyName": "Flathead", "cityID": 3025075, "cityName": "Evergreen" } }
+{ "create_at": datetime("2016-01-06T10:00:16.000Z"), "id": 684796640935321600, "text": "Can you recommend anyone for this #Nursing #job? https://t.co/HrJOw7GTn7 #Indianapolis, IN #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.1580423,39.7683765"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "Indianapolis", "Hiring" }}, "user": { "id": 141302910, "name": "Kindred Jobs", "screen_name": "Kindred_Jobs", "lang": "en", "location": "National", "create_at": date("2010-05-07"), "description": "Kindred Healthcare, named a Fortune Magazine Most Admired Company, is the largest provider of post-acute care services in the United States.", "followers_count": 718, "friends_count": 0, "statues_count": 15533 }, "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-01-06T10:00:16.000Z"), "id": 684796641019219968, "text": "@webbc giggity", "in_reply_to_status": 684794765611646977, "in_reply_to_user": 14429183, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14429183 }}, "user": { "id": 4525221, "name": "Aaron Harpole", "screen_name": "harpaa01", "lang": "en", "location": "Santa Monica, CA", "create_at": date("2007-04-13"), "description": "Working on software at @Ring, all around nerd, solver of first world problems. Proud INTJ and also LGBT.", "followers_count": 487, "friends_count": 393, "statues_count": 17024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mid City, Santa Monica", "id": "0b29f6294ac6f324", "name": "Mid City", "place_type": "neighborhood", "bounding_box": rectangle("-118.494537,34.019495 -118.464605,34.041356") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 670000, "cityName": "Santa Monica" } }
+{ "create_at": datetime("2016-01-06T10:00:16.000Z"), "id": 684796641036099584, "text": "Current mood https://t.co/tcSUS6IY6m", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2662370085, "name": "Tony Grappa", "screen_name": "ant_colon23", "lang": "en", "location": "Sayreville, NJ", "create_at": date("2014-07-01"), "description": "Been around the world that's why they call us Wavy", "followers_count": 256, "friends_count": 665, "statues_count": 4109 }, "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-01-06T10:00:16.000Z"), "id": 684796641048530944, "text": "Winn Dixie #Retail #Job: Center Store Area Manager (#NEWROADS, LA) https://t.co/qTbsrNXv9C #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-91.467216,30.688334"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "Job", "NEWROADS", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 59950484, "name": "TMJ-LA Retail Jobs", "screen_name": "tmj_la_retail", "lang": "en", "location": "Louisiana", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in Louisiana Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 405, "friends_count": 301, "statues_count": 1020 }, "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-06T10:00:16.000Z"), "id": 684796641178550272, "text": "@BrooklynStaubs thank you ����", "in_reply_to_status": 684792987549708289, "in_reply_to_user": 472490023, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 472490023 }}, "user": { "id": 863712260, "name": "Noga", "screen_name": "MichaelNoga6", "lang": "en", "location": "Andover Highschool - 17'", "create_at": date("2012-10-05"), "description": "BANG BROS | Baseball/Football #6️⃣. You Don't Score Until You Score", "followers_count": 1485, "friends_count": 933, "statues_count": 8406 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Andover, MN", "id": "219ccc645d458eee", "name": "Andover", "place_type": "city", "bounding_box": rectangle("-93.40681,45.21178 -93.265395,45.304142") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27003, "countyName": "Anoka", "cityID": 2701486, "cityName": "Andover" } }
+{ "create_at": datetime("2016-01-06T10:00:16.000Z"), "id": 684796641392508928, "text": "PARTY ALERT: Ponchatoula, Hammond & Tangipahoa Parish LA. Come Celebrate Elvis Actual Bday Fri 1/8/15 w/ Vince Vance https://t.co/kvPAfdDs8k", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25465083, "name": "Vince Vance", "screen_name": "vincevance", "lang": "en", "location": "New Orleans, Louisiana, USA", "create_at": date("2009-03-19"), "description": "I'm Vince Vance: composer, pianist, entertainer. In 40 yrs I played 15 countries, 34 states, twice White House & Bush W's Inauguration. Of 19 CD albums: 4 #1's!", "followers_count": 729, "friends_count": 220, "statues_count": 2438 }, "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-06T10:00:16.000Z"), "id": 684796641531006978, "text": "I don't take no shit from no nigga...........period", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 233065448, "name": "Si_mazin", "screen_name": "beautybrains5", "lang": "en", "location": "myrtle beach ", "create_at": date("2011-01-01"), "description": "His wifeTheir mother! living my life to the fulliest! couldnt be any happier ! #Blessed", "followers_count": 563, "friends_count": 578, "statues_count": 8862 }, "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-06T10:00:16.000Z"), "id": 684796641547804672, "text": "annoyed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 546203619, "name": "lexi", "screen_name": "lexibeier14", "lang": "en", "location": "cincinnati", "create_at": date("2012-04-05"), "description": "null", "followers_count": 580, "friends_count": 376, "statues_count": 15090 }, "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-01-06T10:00:16.000Z"), "id": 684796641719762945, "text": "Free birthday coffee. ❤️❤️❤️ (@ Caribou Coffee in Woodbury, MN) https://t.co/CgOLEZ8PcJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.90481818,44.94404878"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21377445, "name": "Brigitte Leininger", "screen_name": "BKathleen_", "lang": "en", "location": "Minneapolis, MN", "create_at": date("2009-02-19"), "description": "People like me for my wits. And turned runner overnight. #Adventure", "followers_count": 466, "friends_count": 877, "statues_count": 16512 }, "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-01-06T10:00:16.000Z"), "id": 684796641900150784, "text": "s/o to malik", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pl", "is_retweet": false, "user": { "id": 401178739, "name": ".", "screen_name": "iSelenalynn", "lang": "en", "location": "null", "create_at": date("2011-10-29"), "description": "18. LA", "followers_count": 2753, "friends_count": 1237, "statues_count": 63369 }, "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-06T10:00:16.000Z"), "id": 684796641920991232, "text": "temperature up 56°F -> 59°F\nhumidity up 67% -> 72%\npressure 30.11in rising", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.10542,32.67194"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 184241911, "name": "Weather", "screen_name": "NationalCityCA", "lang": "en", "location": "National City, CA", "create_at": date("2010-08-28"), "description": "Weather updates, forecast, warnings and information for National City, CA. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 16, "friends_count": 1, "statues_count": 20816 }, "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-06T10:00:17.000Z"), "id": 684796641954562048, "text": "Cloudy this afternoon, high 45 (7 C). Low 41 (5 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575409, "name": "Fort Smith Weather", "screen_name": "FortSmithWX", "lang": "en", "location": "Fort Smith, AR", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Fort Smith, AR", "followers_count": 442, "friends_count": 92, "statues_count": 8024 }, "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-06T10:00:17.000Z"), "id": 684796641992425472, "text": "I guess only *1* of my followers is a writer who watches Transparent?\n*skeptically lowers glasses*", "in_reply_to_status": 684794720040673280, "in_reply_to_user": 137220013, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 137220013, "name": "Jameson Fitzpatrick", "screen_name": "twinkpiece", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2010-04-25"), "description": "all opinions are the speaker of the poem's // words in @awl @theoffingmag @preludemag @poetrymagazine etc.", "followers_count": 1314, "friends_count": 795, "statues_count": 8207 }, "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-06T10:00:17.000Z"), "id": 684796642080378880, "text": "#CareerArc #Retail #Job alert: Retail Store Shift Supervisor | CVS Health | #Honolulu, HI https://t.co/fMgF9kYh2f #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CareerArc", "Retail", "Job", "Honolulu", "Jobs", "Hiring" }}, "user": { "id": 28625039, "name": "TMJ-HON Retail Jobs", "screen_name": "tmj_hon_retail", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 377, "friends_count": 311, "statues_count": 447 }, "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-06T10:00:17.000Z"), "id": 684796642139086848, "text": "FDA requested that Bristol-Myers Squibb (which developed Onglyza along with AstraZeneca) conduct postmarketing studies.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 493123841, "name": "Matthews&Associates", "screen_name": "Lawfirm_MA", "lang": "en", "location": "null", "create_at": date("2012-02-15"), "description": "Matthews & Associates helps people injured by dangerous drugs and medical devices recover civil damages in all 50 states & Puerto Rico. We follow our followers.", "followers_count": 17835, "friends_count": 17862, "statues_count": 24356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-06T10:00:17.000Z"), "id": 684796642147565568, "text": "������ add me stunnafly123 https://t.co/e77UWwd7rX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2882588322, "name": "AJ Lee", "screen_name": "thatboikingin_", "lang": "en", "location": "#4evakingin", "create_at": date("2014-10-29"), "description": "Old twitter deleted..! #RIPCHRIS #KINGSHIT ✈️Queens NY, Lauderdale, Kingston JA", "followers_count": 430, "friends_count": 303, "statues_count": 10221 }, "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-06T10:00:17.000Z"), "id": 684796642336333824, "text": "#roscadereyes #lameramera1050 by @djmannyoficial #grepostapp @ Vaughan Place at McLean Gardens https://t.co/iQpNkmkDcj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.0739288,38.9355316"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "roscadereyes", "lameramera1050", "grepostapp" }}, "user_mentions": {{ 27200917 }}, "user": { "id": 242449968, "name": "La Mera Mera 1050AM", "screen_name": "Lameramera1050", "lang": "en", "location": "Washington, DC", "create_at": date("2011-01-24"), "description": "Desde la capital del pais!", "followers_count": 786, "friends_count": 242, "statues_count": 12466 }, "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-06T10:00:17.000Z"), "id": 684796642504015873, "text": "Can you recommend anyone for this #SupplyChain #job? https://t.co/TuMzP3zMep #FortMeade, MD #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.7432645,39.1088867"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupplyChain", "job", "FortMeade", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 395696876, "name": "MD Veteran Jobs", "screen_name": "tmj_md_vets", "lang": "en", "location": "Maryland Non-Metro", "create_at": date("2011-10-21"), "description": "Follow this account for geo-targeted Veteran job tweets in Maryland Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 183, "friends_count": 117, "statues_count": 415 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort George G Meade, MD", "id": "0109616d774c7fa0", "name": "Fort George G Meade", "place_type": "city", "bounding_box": rectangle("-76.77797,39.084364 -76.71328,39.14007") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24003, "countyName": "Anne Arundel", "cityID": 2429400, "cityName": "Fort Meade" } }
+{ "create_at": datetime("2016-01-06T10:00:17.000Z"), "id": 684796642826977283, "text": "@aesandv LMFAO, but dude now I'm am stuck with this small ass umbrella made for like a 4-year old ��", "in_reply_to_status": 684796361254907904, "in_reply_to_user": 1953222247, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1953222247 }}, "user": { "id": 420825817, "name": "olive", "screen_name": "Olivia_Oropesa", "lang": "en", "location": "null", "create_at": date("2011-11-24"), "description": "26E4U", "followers_count": 657, "friends_count": 601, "statues_count": 10771 }, "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-06T10:00:17.000Z"), "id": 684796643099688961, "text": "16 daysssssss @morganseamon ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 65746093 }}, "user": { "id": 17764902, "name": "Ashley Young", "screen_name": "sparklyash90", "lang": "en", "location": "ILLINOIS", "create_at": date("2008-11-30"), "description": "Moving on to better| Exodus 14:14| Jesus is hope to the broken| 2/3/15|", "followers_count": 569, "friends_count": 1878, "statues_count": 12505 }, "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-01-06T10:00:17.000Z"), "id": 684796643103928321, "text": "@dwainchambers hi", "in_reply_to_status": -1, "in_reply_to_user": 2481731814, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2481731814 }}, "user": { "id": 3100062220, "name": "Pierre D. Mcduell", "screen_name": "d_mcduell", "lang": "en", "location": "null", "create_at": date("2015-03-20"), "description": "null", "followers_count": 698, "friends_count": 2047, "statues_count": 10613 }, "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-06T10:00:17.000Z"), "id": 684796643116494848, "text": "Fuck you if you get irritated at someone for sneezing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 534471593, "name": "sludge lord", "screen_name": "AllHailSkatan", "lang": "en", "location": "null", "create_at": date("2012-03-23"), "description": "see you at the bottom", "followers_count": 304, "friends_count": 98, "statues_count": 5023 }, "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-06T10:00:17.000Z"), "id": 684796643263184896, "text": "Pretty soon brother sour mash started tasting sweet...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3838744993, "name": "Adrian Torres", "screen_name": "_AdrianTorres__", "lang": "en", "location": "null", "create_at": date("2015-10-09"), "description": "New Twitter", "followers_count": 42, "friends_count": 55, "statues_count": 338 }, "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-06T10:00:17.000Z"), "id": 684796643972104192, "text": "I asked God for consistently and patience ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 230378137, "name": ".", "screen_name": "_IXXI", "lang": "en", "location": "warren", "create_at": date("2010-12-24"), "description": "A'siya ❤️", "followers_count": 1236, "friends_count": 946, "statues_count": 77891 }, "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-06T10:00:17.000Z"), "id": 684796644261494785, "text": "\"the goddess is in\" ...Day 247 of 365 days of art #goddessart #feminism #intuitiveart #goddess… https://t.co/AvfAnnODkC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.0461154,33.54529193"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "goddessart", "feminism", "intuitiveart", "goddess" }}, "user": { "id": 16653128, "name": "KathyCrabbe", "screen_name": "KathyCrabbe", "lang": "en", "location": "Temecula, California", "create_at": date("2008-10-08"), "description": "artist. writer. creative soul guide. socal.", "followers_count": 2507, "friends_count": 2646, "statues_count": 10846 }, "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-01-06T10:00:17.000Z"), "id": 684796644458500096, "text": "We're #hiring! Read about our latest #job opening here: Baker - https://t.co/DyKLvMH9qy #Tigard, OR #Hospitality #Veterans", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.7499669,45.3937529"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Tigard", "Hospitality", "Veterans" }}, "user": { "id": 2904664975, "name": "Einstein Bros. Jobs", "screen_name": "WorkAtEBB", "lang": "en", "location": "null", "create_at": date("2014-11-19"), "description": "For more information on careers at Einstein Bros. visit http://careers.einsteinnoah.com", "followers_count": 206, "friends_count": 38, "statues_count": 2214 }, "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-01-06T10:00:17.000Z"), "id": 684796644534149120, "text": "@Matt_Fortuna @NDsidBertschy @BGI_LouSomogyi today's Lou's birthday too? His really takes the cake.", "in_reply_to_status": 684796403432951808, "in_reply_to_user": 365648181, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 365648181, 430851155, 348523722 }}, "user": { "id": 21381102, "name": "JJ Stankevitz", "screen_name": "JJStankevitz", "lang": "en", "location": "Chicago", "create_at": date("2009-02-19"), "description": "@CSNChicago #NotreDame Insider. Pinch-hitter on the #WhiteSox beat. @CFTalk contributor. Mizzou grad. #mcfc. Email: jstankevitz@comcastsportsnet.com", "followers_count": 4895, "friends_count": 1386, "statues_count": 47879 }, "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-06T10:00:17.000Z"), "id": 684796645029089281, "text": "Jajajajajajajajajajajajajajajaja !!! No se puede tener al difunto por todos los rincones ! Que pavoso mano ! https://t.co/xOOTjP7i9r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 70545521, "name": "PATRICIA PRINZ", "screen_name": "PATRICIAPRINZ", "lang": "es", "location": "En el imperio messmo ! jajaja", "create_at": date("2009-08-31"), "description": "PROFESIONAL,SOÑADORA,CON EXCESO DE BUEN HUMOR LAS 24 HORAS !!!! ... IG : PRINZPATRICIA", "followers_count": 2227, "friends_count": 464, "statues_count": 43487 }, "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-06T10:00:17.000Z"), "id": 684796645083500544, "text": "Starbucks #Hospitality #Job: Shift Supervisor (US) https://t.co/QAVzJizw2X #Veterans #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.0362967,38.9191485"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "Job", "Veterans", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22453323, "name": "TMJ-WAS HRTA Jobs", "screen_name": "tmj_dc_hrta", "lang": "en", "location": "Washington, DC", "create_at": date("2009-03-01"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in DC Metro. Need help? Tweet us at @CareerArc!", "followers_count": 434, "friends_count": 295, "statues_count": 774 }, "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-06T10:00:17.000Z"), "id": 684796645243006976, "text": "I make Maddy do everything for me because I don't have an id lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3319907799, "name": "Jen", "screen_name": "trapnjen", "lang": "en", "location": "Trap", "create_at": date("2015-06-11"), "description": "FUCK 12 ‼️", "followers_count": 207, "friends_count": 402, "statues_count": 3267 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynnwood-Pricedale, PA", "id": "0b9d4774e7830eeb", "name": "Lynnwood-Pricedale", "place_type": "city", "bounding_box": rectangle("-79.862225,40.117781 -79.82191,40.156656") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42129, "countyName": "Westmoreland", "cityID": 4245732, "cityName": "Lynnwood-Pricedale" } }
+{ "create_at": datetime("2016-01-06T10:00:17.000Z"), "id": 684796645251354629, "text": "suh weh it deh https://t.co/XfnwpRbvfC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 89996381, "name": "A ✨", "screen_name": "_anastasiaaaax", "lang": "en", "location": "On the track ", "create_at": date("2009-11-14"), "description": "get you an atTracKtive girl • ECSU '19", "followers_count": 543, "friends_count": 263, "statues_count": 32012 }, "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-01-06T10:00:17.000Z"), "id": 684796645435961344, "text": "So why do old people get mad about bat flips now? I have a theory https://t.co/mxDZNnR3Hu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 298772584, "name": "Chad Krietemeyer", "screen_name": "ChadAllenKriete", "lang": "en", "location": "Auburn, AL", "create_at": date("2011-05-14"), "description": "Do not go where the path may lead; go instead where there is no path.", "followers_count": 656, "friends_count": 614, "statues_count": 23070 }, "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-01-06T10:00:17.000Z"), "id": 684796645616123904, "text": "Inova Health System: REGISTERED NURSE - Ambulatory Surgery Center (#FALLSCHURCH, VA) https://t.co/yfpTY3kvvX #Nursing #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.2300136,38.858725"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "hashtags": {{ "FALLSCHURCH", "Nursing", "Job", "Jobs", "Hiring" }}, "user": { "id": 22148215, "name": "TMJ-WAS Nursing Jobs", "screen_name": "tmj_dc_nursing", "lang": "en", "location": "Washington, DC", "create_at": date("2009-02-27"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in DC Metro. Need help? Tweet us at @CareerArc!", "followers_count": 386, "friends_count": 306, "statues_count": 687 }, "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-01-06T10:00:17.000Z"), "id": 684796645758775296, "text": "#KAPOLEI, HI #Labor #Job: Truck Driver – CDL Class A or B at CVS Health https://t.co/1DIrG17wfX #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-158.056892,21.3354034"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "KAPOLEI", "Labor", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 118676555, "name": "TMJ-HI Labor Jobs", "screen_name": "tmj_HI_LABOR", "lang": "en", "location": "Hawaii", "create_at": date("2010-03-01"), "description": "Follow this account for geo-targeted General Labor job tweets in Hawaii Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 283, "friends_count": 279, "statues_count": 4 }, "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-06T10:00:17.000Z"), "id": 684796645809242113, "text": "My heart is so happy knowing i have an extra week of break & not one ounce of it has to deal with recruitment bullshit ☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 188103515, "name": "Jordyn Smith", "screen_name": "j_smithhh", "lang": "en", "location": "null", "create_at": date("2010-09-07"), "description": "sorry my phone died", "followers_count": 382, "friends_count": 304, "statues_count": 16717 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hemby Bridge, NC", "id": "0089d580468d861a", "name": "Hemby Bridge", "place_type": "city", "bounding_box": rectangle("-80.653965,35.085002 -80.600484,35.150749") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37179, "countyName": "Union", "cityID": 3730620, "cityName": "Hemby Bridge" } }
+{ "create_at": datetime("2016-01-06T10:00:17.000Z"), "id": 684796646052331520, "text": "Project Engineer - Puerto Rico: (#Juncos) https://t.co/LksNPdTbrU #Ingeniería #Empleo #Empleos #Contratar #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-65.9155215,18.2342465"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "Juncos", "Ingeniería", "Empleo", "Empleos", "Contratar", "CareerArc" }}, "user": { "id": 631446866, "name": "PR Engineering", "screen_name": "tmj_ptr_eng", "lang": "en", "location": "Puerto Rico", "create_at": date("2012-07-09"), "description": "Follow this account for geo-targeted Engineering job tweets in Puerto Rico from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 475, "friends_count": 254, "statues_count": 32 }, "place": { "country": "Puerto Rico", "country_code": "Puerto Rico", "full_name": "Ceiba Norte, Puerto Rico", "id": "d820f4b738a5300c", "name": "Ceiba Norte", "place_type": "city", "bounding_box": rectangle("-65.920975,18.198354 -65.86556,18.242446") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72077, "countyName": "Juncos", "cityID": 7239574, "cityName": "Juncos" } }
+{ "create_at": datetime("2016-01-06T10:00:18.000Z"), "id": 684796646157201408, "text": "#ColoradoSprings, CO #Nursing #Job: RN/Registered Nurse Clinic - Part Time - Days - Penrose... at Centura Health https://t.co/bmRIfwbXmV", "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": {{ "ColoradoSprings", "Nursing", "Job" }}, "user": { "id": 22873940, "name": "Colorado Nursing Job", "screen_name": "tmj_co_nursing", "lang": "en", "location": "Colorado", "create_at": date("2009-03-04"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Colorado Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 514, "friends_count": 297, "statues_count": 752 }, "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-06T10:00:18.000Z"), "id": 684796646203461632, "text": "Just listening to the New Years @PuckOffPodcast and you would have thought @joebartnick was a Blackhawk homer.. Hossa? Really? Come on now!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3442261034, 199878227 }}, "user": { "id": 512500695, "name": "jacob imhoff", "screen_name": "jacobimhoff", "lang": "en", "location": "null", "create_at": date("2012-03-02"), "description": "stay young, stay humble, stay fun", "followers_count": 131, "friends_count": 554, "statues_count": 4016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winter Garden, FL", "id": "dc96a032c98a1ea9", "name": "Winter Garden", "place_type": "city", "bounding_box": rectangle("-81.625844,28.49675 -81.554843,28.595488") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1278250, "cityName": "Winter Garden" } }
+{ "create_at": datetime("2016-01-06T10:00:18.000Z"), "id": 684796646517952512, "text": "Wow I love coffee☕️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 393771612, "name": "Dennãy", "screen_name": "dennis_bench", "lang": "en", "location": "Eugene, OR", "create_at": date("2011-10-18"), "description": "null", "followers_count": 393, "friends_count": 348, "statues_count": 4488 }, "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-06T10:00:18.000Z"), "id": 684796646522249216, "text": "I really don't care about school or college and it really worries me but at the same time as previously stated, 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": 350706075, "name": "mane", "screen_name": "GuessImTony", "lang": "en", "location": "null", "create_at": date("2011-08-07"), "description": "methamphetamine", "followers_count": 158, "friends_count": 134, "statues_count": 27829 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franconia, VA", "id": "7ca522a59e592638", "name": "Franconia", "place_type": "city", "bounding_box": rectangle("-77.184452,38.739182 -77.119991,38.798553") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5129552, "cityName": "Franconia" } }
+{ "create_at": datetime("2016-01-06T10:00:18.000Z"), "id": 684796646639575041, "text": "@NintendoAmerica make a mario kart app for the iphone :)", "in_reply_to_status": -1, "in_reply_to_user": 5162861, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 5162861 }}, "user": { "id": 2157672884, "name": "caleb", "screen_name": "DlNODAN", "lang": "en", "location": "PNW", "create_at": date("2013-10-26"), "description": "be a true heart not a follower", "followers_count": 915, "friends_count": 350, "statues_count": 37046 }, "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-06T10:00:18.000Z"), "id": 684796646719361024, "text": "@gracehelbig Can you please tweet for a fan who urgently needs a handicapped vehicle? https://t.co/trNsxhHj1n #VanForSean", "in_reply_to_status": -1, "in_reply_to_user": 21502768, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VanForSean" }}, "user_mentions": {{ 21502768 }}, "user": { "id": 1651437565, "name": "SeanSmith", "screen_name": "SSmith1138", "lang": "en", "location": "null", "create_at": date("2013-08-06"), "description": "null", "followers_count": 9082, "friends_count": 9780, "statues_count": 6677 }, "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-06T10:00:18.000Z"), "id": 684796646803116032, "text": "SONIC Drive-In: Assistant Manager (#Pflugerville, TX) https://t.co/JKyfWAordq #Hospitality #SONIC #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.607662,30.44537"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Pflugerville", "Hospitality", "SONIC", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22486616, "name": "TMJ-AUS HRTA Jobs", "screen_name": "tmj_aus_hrta", "lang": "en", "location": "Austin, TX", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Austin, TX. Need help? Tweet us at @CareerArc!", "followers_count": 436, "friends_count": 294, "statues_count": 695 }, "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-06T10:00:18.000Z"), "id": 684796647025446912, "text": "@7_GOD__ see? Can't do it bruh", "in_reply_to_status": 684796574673670144, "in_reply_to_user": 45667528, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 45667528 }}, "user": { "id": 222960835, "name": "Riq", "screen_name": "Tariq_Gordon_21", "lang": "en", "location": "Lamar University ", "create_at": date("2010-12-04"), "description": "|Lamar University Cornerback| |Southlake Carroll Alum|", "followers_count": 1577, "friends_count": 1205, "statues_count": 10805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Southlake, TX", "id": "3cf92a36fad89185", "name": "Southlake", "place_type": "city", "bounding_box": rectangle("-97.20323,32.914293 -97.098767,33.007009") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4869032, "cityName": "Southlake" } }
+{ "create_at": datetime("2016-01-06T10:00:18.000Z"), "id": 684796647029620737, "text": "Want to work at Department of Veterans Affairs? We're #hiring in #Columbia, SC! Click for details: https://t.co/L28S0wTEtM #Nursing #Job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.0368211,33.9873389"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Columbia", "Nursing", "Job" }}, "user": { "id": 22287588, "name": "TMJ-CAE Nursing Jobs", "screen_name": "tmj_cae_nursing", "lang": "en", "location": "Columbia, SC", "create_at": date("2009-02-28"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Columbia, SC. Need help? Tweet us at @CareerArc!", "followers_count": 420, "friends_count": 324, "statues_count": 207 }, "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": 45063, "countyName": "Lexington", "cityID": 4512655, "cityName": "Cayce" } }
+{ "create_at": datetime("2016-01-06T10:00:18.000Z"), "id": 684796647033798658, "text": "Cloudy this afternoon, high 43 (6 C). Low 37 (3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26195145, "name": "Wichita Weather", "screen_name": "WichitaWeather", "lang": "en", "location": "Wichita, KS", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Wichita, KS", "followers_count": 1849, "friends_count": 88, "statues_count": 7753 }, "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-06T10:00:18.000Z"), "id": 684796647381966849, "text": "#Hudson, MA #Sales #Job: Verizon (TCC) Sales Consultant at The Cellular Connection https://t.co/hmRdhvWwe2 #Jobs https://t.co/Txrz3AkL0L", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.5661769,42.3917598"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "Hudson", "Sales", "Job", "Jobs" }}, "user": { "id": 2617516549, "name": "TCC Careers", "screen_name": "TCCcareers", "lang": "en", "location": "null", "create_at": date("2014-07-11"), "description": "We hire finest talent in the industry and offer exceptional training to those who want to work toward a career. If this sounds like you, we’d love to talk!", "followers_count": 39, "friends_count": 10, "statues_count": 8029 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hudson, MA", "id": "7af2a905c9d6132a", "name": "Hudson", "place_type": "city", "bounding_box": rectangle("-71.608659,42.366424 -71.483814,42.411814") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531575, "cityName": "Hudson" } }
+{ "create_at": datetime("2016-01-06T10:00:18.000Z"), "id": 684796647583420416, "text": "Wind 2 mph NNE. Barometer 30.12 in, Falling. Temperature 69.3 °F. Rain today 0.04 in. Humidity 65%", "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": 51, "friends_count": 70, "statues_count": 24777 }, "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-06T10:00:18.000Z"), "id": 684796647637794817, "text": "Want to work in #Honolulu, HI? View our latest opening: https://t.co/l6rMf1Gw2Z #Manufacturing #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Honolulu", "Manufacturing", "Job", "Jobs", "Hiring" }}, "user": { "id": 23527770, "name": "TMJ - HON Manuf Jobs", "screen_name": "tmj_hon_manuf", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Manufacturing job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 224, "friends_count": 202, "statues_count": 1 }, "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-06T10:00:18.000Z"), "id": 684796647952400384, "text": "#BroadviewHts, OH #Transportation #Job: Driver - Residential at Rumpke Waste & Recycling https://t.co/f75rDaL2br #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.6851271,41.3139426"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BroadviewHts", "Transportation", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 149169431, "name": "TMJ-OH Transport.", "screen_name": "tmj_OH_transp", "lang": "en", "location": "Ohio", "create_at": date("2010-05-28"), "description": "Follow this account for geo-targeted Transportation job tweets in Ohio Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 334, "friends_count": 283, "statues_count": 339 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Broadview Heights, OH", "id": "e8ab35109a08c935", "name": "Broadview Heights", "place_type": "city", "bounding_box": rectangle("-81.705924,41.269721 -81.640936,41.351146") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3909064, "cityName": "Broadview Heights" } }
+{ "create_at": datetime("2016-01-06T10:00:18.000Z"), "id": 684796648120176641, "text": "@M6NSTER @gm2975 @EricEric992 @Miss_Boots1 @lilhak420 for sure skys only 12 but I know its gonna fly....", "in_reply_to_status": 684795493554216960, "in_reply_to_user": 22937672, "favorite_count": 0, "coordinate": point("-115.1462077,36.0806923"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 22937672, 1541833104, 578269894, 428466077, 1406152146 }}, "user": { "id": 1228035134, "name": "VEGAS METALDAD", "screen_name": "jfield1869", "lang": "en", "location": "null", "create_at": date("2013-02-28"), "description": "metalhead husband dad living in viva las vegas I'll never grow up so I hear , live for the metal", "followers_count": 3579, "friends_count": 2422, "statues_count": 47821 }, "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-06T10:00:18.000Z"), "id": 684796648288071680, "text": "@TreSmithers it really is a nice thing to have.", "in_reply_to_status": 684796301381255169, "in_reply_to_user": 46587084, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 46587084 }}, "user": { "id": 222223608, "name": "Tiff ♕", "screen_name": "t_robss", "lang": "en", "location": "CLE", "create_at": date("2010-12-02"), "description": "19. ♌️", "followers_count": 2264, "friends_count": 876, "statues_count": 59477 }, "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-06T10:00:18.000Z"), "id": 684796648304676864, "text": "Can you recommend anyone for this #Retail #job? https://t.co/uMWdDMMxvV #Diversity IN #Veterans #Hiring https://t.co/V3CquWl8ws", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.7624177,36.4919544"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "Diversity", "Veterans", "Hiring" }}, "user": { "id": 110827861, "name": "DollarGeneralCareers", "screen_name": "DGCareers", "lang": "en", "location": "Nashville, TN", "create_at": date("2010-02-02"), "description": "Let’s Grow Together!\nAt 12,000 stores and counting, we're growing fast and so can you! Joining Dollar General is just the beginning of your best career yet.", "followers_count": 951, "friends_count": 726, "statues_count": 16051 }, "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-01-06T10:00:18.000Z"), "id": 684796648560574464, "text": "I see you @LastNameVerde hitting me with the FB request!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 55682883 }}, "user": { "id": 1070118954, "name": "Gereme Viray", "screen_name": "_viray20", "lang": "en", "location": "null", "create_at": date("2013-01-07"), "description": "Long Beach, CA • CSUF '18 • Zeta Mu Beta", "followers_count": 149, "friends_count": 183, "statues_count": 3206 }, "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-06T10:00:18.000Z"), "id": 684796648829005824, "text": "I can 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": 125477900, "name": "a m a n d a ♡", "screen_name": "ao_bliss", "lang": "en", "location": "null", "create_at": date("2010-03-22"), "description": "null", "followers_count": 294, "friends_count": 224, "statues_count": 1714 }, "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-01-06T10:00:18.000Z"), "id": 684796649328123904, "text": "I never feel any earthquakes ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2191410776, "name": "King.", "screen_name": "HooArrYuu", "lang": "en", "location": "San Bernardino, CA", "create_at": date("2013-11-12"), "description": "l'amour de la paix. ✨", "followers_count": 458, "friends_count": 271, "statues_count": 10842 }, "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-06T10:00:18.000Z"), "id": 684796649416216576, "text": "I felt really happy to wake up to texts from 5 different people. I feel so loved ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26072669, "name": "hannahkin skywalker", "screen_name": "hannerss", "lang": "en", "location": "coloRADo", "create_at": date("2009-03-23"), "description": "20 | bookworm | love yourself so no one has to", "followers_count": 592, "friends_count": 358, "statues_count": 61684 }, "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-06T10:00:18.000Z"), "id": 684796649646886913, "text": "Light wintry mix this afternoon, high 36 (2 C). Low 32 (0 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26199170, "name": "Omaha Weather", "screen_name": "OmahaWX", "lang": "en", "location": "Omaha, Nebraska", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Omaha, Nebraska", "followers_count": 2588, "friends_count": 88, "statues_count": 7746 }, "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-06T10:00:18.000Z"), "id": 684796649651097600, "text": "I'm not just pretty I'm beautiful", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 513920338, "name": "hilary bank$", "screen_name": "_myrakaye", "lang": "en", "location": "okc", "create_at": date("2012-03-03"), "description": "Put yourself first.", "followers_count": 12823, "friends_count": 1256, "statues_count": 18323 }, "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-06T10:00:18.000Z"), "id": 684796649978347520, "text": "Trying to track down bugs to a deploy when your tools only show relative dates is \"about an hour ago's\" worth 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": 1291711, "name": "Trek Glowacki", "screen_name": "trek", "lang": "en", "location": "Chicago, IL", "create_at": date("2007-03-16"), "description": "I helped start @workantile, made mischief with @GrouponEng, now doing the startup thing at @popularpays. @emberjs core team. @paul_irish once called me 'a hero'", "followers_count": 4966, "friends_count": 748, "statues_count": 34615 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Town, Chicago", "id": "6487fa563bbd3b27", "name": "West Town", "place_type": "neighborhood", "bounding_box": rectangle("-87.706977,41.887934 -87.639653,41.916168") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-06T10:00:18.000Z"), "id": 684796650167078913, "text": "WSJ is now on snapchat @WSJ @carlazanoni https://t.co/7Gfy9hs3mg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3108351, 14370917 }}, "user": { "id": 77596982, "name": "wade lambert", "screen_name": "wadeNYC", "lang": "en", "location": "New York CIty", "create_at": date("2009-09-26"), "description": "Night Editor for The Wall Street Journal. @WSJ", "followers_count": 1018, "friends_count": 1209, "statues_count": 6272 }, "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-06T10:00:18.000Z"), "id": 684796650250866688, "text": "@Y4LYFE #backthepac", "in_reply_to_status": 684793436038279168, "in_reply_to_user": 252654805, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "backthepac" }}, "user_mentions": {{ 252654805 }}, "user": { "id": 301765906, "name": "Jimmie Chesh Sports", "screen_name": "CheshSports", "lang": "en", "location": "Salt Lake City", "create_at": date("2011-05-19"), "description": "Writer for @TornyBySports. 7-time fantasy sports Champion. #49ers #BYU #Mariners #UtahJazz and Golf. Sports opinions are separate from my music. (@JimmieChesh)", "followers_count": 801, "friends_count": 655, "statues_count": 25465 }, "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-06T10:00:19.000Z"), "id": 684796650401955840, "text": "#ResidentialAlarm at 4932 Easter Cir. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.438624,28.601987"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ResidentialAlarm", "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": 214, "friends_count": 1, "statues_count": 75529 }, "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-06T10:00:19.000Z"), "id": 684796650485755904, "text": "@SwiftOnSecurity @jessysaurusrex I WANT A LASER DICK MEASURER WITH DAILY STATUS UPDAES.", "in_reply_to_status": 684629420833767425, "in_reply_to_user": 2436389418, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2436389418, 17797084 }}, "user": { "id": 17081391, "name": "Travis Carr", "screen_name": "tmcarr", "lang": "en", "location": "San Francisco, CA", "create_at": date("2008-10-30"), "description": "Engineer @twitter. My tweets are my own.", "followers_count": 888, "friends_count": 1494, "statues_count": 6808 }, "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-06T10:00:19.000Z"), "id": 684796650594811904, "text": "@ooh_martha @SickssHabbit ��", "in_reply_to_status": 684796501806067712, "in_reply_to_user": 1869883710, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1869883710, 573136318 }}, "user": { "id": 3167871157, "name": "Saderzzz", "screen_name": "y_sadiee", "lang": "en", "location": "Yuma, AZ", "create_at": date("2015-04-22"), "description": "high there! ♍️", "followers_count": 426, "friends_count": 208, "statues_count": 9399 }, "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-06T10:00:19.000Z"), "id": 684796650737405952, "text": "Goodmorning ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 589772071, "name": "bombAssBovi ✍.", "screen_name": "Gorgeous_Kovi", "lang": "en", "location": "sc: exclusive_kovi", "create_at": date("2012-05-25"), "description": "18. #TJC17 ✨", "followers_count": 506, "friends_count": 307, "statues_count": 5779 }, "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-06T10:00:19.000Z"), "id": 684796651081342976, "text": "Maurices: Associate Manager (MIT) (#York, PA) https://t.co/pbJaXyRfAX #Retail #LoveFashionLetsTalk #Job #Jobs https://t.co/9VuU5aNJ90", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.6528225,39.9721149"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "York", "Retail", "LoveFashionLetsTalk", "Job", "Jobs" }}, "user": { "id": 2259229658, "name": "maurices careers", "screen_name": "mauricescareers", "lang": "en", "location": "U.S.", "create_at": date("2013-12-23"), "description": "Love Fashion? Join our team! @maurices is a world-class specialty retailer that caters to young-at-heart fashionistas. Check here for #job opportunities.", "followers_count": 342, "friends_count": 26, "statues_count": 2284 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stonybrook, PA", "id": "01c2b7427c7df109", "name": "Stonybrook", "place_type": "city", "bounding_box": rectangle("-76.661383,39.959369 -76.614632,39.994184") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42133, "countyName": "York", "cityID": 4274416, "cityName": "Stonybrook" } }
+{ "create_at": datetime("2016-01-06T10:00:19.000Z"), "id": 684796651286978560, "text": "Just posted a photo @ Maldonado Home https://t.co/YA5sZJH5H7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-120.51047714,46.57437843"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3119230753, "name": "Priscilla Maldonado", "screen_name": "sweetcurves309", "lang": "en", "location": "null", "create_at": date("2015-03-30"), "description": "null", "followers_count": 14, "friends_count": 72, "statues_count": 332 }, "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-06T10:00:19.000Z"), "id": 684796651345674240, "text": "Can't wait to get my dreads on Saturday������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 616843681, "name": "Young Zae ♣️⚽️", "screen_name": "TeamFisher1", "lang": "en", "location": "null", "create_at": date("2012-06-23"), "description": "⚽️ Is Life #⃣ 1⃣ 18 Years Old: Senior At Sandusky High. Scream At Me I'm A Wrestler snapchat: younggzae.", "followers_count": 1579, "friends_count": 2003, "statues_count": 24527 }, "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-06T10:00:19.000Z"), "id": 684796651416899586, "text": "@hostilez45 @swaggyfishtank I'm too lazy to unfollow a whole bunch of ppl eventually Ima just follow certain accounts on that one.", "in_reply_to_status": 684796314572328960, "in_reply_to_user": 198042599, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 198042599, 2536020582 }}, "user": { "id": 148240818, "name": "Dabzingis", "screen_name": "K00LAiDSLiM", "lang": "en", "location": "The Crib", "create_at": date("2010-05-25"), "description": "I'm just chillin. And I hoop. And I like Nike hoop kicks. sc:j_loading0", "followers_count": 886, "friends_count": 1387, "statues_count": 50017 }, "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-06T10:00:19.000Z"), "id": 684796651865812996, "text": "#TheftByShoplifting at 4048 S Semoran Blvd. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3116312,28.5032307"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TheftByShoplifting", "orlpol", "opd" }}, "user": { "id": 39050411, "name": "Police Calls 32822", "screen_name": "orlpol32822", "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": 268, "friends_count": 1, "statues_count": 62390 }, "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-06T10:00:19.000Z"), "id": 684796652029407237, "text": "Megna played well. Either is concerned or a very fast healer lol https://t.co/rHdAE42kw9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2175880059, "name": "Mark Rennie Jr", "screen_name": "MdotRennie7", "lang": "en", "location": "null", "create_at": date("2013-11-10"), "description": "null", "followers_count": 125, "friends_count": 474, "statues_count": 1610 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Plainfield, NJ", "id": "0687589242640f20", "name": "South Plainfield", "place_type": "city", "bounding_box": rectangle("-74.441019,40.543874 -74.386269,40.600935") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3469390, "cityName": "South Plainfield" } }
+{ "create_at": datetime("2016-01-06T10:00:19.000Z"), "id": 684796652108910592, "text": "boy you know I gotta keep you around me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3016195899, "name": "manda", "screen_name": "amaannduuh", "lang": "en", "location": "PDM ", "create_at": date("2015-02-03"), "description": "18", "followers_count": 505, "friends_count": 341, "statues_count": 7139 }, "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-06T10:00:19.000Z"), "id": 684796652218011648, "text": "Morning clouds/afternoon sun this afternoon, high 44 (7 C). Low 34 (1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433220, "name": "St. Louis Weather", "screen_name": "StLouisWeather", "lang": "en", "location": "St. Louis Metro Area", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for St. Louis Metro Area", "followers_count": 3679, "friends_count": 88, "statues_count": 7719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "St. Louis, MO", "id": "60e6df5778ff9dac", "name": "St. Louis", "place_type": "city", "bounding_box": rectangle("-90.320515,38.531852 -90.166409,38.774346") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29510, "countyName": "St. Louis", "cityID": 2965000, "cityName": "St. Louis" } }
+{ "create_at": datetime("2016-01-06T10:00:19.000Z"), "id": 684796652482211842, "text": "You win or you learn, no L's 2k16", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2898429088, "name": "BabyZ", "screen_name": "XeeenaKhan", "lang": "en", "location": "Facing the One", "create_at": date("2014-11-29"), "description": "#FreePalestine ☪ SC:zeenat_k", "followers_count": 291, "friends_count": 219, "statues_count": 12150 }, "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-01-06T10:00:19.000Z"), "id": 684796652486397952, "text": "Proper #steakfrites #herbbutter #french75 #balthazarnyc #stillamust… https://t.co/thn78blYxU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.998159,40.722712"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "steakfrites", "herbbutter", "french75", "balthazarnyc", "stillamust" }}, "user": { "id": 32234976, "name": "JM Lagdameo", "screen_name": "jmlag", "lang": "en", "location": "Manila, Philippines", "create_at": date("2009-04-16"), "description": "Family + Food + Fitness = THE Trifecta", "followers_count": 222, "friends_count": 103, "statues_count": 2069 }, "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-06T10:00:19.000Z"), "id": 684796652775931905, "text": "I enjoy being challenged on my knowledge of state law by travelers. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 422963998, "name": "Brett", "screen_name": "brett_385", "lang": "en", "location": "Logan,C-bus,Nati,Lex,TheVille", "create_at": date("2011-11-27"), "description": "Just trying to go to work and not end up on Youtube", "followers_count": 405, "friends_count": 469, "statues_count": 19021 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lyndon, KY", "id": "24181cf5031a5ef6", "name": "Lyndon", "place_type": "city", "bounding_box": rectangle("-85.617652,38.246411 -85.557324,38.298299") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2148558, "cityName": "Lyndon" } }
+{ "create_at": datetime("2016-01-06T10:00:19.000Z"), "id": 684796652792582146, "text": "@BrightonBoh look how cute this dog is. https://t.co/DqPoCCP8Qd", "in_reply_to_status": -1, "in_reply_to_user": 461326607, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 461326607 }}, "user": { "id": 834689461, "name": "J. Tobias King", "screen_name": "King_JamesXI", "lang": "en", "location": "Western Oregon ", "create_at": date("2012-09-19"), "description": "if sugars so bad, how come Jesus made it taste so good? | Brighton may look mean, but she's nice.", "followers_count": 615, "friends_count": 283, "statues_count": 14227 }, "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-01-06T10:00:19.000Z"), "id": 684796653174263812, "text": "@evan_hill1 Yep, I'm going. The hotel, ticket and flight was under $500.", "in_reply_to_status": 684796024582373376, "in_reply_to_user": 2999001847, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2999001847 }}, "user": { "id": 1717726219, "name": "Rawls Deep", "screen_name": "lukec_hale", "lang": "en", "location": "null", "create_at": date("2013-08-31"), "description": "null", "followers_count": 67, "friends_count": 285, "statues_count": 1807 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cottage Lake, WA", "id": "1148c4a01a9636d4", "name": "Cottage Lake", "place_type": "city", "bounding_box": rectangle("-122.16171,47.711075 -121.988041,47.775931") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5314940, "cityName": "Cottage Lake" } }
+{ "create_at": datetime("2016-01-06T10:00:19.000Z"), "id": 684796653467877377, "text": "Tune in! 12pm��92.3FM #middaymix @radiomaha #djampm @power92chicago #wingstop #winningweds https://t.co/SenlB1ZkgS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.509174,41.597813"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "middaymix", "djampm", "wingstop", "winningweds" }}, "user_mentions": {{ 198985471, 31212891 }}, "user": { "id": 97866323, "name": "dj ampm", "screen_name": "DJAMPMchicago", "lang": "en", "location": "Chicago", "create_at": date("2009-12-19"), "description": "The Hottest Chick In The Game. 92.3FM MixShow DJ @Power92Chicago Mon-Sat 12pm / Fri-Sat 12am #chicagobullspower92guestdj", "followers_count": 1989, "friends_count": 776, "statues_count": 10081 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Power 92", "id": "07d9e43049483002", "name": "Power 92", "place_type": "poi", "bounding_box": rectangle("-87.5091741,41.5978129 -87.509174,41.597813") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18089, "countyName": "Lake", "cityID": 1831000, "cityName": "Hammond" } }
+{ "create_at": datetime("2016-01-06T10:00:19.000Z"), "id": 684796653707083777, "text": "Too many beats not enough time = my life rn ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.964433,40.807084"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 293820286, "name": "GeS", "screen_name": "ThatGeS", "lang": "en", "location": "NYC by way of SPVCE", "create_at": date("2011-05-05"), "description": "musician/producer @redfuton\nbookings/collabs: ges1229@gmail.com", "followers_count": 202, "friends_count": 277, "statues_count": 1455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hooda's Halal Cart", "id": "07d9f6ff4f086001", "name": "Hooda's Halal Cart", "place_type": "poi", "bounding_box": rectangle("-73.9644331,40.8070839 -73.964433,40.807084") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-06T10:00:19.000Z"), "id": 684796653736472576, "text": "@carolinealannaa answer yo phone", "in_reply_to_status": -1, "in_reply_to_user": 898924105, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 898924105 }}, "user": { "id": 1007238180, "name": "Natalie DiCianna", "screen_name": "natdeech", "lang": "en", "location": "null", "create_at": date("2012-12-12"), "description": "Emerson College '18", "followers_count": 136, "friends_count": 133, "statues_count": 2513 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Meridian, PA", "id": "b88406ced98f5f4b", "name": "Meridian", "place_type": "city", "bounding_box": rectangle("-80.012628,40.832433 -79.937488,40.88617") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42019, "countyName": "Butler", "cityID": 4248728, "cityName": "Meridian" } }
+{ "create_at": datetime("2016-01-06T10:00:19.000Z"), "id": 684796653786820610, "text": "#callmejackie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "callmejackie" }}, "user": { "id": 260235439, "name": "Jared Applegate", "screen_name": "Jaroroyo", "lang": "en", "location": "Evansville, IN", "create_at": date("2011-03-03"), "description": "frontman for @Farringtondrive It's such a stubborn reminder one perfect nights not enough\n\nhttps://soundcloud.com/jared-applegate-1", "followers_count": 679, "friends_count": 608, "statues_count": 9470 }, "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-06T10:00:19.000Z"), "id": 684796653841346560, "text": "I'm at Aiello Cigar Bar in Cottleville, MO https://t.co/S0bVGmmEj2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.649972,38.744319"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14282294, "name": "Wayne Schoeneberg", "screen_name": "Wayne_T", "lang": "en", "location": "St. Louis, MO", "create_at": date("2008-04-02"), "description": "Keynote Speaker, best selling author, coach. http://LiveBeyondFear.com Begin your journey on the road to confidence.http://bit.ly/1QQajmG", "followers_count": 784, "friends_count": 908, "statues_count": 5505 }, "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-01-06T10:00:19.000Z"), "id": 684796654147469312, "text": "Make 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": 318617443, "name": "baby t", "screen_name": "halyconamour", "lang": "en", "location": "null", "create_at": date("2011-06-16"), "description": "16.", "followers_count": 1550, "friends_count": 926, "statues_count": 54954 }, "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-06T10:00:19.000Z"), "id": 684796654168309760, "text": "How can you not at this day in age? https://t.co/QX2vfBFjo5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 273747949, "name": "✨ ashlee ✨", "screen_name": "ashleebwilliams", "lang": "en", "location": "Desoto, Tx ", "create_at": date("2011-03-28"), "description": "❤ Messiah James Wade ❤", "followers_count": 714, "friends_count": 678, "statues_count": 7004 }, "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-06T10:00:19.000Z"), "id": 684796654193528832, "text": "@Saudi_Airlines 1st 787-9 HZ-ARA is airworthy. @PiotrG_ @B787fans https://t.co/qbTL8lRSLW", "in_reply_to_status": -1, "in_reply_to_user": 170620083, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 170620083, 3302233925, 1593763190 }}, "user": { "id": 514828179, "name": "Woodys Aeroimages", "screen_name": "woody2190", "lang": "en", "location": "Mukilteo, Washington", "create_at": date("2012-03-04"), "description": "Covering Pacific Northwest Aviation.", "followers_count": 1101, "friends_count": 387, "statues_count": 5632 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mukilteo, WA", "id": "00acddf65083d6cc", "name": "Mukilteo", "place_type": "city", "bounding_box": rectangle("-122.328148,47.859946 -122.219961,47.951882") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5347735, "cityName": "Mukilteo" } }
+{ "create_at": datetime("2016-01-06T10:00:19.000Z"), "id": 684796654201913344, "text": "OFFSITE SALES REPRESENTATIVES - King's Creek Plantation: (#Woodbridge, VA) https://t.co/Dh1TxdJdEN #Sales #va #Job https://t.co/YiUUYOSZkH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.2497049,38.6581722"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Woodbridge", "Sales", "va", "Job" }}, "user": { "id": 3015648286, "name": "King's Creek Jobs", "screen_name": "KingsCreekJobs", "lang": "en", "location": "null", "create_at": date("2015-02-03"), "description": "Check here for current job opportunities with King's Creek Plantation, in beautiful Williamsburg, VA.", "followers_count": 21, "friends_count": 46, "statues_count": 4 }, "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": 51153, "countyName": "Prince William", "cityID": 5149792, "cityName": "Marumsco" } }
+{ "create_at": datetime("2016-01-06T10:00:19.000Z"), "id": 684796654285897728, "text": "#reality #Newyork #people #hinger #homeless #help #savethehomeless #hotmeal #placetostay #subway… https://t.co/lMgR4rNTQ0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0064,40.7142"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "reality", "Newyork", "people", "hinger", "homeless", "help", "savethehomeless", "hotmeal", "placetostay", "subway" }}, "user": { "id": 205543228, "name": "PROD^COL", "screen_name": "hoodlillz_24", "lang": "en", "location": "Harlem ", "create_at": date("2010-10-20"), "description": "Rapper /photographer /Producer born and raised in new york city!! Kik : PRODACOL Gmail : hoodlillz@gmail.com", "followers_count": 537, "friends_count": 953, "statues_count": 13573 }, "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-06T10:00:20.000Z"), "id": 684796654814281729, "text": "Morning rain this afternoon, high 40 (4 C). Low 37 (3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433608, "name": "Kansas City Weather", "screen_name": "KansasCityWX", "lang": "en", "location": "Kansas City Metro Area", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Kansas City Metro Area", "followers_count": 1932, "friends_count": 88, "statues_count": 7839 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "ddabe75a8f44fe5a", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.765917,38.827176 -94.385522,39.356662") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-01-06T10:00:20.000Z"), "id": 684796654986342400, "text": "I'm having it rough today. Can't wait for my morning shift tomorrow and to get shit done. I'll feel so much better about 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": 179146407, "name": "Angela Martin", "screen_name": "Angeyyyyy", "lang": "en", "location": "Mentor, OH", "create_at": date("2010-08-16"), "description": "Do no harm : Take no shit\n♒ #NOH8\nIG: littleang4\n02/23❤", "followers_count": 157, "friends_count": 190, "statues_count": 17704 }, "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-06T10:00:20.000Z"), "id": 684796655141425152, "text": "This dog is normally crazy hyper but you put her in a bath and she's a scared baby haha https://t.co/x6igTpAPdy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2704578374, "name": "Kayden", "screen_name": "chrisdolmethx", "lang": "en", "location": "colorado", "create_at": date("2014-08-03"), "description": "animals over people", "followers_count": 83, "friends_count": 86, "statues_count": 2449 }, "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-06T10:00:20.000Z"), "id": 684796655246405633, "text": "Don't forget today 3pm-7pm at City Hall! 550 Main St Hartford, Come by, have some fun with us! https://t.co/w2H3Uyhu0t", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 52591653, "name": "HartfordMagnetSchool", "screen_name": "HartfordMagnet", "lang": "en", "location": "Hartford, CT ", "create_at": date("2009-06-30"), "description": "Sharing information about the 20 theme based tuition-free magnet schools located throughout Hartford County.", "followers_count": 226, "friends_count": 199, "statues_count": 329 }, "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-06T10:00:20.000Z"), "id": 684796655351103488, "text": "Intel Corporation #FacilitiesMgmt #Job: UPW Group Leader (#Hillsboro, Oregon) https://t.co/bsPwfsJLyV #Intel #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.989827,45.5228939"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FacilitiesMgmt", "Job", "Hillsboro", "Intel", "Jobs", "Hiring" }}, "user": { "id": 71452074, "name": "TMJ-PDX Fac. Mgmt.", "screen_name": "tmj_pdx_facmgmt", "lang": "en", "location": "Portland, OR", "create_at": date("2009-09-03"), "description": "Follow this account for geo-targeted Facilities Management job tweets in Portland, OR from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 150, "friends_count": 129, "statues_count": 32 }, "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-06T10:00:20.000Z"), "id": 684796655372111873, "text": "@OThreaTz some bullshit bro", "in_reply_to_status": 684795859683262465, "in_reply_to_user": 2184662750, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2184662750 }}, "user": { "id": 2967778194, "name": "Etc.", "screen_name": "tyronethomas_", "lang": "en", "location": "Dallas, TX Allen, TX", "create_at": date("2015-01-08"), "description": "October 9th, Libra ♎", "followers_count": 207, "friends_count": 205, "statues_count": 1142 }, "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-06T10:00:20.000Z"), "id": 684796655464386560, "text": "We're #hiring! Click to apply: Store Team Leader - https://t.co/FZW0mQkO8X #Retail #Brevard, NC #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.7150218,35.2591682"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Retail", "Brevard", "Job", "Jobs", "CareerArc" }}, "user": { "id": 59967889, "name": "TMJ-NC Retail Jobs", "screen_name": "tmj_nc_retail", "lang": "en", "location": "North Carolina", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in North Carolina. Need help? Tweet us at @CareerArc!", "followers_count": 393, "friends_count": 305, "statues_count": 757 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brevard, NC", "id": "0191712937fa90c0", "name": "Brevard", "place_type": "city", "bounding_box": rectangle("-82.76233,35.205636 -82.649874,35.291172") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37175, "countyName": "Transylvania", "cityID": 3707720, "cityName": "Brevard" } }
+{ "create_at": datetime("2016-01-06T10:00:20.000Z"), "id": 684796655518916608, "text": "Ptl ���� that my English class is short so I can go back to my dorm and nap hehe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 56533608, "name": "Angelica ✩", "screen_name": "anjellycuhyo", "lang": "en", "location": "Walnut, CA", "create_at": date("2009-07-13"), "description": "UCR // 18 // @ayeeitslee24", "followers_count": 228, "friends_count": 188, "statues_count": 9167 }, "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-06T10:00:20.000Z"), "id": 684796655695171585, "text": "https://t.co/RNShRXVvby", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3295805267, "name": "Claire Z", "screen_name": "RobotPizzaParty", "lang": "en", "location": "null", "create_at": date("2015-05-23"), "description": "Artist, Cosplayer, Gamer, Loser. Follow me on tumblr and instagram @ Robotpizzaparty", "followers_count": 49, "friends_count": 95, "statues_count": 197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eagleview, PA", "id": "018dc44559c0a583", "name": "Eagleview", "place_type": "city", "bounding_box": rectangle("-75.698795,40.052902 -75.657074,40.069516") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42029, "countyName": "Chester", "cityID": 4220662, "cityName": "Eagleview" } }
+{ "create_at": datetime("2016-01-06T10:00:20.000Z"), "id": 684796656194170881, "text": "I'm dying rn https://t.co/PbeK3UeRnz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1861801604, "name": "tDog tha Savvy", "screen_name": "TaronKuhn", "lang": "en", "location": "the bando", "create_at": date("2013-09-13"), "description": "i been A1 since day 1, you niggas boo boo.. #SideNiggaLegend / Young Visionary / MX / Only Positive Energy / If I fucked 6 hoes, I got 6 assists", "followers_count": 288, "friends_count": 151, "statues_count": 9571 }, "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-06T10:00:20.000Z"), "id": 684796656810856449, "text": "Legend @BenSimmons25", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1063034450 }}, "user": { "id": 1449718008, "name": "Philly", "screen_name": "aphilly02", "lang": "en", "location": "Cleveland / Columbus", "create_at": date("2013-05-22"), "description": "Still waiting for that championship parade.", "followers_count": 410, "friends_count": 842, "statues_count": 25361 }, "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-06T10:00:20.000Z"), "id": 684796656928165888, "text": "@TooJuiceedUp just lemme know!", "in_reply_to_status": 684796435510853632, "in_reply_to_user": 322913509, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 322913509 }}, "user": { "id": 1518481333, "name": "Marcais Hudson", "screen_name": "HudsonMarcais", "lang": "en", "location": "null", "create_at": date("2013-06-14"), "description": "null", "followers_count": 1197, "friends_count": 986, "statues_count": 20738 }, "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-06T10:00:20.000Z"), "id": 684796657238683648, "text": "@Coach_Coreno @tori_pelphrey love you coach!!! See you at the game tomorrow!", "in_reply_to_status": 684794031256268800, "in_reply_to_user": 1031525916, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1031525916, 326624896 }}, "user": { "id": 453202978, "name": "Aleah Pelphrey", "screen_name": "Apelph11", "lang": "en", "location": "Gallipolis, OH/Piketon,OH ", "create_at": date("2012-01-02"), "description": "URG. Volleyball. Jesus lover. Jeremiah 29:11. Your future childrens kindergarten teacher. ✏️ #ZF34", "followers_count": 1253, "friends_count": 886, "statues_count": 12768 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Piketon, OH", "id": "013fd00573916442", "name": "Piketon", "place_type": "city", "bounding_box": rectangle("-83.024188,39.052091 -82.976453,39.076503") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39131, "countyName": "Pike", "cityID": 3962708, "cityName": "Piketon" } }
+{ "create_at": datetime("2016-01-06T10:00:20.000Z"), "id": 684796657347751936, "text": "@ItsGarretSelves https://t.co/yes3x5znp7", "in_reply_to_status": -1, "in_reply_to_user": 173658561, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 173658561 }}, "user": { "id": 1566939445, "name": "Tyler", "screen_name": "TylerDonnellyTD", "lang": "en", "location": "Fenton, MI", "create_at": date("2013-07-03"), "description": "null", "followers_count": 315, "friends_count": 731, "statues_count": 3613 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fenton, MI", "id": "d1e16b409e42e6df", "name": "Fenton", "place_type": "city", "bounding_box": rectangle("-83.80635,42.75267 -83.686497,42.841079") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2627760, "cityName": "Fenton" } }
+{ "create_at": datetime("2016-01-06T10:00:20.000Z"), "id": 684796657452503040, "text": "@_queen_x is statistics hard?", "in_reply_to_status": -1, "in_reply_to_user": 1494721134, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1494721134 }}, "user": { "id": 3248278280, "name": "Ky ❤️", "screen_name": "Kylee722", "lang": "en", "location": "null", "create_at": date("2015-06-17"), "description": "instagram: kylee_722", "followers_count": 234, "friends_count": 197, "statues_count": 1620 }, "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-01-06T10:00:20.000Z"), "id": 684796657477619712, "text": "@Jennifarttt skipped out on killing zombies....@ZachMCF come kill zombies with me :) and @Sir_Villa_25", "in_reply_to_status": -1, "in_reply_to_user": 1152535124, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1152535124, 400390692, 828200988 }}, "user": { "id": 74871097, "name": "Daniel Sandoval", "screen_name": "Dannyy_Yo", "lang": "en", "location": "Round Rock, TX", "create_at": date("2009-09-16"), "description": "I scream for a band called My Struggle, My Pride. Check us out! Click on the link! Do it! :D \\m/\nFollow me on:\nIG: Dannyy_Yoo\nSnapChat: Dannyy_Yo", "followers_count": 43, "friends_count": 46, "statues_count": 548 }, "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-06T10:00:20.000Z"), "id": 684796657595232258, "text": "#porchinitwithlowes (at @Lowes Home Improvement in Buckeye, AZ) https://t.co/UfssJrenKd https://t.co/4lUslbYAYy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.559372,33.440577"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "porchinitwithlowes" }}, "user_mentions": {{ 19336500 }}, "user": { "id": 3322866258, "name": "Marci Woods", "screen_name": "PhoenixPorchie", "lang": "en", "location": "Phoenix, Arizona", "create_at": date("2015-08-21"), "description": "Interested in a home improvement project? Check out http://www.Porch.com to get started. Our goal is to help you love your home!", "followers_count": 42, "friends_count": 63, "statues_count": 63 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buckeye, AZ", "id": "0015cc0d71d49e19", "name": "Buckeye", "place_type": "city", "bounding_box": rectangle("-112.625877,33.355798 -112.461428,33.515442") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 407940, "cityName": "Buckeye" } }
+{ "create_at": datetime("2016-01-06T10:00:20.000Z"), "id": 684796657716727808, "text": "9 d. Celsius mabuti sa friday Jan 8 MBL wlang ulan #ALDUB25thWeeksary ������ pagmasdan ang ulan unti unting pumapatak�� https://t.co/F0QMYYpUcK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "ALDUB25thWeeksary" }}, "user": { "id": 1374968269, "name": "RichMaine02_16/calif", "screen_name": "ginamarasigan1", "lang": "en", "location": "Long Beach, CA", "create_at": date("2013-04-23"), "description": "Life is short, Break the rules, Forgive quickly, Kiss slowly, Love truly, laugh uncontrollably and never regret anything that madeyou Smile gina0266@yahoo.com", "followers_count": 47, "friends_count": 83, "statues_count": 1027 }, "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-06T10:00:20.000Z"), "id": 684796657775448065, "text": "Mostly sunny this afternoon, high 50 (10 C). Low 36 (2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433972, "name": "Memphis Weather", "screen_name": "MemphisWX", "lang": "en", "location": "Memphis, TN", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Memphis, TN", "followers_count": 1688, "friends_count": 88, "statues_count": 7751 }, "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-06T10:00:20.000Z"), "id": 684796657976774656, "text": "@bekahproctor who still plays words with friends...", "in_reply_to_status": 684588660872708096, "in_reply_to_user": 320533294, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 320533294 }}, "user": { "id": 457752335, "name": "Heather Quick", "screen_name": "heatherquick711", "lang": "en", "location": "null", "create_at": date("2012-01-07"), "description": "LSU ΚΑΘ", "followers_count": 364, "friends_count": 668, "statues_count": 916 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Breckenridge, CO", "id": "00401a907e12e1b1", "name": "Breckenridge", "place_type": "city", "bounding_box": rectangle("-106.0747,39.460879 -105.997415,39.550435") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8117, "countyName": "Summit", "cityID": 808400, "cityName": "Breckenridge" } }
+{ "create_at": datetime("2016-01-06T10:00:20.000Z"), "id": 684796658161442816, "text": "Feliz día de los reyes ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 1322708659, "name": "Stacey Castro Cruz", "screen_name": "stace_face24", "lang": "en", "location": "null", "create_at": date("2013-04-02"), "description": "Washington Heights ||•PURA VIDA•||", "followers_count": 430, "friends_count": 498, "statues_count": 4197 }, "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-06T10:00:20.000Z"), "id": 684796658278780928, "text": "@moviesovernewz https://t.co/aGK0Syn7hh", "in_reply_to_status": 684204111324164096, "in_reply_to_user": 3240418900, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3240418900 }}, "user": { "id": 377844134, "name": "women.com", "screen_name": "womendotcom", "lang": "en", "location": "Los Angeles, California", "create_at": date("2011-09-21"), "description": "The ultimate destination for women", "followers_count": 3192, "friends_count": 1228, "statues_count": 7342 }, "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-06T10:00:20.000Z"), "id": 684796658517962754, "text": "@XO_Tamar @J_Abron5 @_xPrettyGang mannnnn yeen gone do shit", "in_reply_to_status": 684796421690769408, "in_reply_to_user": 1029199574, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1029199574, 3105415274, 258516888 }}, "user": { "id": 299138503, "name": "OTS/SauceSquad", "screen_name": "Tayda_TooMuch", "lang": "en", "location": "iG: @_taydaruks", "create_at": date("2011-05-15"), "description": "..... I'm kinda a big deal , I'm a dad.... and I really don't have a filter for shit", "followers_count": 932, "friends_count": 699, "statues_count": 20232 }, "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-01-06T10:00:21.000Z"), "id": 684796659050627073, "text": "just started crying bc the family on family feud just won the car and the money, what is 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": 855501602, "name": "cassidy ♒️", "screen_name": "cASSuscilowski4", "lang": "en", "location": "null", "create_at": date("2012-09-30"), "description": "Id rather do nothing and be happy then do something i know i dont love #ripval #SVC17", "followers_count": 837, "friends_count": 829, "statues_count": 14866 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maumee, OH", "id": "ccf3f9123267d97d", "name": "Maumee", "place_type": "city", "bounding_box": rectangle("-83.718691,41.527192 -83.606563,41.593643") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39095, "countyName": "Lucas", "cityID": 3948342, "cityName": "Maumee" } }
+{ "create_at": datetime("2016-01-06T10:00:21.000Z"), "id": 684796659306397700, "text": "@yazzyy9 Your definition of cooking is probably just microwaving your leftovers from Olive Garden. Lol!", "in_reply_to_status": 684796369131835392, "in_reply_to_user": 1624743612, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1624743612 }}, "user": { "id": 630405471, "name": "num", "screen_name": "NammyPoo", "lang": "en", "location": "☁️ Chi Town ☁️", "create_at": date("2012-07-08"), "description": "Do you even got the ⛽️ like that?", "followers_count": 478, "friends_count": 250, "statues_count": 14092 }, "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-06T10:00:21.000Z"), "id": 684796659444756480, "text": "@EDDiEHiGHROLLER https://t.co/brPh4tCteB", "in_reply_to_status": 684796513092939776, "in_reply_to_user": 1534455768, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1534455768 }}, "user": { "id": 1003183008, "name": "gyvegRace", "screen_name": "rraciegothe", "lang": "en", "location": "null", "create_at": date("2012-12-10"), "description": "Lalo//Mi Amor", "followers_count": 492, "friends_count": 200, "statues_count": 16900 }, "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-01-06T10:00:21.000Z"), "id": 684796659570614272, "text": "Funniest thing I've seen in a while https://t.co/vqYjLOssXF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 244258494, "name": "Teeee", "screen_name": "TeemakHayes", "lang": "en", "location": "The Valley", "create_at": date("2011-01-28"), "description": "null", "followers_count": 873, "friends_count": 544, "statues_count": 16016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:21.000Z"), "id": 684796659600089089, "text": "It's only an excuse to get back in a situation that at one point you knew you wanted out of because it wasn't serving God's plan 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": 301862079, "name": "Shanell Tyler", "screen_name": "LaBelleShanell", "lang": "en", "location": "Memphis, TN", "create_at": date("2011-05-19"), "description": "Saved. Law student at UofM. Woman of God. Friend. Fulltime Prayer Partner. Daughter of a King. Victoria Grace's Sister ❤️looking to be a light in dark places", "followers_count": 453, "friends_count": 308, "statues_count": 22465 }, "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-06T10:00:21.000Z"), "id": 684796659784523777, "text": "@Heyjessss__ text me", "in_reply_to_status": 684796589781692416, "in_reply_to_user": 522785478, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 522785478 }}, "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? I the man Since the Saints Had Joe Horn!", "followers_count": 1041, "friends_count": 670, "statues_count": 60921 }, "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-06T10:00:21.000Z"), "id": 684796659834826752, "text": "Favorite Health Care Staffing #Nursing : Registered Nurse | RN | CCU... (#PUYALLUP, WA) https://t.co/rhFvSMsMMv https://t.co/s3xJGANj1t", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.2928974,47.1853785"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "PUYALLUP" }}, "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": 64, "friends_count": 99, "statues_count": 2519 }, "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-06T10:00:21.000Z"), "id": 684796660283617280, "text": "I want them hunnids......hunnids ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1449421784, "name": "BOSS Daytron", "screen_name": "OB4_Life", "lang": "en", "location": "In The Gym, Texas", "create_at": date("2013-05-22"), "description": "I just wanna Ballout #RIPMARCUS", "followers_count": 599, "friends_count": 506, "statues_count": 7223 }, "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-06T10:00:21.000Z"), "id": 684796660535431170, "text": "I'm only one call away, I'll be there to save the day. Superman got nothing on me. I'm only one call awayyy.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2513047397, "name": "Hannah Willis", "screen_name": "h_freakinwillis", "lang": "en", "location": "hartwell", "create_at": date("2014-04-27"), "description": "rip vezzy baby • c/o 2015 • sc: hannahhh_willis •", "followers_count": 597, "friends_count": 256, "statues_count": 4986 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hartwell, GA", "id": "63604225ab6c53c3", "name": "Hartwell", "place_type": "city", "bounding_box": rectangle("-82.954074,34.318397 -82.904799,34.376009") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13147, "countyName": "Hart", "cityID": 1337144, "cityName": "Hartwell" } }
+{ "create_at": datetime("2016-01-06T10:00:21.000Z"), "id": 684796660736643073, "text": "I really 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": 3484300453, "name": "ANA", "screen_name": "anaachavarriaa1", "lang": "en", "location": "null", "create_at": date("2015-09-07"), "description": "Basketball & Track", "followers_count": 234, "friends_count": 251, "statues_count": 400 }, "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-01-06T10:00:21.000Z"), "id": 684796660992573440, "text": "Gotta remember the super trash Superman Returns https://t.co/TwmRn1km5I", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 222279104, "name": "E.", "screen_name": "LowProCessna", "lang": "en", "location": "NCAT", "create_at": date("2010-12-02"), "description": "Won't sell you no dreams, but the inspiration is free. ΦΒΣ 20Η15.", "followers_count": 1445, "friends_count": 1168, "statues_count": 111711 }, "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-06T10:00:21.000Z"), "id": 684796661038735360, "text": "@Charcobo if I had the money I pre and be done with it I also have a feeling that anyone watching kind of knew every orders were coming soon", "in_reply_to_status": 684795663297593344, "in_reply_to_user": 74415789, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 74415789 }}, "user": { "id": 44522899, "name": "FURRY SUPLEX @ MWFF", "screen_name": "SwiftLPRD", "lang": "en", "location": "Detroit MI", "create_at": date("2009-06-03"), "description": "Cat that plays fighting games. I like them. They don't like me! Just another try hard that isn't doing anything special. Yet. :3 #BBCP #XRD #SFIV #MH4U #Melty", "followers_count": 927, "friends_count": 1276, "statues_count": 36401 }, "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-06T10:00:21.000Z"), "id": 684796661521068032, "text": "Plug talk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 131350197, "name": "January 7th™", "screen_name": "TooReal_DotCom", "lang": "en", "location": "Florida, USA", "create_at": date("2010-04-09"), "description": "Juice™.", "followers_count": 608, "friends_count": 509, "statues_count": 14074 }, "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-01-06T10:00:21.000Z"), "id": 684796661596459008, "text": "This guy got color eyes no wonder I find him attractive ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1263811082, "name": "Gaby", "screen_name": "_SkyMightFall", "lang": "en", "location": "Houston, TX", "create_at": date("2013-03-13"), "description": "Leave in peace or leave in pieces .", "followers_count": 419, "friends_count": 436, "statues_count": 3704 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-06T10:00:21.000Z"), "id": 684796661730787328, "text": "I think I'm too humble 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": 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": 1723, "friends_count": 1451, "statues_count": 118460 }, "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-01-06T10:00:21.000Z"), "id": 684796661957263364, "text": "Hard �� and a shoulder to lean on", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2715075657, "name": "Selfmade", "screen_name": "SelfmadeZonta_", "lang": "en", "location": "null", "create_at": date("2014-07-16"), "description": "the next image is my creation.", "followers_count": 517, "friends_count": 238, "statues_count": 8682 }, "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-06T10:00:21.000Z"), "id": 684796662091509761, "text": "Ain't enough hours in 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": 282158619, "name": "QUARON", "screen_name": "QUARONsp", "lang": "en", "location": "SouthPhilly/DaBottom", "create_at": date("2011-04-14"), "description": "#OTF ......if you know me you know wassup #Humble", "followers_count": 1080, "friends_count": 915, "statues_count": 48493 }, "place": { "country": "United States", "country_code": "United States", "full_name": "King of Prussia, PA", "id": "23b12afb6fe7507a", "name": "King of Prussia", "place_type": "city", "bounding_box": rectangle("-75.420016,40.071936 -75.343479,40.117633") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42091, "countyName": "Montgomery", "cityID": 4239736, "cityName": "King of Prussia" } }
+{ "create_at": datetime("2016-01-06T10:00:21.000Z"), "id": 684796662221373440, "text": "It's fuckin green to white technology", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2255813618, "name": "ü pissed", "screen_name": "Maxwell_Lyman", "lang": "en", "location": "Kwik-E-Mart", "create_at": date("2013-12-20"), "description": "the gental fist god.", "followers_count": 91, "friends_count": 274, "statues_count": 457 }, "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-06T10:00:21.000Z"), "id": 684796662737272832, "text": "THIS IS NOT WORK APPROPRIATE https://t.co/LIDUvvAEFf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3268559538, "name": "Misia || Jan 21", "screen_name": "branmuffinbob", "lang": "en", "location": "The Scorch", "create_at": date("2015-07-04"), "description": "#BobMorleyDefenseSquad/One time Bob Morley followed me on accident 9.14.15/Excessive Multishipper/(she/her)", "followers_count": 532, "friends_count": 404, "statues_count": 10999 }, "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-06T10:00:21.000Z"), "id": 684796662896693248, "text": "Mostly sunny this afternoon, high 47 (8 C). Low 34 (1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119697197, "name": "Huntsville Weather", "screen_name": "HuntsvilleWX", "lang": "en", "location": "null", "create_at": date("2010-03-04"), "description": "null", "followers_count": 672, "friends_count": 65, "statues_count": 6062 }, "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-06T10:00:22.000Z"), "id": 684796662959726593, "text": "@grassfed_butter @omaryak @USAhuntress ROFLOL", "in_reply_to_status": 684796482176745472, "in_reply_to_user": 2854075845, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 2854075845, 14102015, 2411071087 }}, "user": { "id": 716048887, "name": "Dan Roberts", "screen_name": "DRoberts556", "lang": "en", "location": "null", "create_at": date("2012-07-25"), "description": "Firearms Instructor, Father, 2A Scholar and Activist, columnist at http://Ammoland.com , Proud 3 percenter", "followers_count": 506, "friends_count": 346, "statues_count": 12503 }, "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-06T10:00:22.000Z"), "id": 684796663722934272, "text": "@MiKaylaa___ Preciate that!", "in_reply_to_status": 684790595701895169, "in_reply_to_user": 744584720, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 744584720 }}, "user": { "id": 153670458, "name": "Charles Scott", "screen_name": "cwsjr_", "lang": "en", "location": "Meridian, MS", "create_at": date("2010-06-08"), "description": "Blessed and highly favored! MCMBB #0", "followers_count": 955, "friends_count": 696, "statues_count": 12577 }, "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-06T10:00:22.000Z"), "id": 684796663794253824, "text": "Happy birthday Abbie! Hope it's a great one! @AbidaleW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 163214862 }}, "user": { "id": 553298248, "name": "Jamie Bryan", "screen_name": "jamie_b_yc1", "lang": "en", "location": "West Korea", "create_at": date("2012-04-13"), "description": "⠀⠀⠀⠀⠀⠀Whoever is the owner of the ⠀⠀⠀ ⠀⠀⠀white sedan, you left your lights on", "followers_count": 404, "friends_count": 336, "statues_count": 3906 }, "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-06T10:00:22.000Z"), "id": 684796663962185730, "text": "Why is the 1st question people ask when they find out I'm engaged are u having a summer wedding? Do I look like a summer person? #paleprobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "paleprobs" }}, "user": { "id": 130679514, "name": "Brielle P", "screen_name": "brielleliz", "lang": "en", "location": "Hamilton, NJ", "create_at": date("2010-04-07"), "description": "Hope is putting faith to work when doubting would be easier.\nSocial Media Ambassador for NJ CASA #njcasasSMA for info :)", "followers_count": 55, "friends_count": 263, "statues_count": 1942 }, "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-06T10:00:22.000Z"), "id": 684796664159158272, "text": "@thatslife21 it don't work like that around here I got time for it", "in_reply_to_status": 684796422135263232, "in_reply_to_user": 413448566, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 413448566 }}, "user": { "id": 320884439, "name": "6 Mile", "screen_name": "313doe", "lang": "en", "location": "Chasing Money", "create_at": date("2011-06-20"), "description": "Just a Detroit nigga living in Tx #IfItAintFoodIAintEatingIt", "followers_count": 3115, "friends_count": 2991, "statues_count": 159565 }, "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-06T10:00:22.000Z"), "id": 684796664419344385, "text": "I'm moving. https://t.co/dgWlGWiOe8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1051389956, "name": "Jed ‼️", "screen_name": "JedidiahRamos22", "lang": "en", "location": "Zephyrhills, FL", "create_at": date("2012-12-31"), "description": "sc:JedRamos5|ZCA'16|#0|", "followers_count": 639, "friends_count": 320, "statues_count": 17689 }, "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-01-06T10:00:22.000Z"), "id": 684796664620683264, "text": "Let's hope today is 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": 344811292, "name": "Jus Dsyr", "screen_name": "JordanGrigg19", "lang": "en", "location": "Marlton, NJ", "create_at": date("2011-07-29"), "description": "Sales Director for @S6_eSports / Manager for @Justus_Pro /Professional Body Boarder for Carbon/ Email: jordan@sectorsixapparel.com", "followers_count": 21002, "friends_count": 8133, "statues_count": 11797 }, "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-01-06T10:00:22.000Z"), "id": 684796664784252930, "text": "@TasiasWord WE LOVE YA N IM SO READY TO HEAR MY #1 ARTIST/ FAVORITE FEMALE NEW SINGLE ❤��❤��", "in_reply_to_status": -1, "in_reply_to_user": 161802684, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 161802684 }}, "user": { "id": 2868625955, "name": "CHOCOLATE", "screen_name": "nicolejones3434", "lang": "en", "location": "NEW YORK", "create_at": date("2014-11-09"), "description": "DON'T GET IT TWISTED", "followers_count": 80, "friends_count": 395, "statues_count": 916 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Utica, NY", "id": "41f575b7eebcd4b7", "name": "Utica", "place_type": "city", "bounding_box": rectangle("-75.295291,43.050251 -75.158681,43.189108") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36065, "countyName": "Oneida", "cityID": 3676540, "cityName": "Utica" } }
+{ "create_at": datetime("2016-01-06T10:00:22.000Z"), "id": 684796664880705536, "text": "@JPCzerniak Tengo que esperar a llegar al apa pq aqui no se que le pasa que el cel no se conecta.", "in_reply_to_status": 684796122640986112, "in_reply_to_user": 90526062, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 90526062 }}, "user": { "id": 1319576966, "name": "Giovanny DC", "screen_name": "giovabatista", "lang": "en", "location": "Ponce, Puerto Rico", "create_at": date("2013-03-31"), "description": "Future Chiropractor. Baloncesto y la salsa.", "followers_count": 469, "friends_count": 893, "statues_count": 16002 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valley Falls, SC", "id": "83f69050c89a6a95", "name": "Valley Falls", "place_type": "city", "bounding_box": rectangle("-82.024291,34.989317 -81.937438,35.028922") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45083, "countyName": "Spartanburg", "cityID": 4573465, "cityName": "Valley Falls" } }
+{ "create_at": datetime("2016-01-06T10:00:22.000Z"), "id": 684796665195184128, "text": "1. #Twitter10k\n2. #NHLAllStar\n3. Leo Komarov\n4. Guy Turcotte\n5. #RAEForecast\n\n2016/1/6 11:51 CST #trndnl https://t.co/SHjd3tkINE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.309,56.9547"), "retweet_count": 0, "lang": "hu", "is_retweet": false, "hashtags": {{ "Twitter10k", "NHLAllStar", "RAEForecast", "trndnl" }}, "user": { "id": 1270239397, "name": "Trendinalia Canada", "screen_name": "trendinaliaCA", "lang": "es", "location": "Canada", "create_at": date("2013-03-15"), "description": "Information about Twitter's Trending Topics in Canada", "followers_count": 1250, "friends_count": 7, "statues_count": 257722 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Division No. 23, Unorganized, Manitoba", "id": "654fd5617c7361d0", "name": "Division No. 23, Unorganized", "place_type": "city", "bounding_box": rectangle("-102.009382,55.158501 -88.989166,60.000283") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-06T10:00:22.000Z"), "id": 684796665388240898, "text": "The 15 coolest coffee shops I've ever been to have been in Salt Lake City", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3128945823, "name": "Tyler Hicks", "screen_name": "Tylerbhicks", "lang": "en", "location": "ga", "create_at": date("2015-03-30"), "description": "coffee", "followers_count": 282, "friends_count": 136, "statues_count": 81 }, "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-06T10:00:22.000Z"), "id": 684796665455218688, "text": "Partly cloudy this afternoon, high 50 (10 C). Low 36 (2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119698223, "name": "Birmingham Weather", "screen_name": "BirminghamWX", "lang": "en", "location": "Birmingham, AL", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Birmingham, AL", "followers_count": 503, "friends_count": 68, "statues_count": 6074 }, "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-06T10:00:22.000Z"), "id": 684796665727815681, "text": "#Swiftwater, Pennsylvania #Pharmaceutical #Job: Production Tech at Sanofi Pasteur https://t.co/G1r44dHBq7 #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.3266416,41.0881409"), "retweet_count": 0, "lang": "cy", "is_retweet": false, "hashtags": {{ "Swiftwater", "Pharmaceutical", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 103195190, "name": "TMJ-PA Pharm. Jobs", "screen_name": "tmj_pa_pharm", "lang": "en", "location": "Pennsylvania", "create_at": date("2010-01-08"), "description": "Follow this account for geo-targeted Pharm./Biotech job tweets in Pennsylvania Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 384, "friends_count": 308, "statues_count": 87 }, "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": 42089, "countyName": "Monroe" } }
+{ "create_at": datetime("2016-01-06T10:00:22.000Z"), "id": 684796665878843394, "text": "I need everyone to go to Swanson and tell him to shave his beard like tony stark", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3345843071, "name": "Savanna Lynn", "screen_name": "SevLynnSays", "lang": "en", "location": "mentally in NYC", "create_at": date("2015-06-25"), "description": "my youth is yours", "followers_count": 232, "friends_count": 534, "statues_count": 1501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grants Pass, OR", "id": "00a8b2b85821e087", "name": "Grants Pass", "place_type": "city", "bounding_box": rectangle("-123.360581,42.386741 -123.283133,42.471891") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41033, "countyName": "Josephine", "cityID": 4130550, "cityName": "Grants Pass" } }
+{ "create_at": datetime("2016-01-06T10:00:22.000Z"), "id": 684796666009006080, "text": "Young Rich Nigga with the Melo Deal you Needa CoME to OVO FOR A BETTER DEAL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 381842532, "name": "MR. L.I.T", "screen_name": "J_Hardaway", "lang": "en", "location": "BPT, CT", "create_at": date("2011-09-28"), "description": "OVO Halloween Gang IG: Jxmmy_Sosa Snapchat: A1_Hardaway", "followers_count": 1226, "friends_count": 866, "statues_count": 104110 }, "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-01-06T10:00:22.000Z"), "id": 684796666050928640, "text": "Self made, and it's only getting better �� #CantKnockMyHustle", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CantKnockMyHustle" }}, "user": { "id": 248742316, "name": "GiGi", "screen_name": "EyeMezmerizee", "lang": "en", "location": "money is the motive", "create_at": date("2011-02-07"), "description": "Juliette Isabel Jianni Rose Julius Alberto", "followers_count": 168, "friends_count": 235, "statues_count": 4579 }, "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-01-06T10:00:22.000Z"), "id": 684796666432630785, "text": "@davidcrosss is going on tour...I need to attend that oh my god", "in_reply_to_status": -1, "in_reply_to_user": 4356430036, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4356430036 }}, "user": { "id": 34795013, "name": "A.", "screen_name": "whatupallie", "lang": "en", "location": "Cincinnati, OH", "create_at": date("2009-04-23"), "description": "CLE➡️UC", "followers_count": 1358, "friends_count": 589, "statues_count": 46870 }, "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-06T10:00:22.000Z"), "id": 684796666659139585, "text": "@_AshKetchum___ I see u being dat mom dat don't age", "in_reply_to_status": 684796544852357120, "in_reply_to_user": 73542163, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 73542163 }}, "user": { "id": 1326297024, "name": "Lit GOD", "screen_name": "Cakeddupcarl", "lang": "en", "location": "null", "create_at": date("2013-04-03"), "description": "#schwayzznigga #mrbottlepopper #weedcopper #showstopper #queens", "followers_count": 479, "friends_count": 366, "statues_count": 14701 }, "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-06T10:00:22.000Z"), "id": 684796666923225088, "text": "6. Oculus Rift\n7. #DrummondPuddleWatch\n8. Rob Ford\n9. #Muzzo\n10. P.K. Subban\n\n2016/1/6 11:51 CST #trndnl https://t.co/SHjd3tkINE", "in_reply_to_status": 684796665195184128, "in_reply_to_user": 1270239397, "favorite_count": 0, "coordinate": point("-98.309,56.9547"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DrummondPuddleWatch", "Muzzo", "trndnl" }}, "user": { "id": 1270239397, "name": "Trendinalia Canada", "screen_name": "trendinaliaCA", "lang": "es", "location": "Canada", "create_at": date("2013-03-15"), "description": "Information about Twitter's Trending Topics in Canada", "followers_count": 1250, "friends_count": 7, "statues_count": 257723 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Division No. 23, Unorganized, Manitoba", "id": "654fd5617c7361d0", "name": "Division No. 23, Unorganized", "place_type": "city", "bounding_box": rectangle("-102.009382,55.158501 -88.989166,60.000283") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-06T10:00:22.000Z"), "id": 684796666961002497, "text": "I wanted to go out to eat but it's starting to sprinkle ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2393419261, "name": "kraykay", "screen_name": "kaylaaaa167", "lang": "en", "location": "null", "create_at": date("2014-03-16"), "description": "tths|", "followers_count": 152, "friends_count": 139, "statues_count": 1210 }, "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-06T10:00:22.000Z"), "id": 684796667040669696, "text": "Production Associate - Panera Bread: (#Skokie, IL) https://t.co/SuYGOlUhdY #Hospitality #parttime #Veterans #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.7466886,42.0553226"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Skokie", "Hospitality", "parttime", "Veterans", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 88114220, "name": "TMJ-IL HRTA Jobs", "screen_name": "tmj_il_hrta", "lang": "en", "location": "Illinois", "create_at": date("2009-11-06"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Illinois Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 421, "friends_count": 291, "statues_count": 1245 }, "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-01-06T10:00:22.000Z"), "id": 684796667082625024, "text": "Apparently the bartender told this white boy that keeps talking about me, \"you know Alex only likes black boys right?\" ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 507802805, "name": "aye", "screen_name": "alexoffdemeter", "lang": "en", "location": "portland ", "create_at": date("2012-02-28"), "description": "if it's not a cheeseburger, some trees, or anything oakland... I probably don't want it #askmeaboutbeejus #walnutnotpistachio", "followers_count": 492, "friends_count": 348, "statues_count": 78039 }, "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-06T10:00:23.000Z"), "id": 684796667162312704, "text": "Godiva ships chocolate through space to be baked inside of the sun to later be placed on black women.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 40220433, "name": "Sauce Castillo", "screen_name": "_Taveus", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-05-15"), "description": "Ham-Burger?", "followers_count": 981, "friends_count": 89, "statues_count": 169482 }, "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-06T10:00:23.000Z"), "id": 684796667300720641, "text": "Deadass spring term sober mani", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 180457743, "name": "Sailor Moon", "screen_name": "_emanikrystal", "lang": "en", "location": "Jamaican&Grenadian ", "create_at": date("2010-08-19"), "description": "#fiu19 | island pickney | SC: emanikrystal | diplo&borgore are my daddies", "followers_count": 485, "friends_count": 350, "statues_count": 25300 }, "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-06T10:00:23.000Z"), "id": 684796667523108866, "text": "Why is it hard to get people to be references for me? I'm I really the worst and no one is telling me?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14493902, "name": "Wendy Packard", "screen_name": "WendEclair", "lang": "en", "location": "Chicago, IL, USA", "create_at": date("2008-04-23"), "description": "Chicago. Engineer, Lawyer, trying to escape to California.", "followers_count": 384, "friends_count": 392, "statues_count": 45710 }, "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-06T10:00:23.000Z"), "id": 684796667762102272, "text": "y'all hatred for black women make it hard for me to solely tweet bout art and music. shit annoying dawg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 102540879, "name": "dreadhead papi", "screen_name": "melvinsucks", "lang": "en", "location": "Howard University", "create_at": date("2010-01-06"), "description": "vibes n stuff", "followers_count": 1508, "friends_count": 981, "statues_count": 95182 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inniswold, LA", "id": "fbcefe96d630085b", "name": "Inniswold", "place_type": "city", "bounding_box": rectangle("-91.092831,30.363623 -91.040089,30.412138") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2237270, "cityName": "Inniswold" } }
+{ "create_at": datetime("2016-01-06T10:00:23.000Z"), "id": 684796667808333828, "text": "Apartment shopping with Ashley is hilarious ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2785932563, "name": "Queen B", "screen_name": "iBella_xO", "lang": "en", "location": "null", "create_at": date("2014-09-26"), "description": "C'est La Vie ..", "followers_count": 107, "friends_count": 307, "statues_count": 2427 }, "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-06T10:00:23.000Z"), "id": 684796667820773376, "text": "Let me know what they say \n https://t.co/lTLshYnLCM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2971162658, "name": "L'élu", "screen_name": "StrictlyBucketz", "lang": "en", "location": "null", "create_at": date("2015-01-09"), "description": "sc: hoopaz_life\nBaccground of a street dude with the mind of a scholar", "followers_count": 327, "friends_count": 358, "statues_count": 785 }, "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-06T10:00:23.000Z"), "id": 684796667971801088, "text": "We're #hiring! Click to apply: Inpatient Services RN (Part time) Cape May Court House - https://t.co/ZjwNnAo6ED #CapeMayCourtHouse, NJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.8237786,39.0826135"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "CapeMayCourtHouse" }}, "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": 452, "friends_count": 292, "statues_count": 594 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cape May Court House, NJ", "id": "499aa36616bd1cfb", "name": "Cape May Court House", "place_type": "city", "bounding_box": rectangle("-74.859917,39.050335 -74.791582,39.125159") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34009, "countyName": "Cape May", "cityID": 3410300, "cityName": "Cape May Court House" } }
+{ "create_at": datetime("2016-01-06T10:00:23.000Z"), "id": 684796668009562112, "text": "Sunny this afternoon, high 58 (14 C). Low 44 (7 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119698549, "name": "Mobile Weather", "screen_name": "MobileWX", "lang": "en", "location": "Mobile, Al", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Mobile, Al", "followers_count": 403, "friends_count": 68, "statues_count": 6149 }, "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-06T10:00:23.000Z"), "id": 684796668244586496, "text": "Thank you to everyone that reached out to me yesterday & thank you to everyone that celebrated with me. It was a birthday I'll never forget", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1284476484, "name": "Justin Byrd", "screen_name": "ItsJByrd", "lang": "en", "location": "null", "create_at": date("2013-03-20"), "description": "University of Kentucky", "followers_count": 766, "friends_count": 394, "statues_count": 2444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Landen, OH", "id": "ffca15636bd3943a", "name": "Landen", "place_type": "city", "bounding_box": rectangle("-84.316754,39.288723 -84.252609,39.338068") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39165, "countyName": "Warren", "cityID": 3941755, "cityName": "Landen" } }
+{ "create_at": datetime("2016-01-06T10:00:23.000Z"), "id": 684796668483469312, "text": "@5oz3balat \nكل سنه و انتي طيبه https://t.co/dm0QdWeLYI", "in_reply_to_status": -1, "in_reply_to_user": 2291703346, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user_mentions": {{ 2291703346 }}, "user": { "id": 2419344219, "name": "Masry", "screen_name": "BraveHeart306", "lang": "en", "location": "California, USA", "create_at": date("2014-03-17"), "description": "الأرواح تتصافح .. قبل اليد و قبل العين .", "followers_count": 2292, "friends_count": 128, "statues_count": 16475 }, "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-06T10:00:23.000Z"), "id": 684796668525604865, "text": "@BruceWStanley may be that time in Midtown...", "in_reply_to_status": 684794747211362305, "in_reply_to_user": 47023313, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 47023313 }}, "user": { "id": 19422491, "name": "Grant Stern", "screen_name": "grantstern", "lang": "en", "location": "Miami-Wade County", "create_at": date("2009-01-23"), "description": "#Miami native @MiamiHeat STH, Mortgage Broker @morningsidemtg #RealEstate consultant @condoterminator writes @HuffingtonPost & #OnlyinMiami show host @880thebiz", "followers_count": 6438, "friends_count": 6265, "statues_count": 47201 }, "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-06T10:00:23.000Z"), "id": 684796668622061568, "text": "whenever they say somethin bout us you listen", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2626004546, "name": "Karley", "screen_name": "ksn_xoxo", "lang": "en", "location": "Red Bank, SC", "create_at": date("2014-07-12"), "description": "•• Sweeter than heaven, hotter than Hell•• || SC- karleysummerxox || IG- karleysummer_ ||", "followers_count": 614, "friends_count": 281, "statues_count": 4524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Red Bank, SC", "id": "96ef9310fd1d1cb8", "name": "Red Bank", "place_type": "city", "bounding_box": rectangle("-81.30108,33.874722 -81.152072,33.966779") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45063, "countyName": "Lexington", "cityID": 4559110, "cityName": "Red Bank" } }
+{ "create_at": datetime("2016-01-06T10:00:23.000Z"), "id": 684796668647182336, "text": "Who coming to Atlanta with me? Book your rooms. \n#edm #hiphop #dancehall #chicagoproducer #music… https://t.co/tD7mjzVGLs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6844,41.8369"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "edm", "hiphop", "dancehall", "chicagoproducer", "music" }}, "user": { "id": 192662219, "name": "ig: djfrankiefrench", "screen_name": "Djfrankiefrench", "lang": "en", "location": "Chicago, IL", "create_at": date("2010-09-19"), "description": "Dj for @StuntTaylor / @Zonaman72 Producer djfrankiefrench@gmail.com INSTAGRAM: https://t.co/TF8oU6s8Dj FACEBOOK. https://t.co/MXgyV3Kq3S", "followers_count": 10152, "friends_count": 6223, "statues_count": 74962 }, "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-06T10:00:23.000Z"), "id": 684796668848517120, "text": "did I really just hear this kid shorten \"Pythagorean theorem\" to \"Pythag\" like whattttttttttt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 776808798, "name": "kimKT", "screen_name": "katiecoombs04", "lang": "en", "location": "jersey", "create_at": date("2012-08-23"), "description": "expect nothing & hope for it all.", "followers_count": 358, "friends_count": 425, "statues_count": 2277 }, "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-06T10:00:23.000Z"), "id": 684796668932427780, "text": "50 shades of gray kind of Vday..�� @britney_davis1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2722896697 }}, "user": { "id": 926069474, "name": "❗️Alexandria❗️", "screen_name": "Guera_mexicana", "lang": "en", "location": "ln Applebee's, 62040", "create_at": date("2012-11-04"), "description": "No hago amor, yo hago el sexo ... y duro. -Siempre con @leslii315 & @KittyMeowmz", "followers_count": 366, "friends_count": 285, "statues_count": 4507 }, "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-01-06T10:00:23.000Z"), "id": 684796668953231360, "text": "Regardless of you acknowledgement of the recipe it'll never taste or look the same. The portions gotta be correct���� https://t.co/YOow4auXpm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 327425845, "name": "ReallyDontComeForMe", "screen_name": "Hotboytrvy", "lang": "en", "location": "New Orleans, LA", "create_at": date("2011-07-01"), "description": "Follow my instagram and snapchat: @hotboytrvy #UL'19.", "followers_count": 680, "friends_count": 1020, "statues_count": 6533 }, "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-06T10:00:23.000Z"), "id": 684796668957585408, "text": "Conozca Sus Derechos CHIRLA.wmv https://t.co/TysITxjan8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2432755034, "name": "Consulado Honduras", "screen_name": "Consuladohn", "lang": "en", "location": "Miami FL USA", "create_at": date("2014-04-07"), "description": "Consulado General de Honduras 8600 Nw 36th ST Miami FL 33166", "followers_count": 700, "friends_count": 224, "statues_count": 398 }, "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-01-06T10:00:23.000Z"), "id": 684796669267820544, "text": "I understand Seinfeld and his dating tactics now. Like... I can't date you because you have read notifications on iMessage #ontothenext", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ontothenext" }}, "user": { "id": 260608086, "name": "Gissel Acevedo", "screen_name": "gisselacev", "lang": "en", "location": "San Diego, CA", "create_at": date("2011-03-03"), "description": "A designer knows he has achieved perfection not when there is nothing left to add, but when there is nothing left to take away. (Terre des Hommes, 1939)", "followers_count": 174, "friends_count": 153, "statues_count": 1573 }, "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-06T10:00:23.000Z"), "id": 684796669435719680, "text": "So according to my doctor (from my checkup yesterday), I need to lose weight, exercise and eat… https://t.co/w3YG34sO1k", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.62564,41.88132"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29231959, "name": "Derrick Alba", "screen_name": "iamderrickstwtr", "lang": "en", "location": "Chicago", "create_at": date("2009-04-06"), "description": "i'm just a dude playin' a dude disguised as another dude...", "followers_count": 487, "friends_count": 712, "statues_count": 11417 }, "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-06T10:00:23.000Z"), "id": 684796669876158465, "text": "I ain't gonna loose what I got 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": 1888573070, "name": "W∆ST£D", "screen_name": "Genny_135", "lang": "en", "location": "North Plainfield, NJ", "create_at": date("2013-09-20"), "description": "aries ♈ #TGOD insta:genny_135", "followers_count": 274, "friends_count": 494, "statues_count": 8304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Plainfield, NJ", "id": "6ecd80a684f06657", "name": "North Plainfield", "place_type": "city", "bounding_box": rectangle("-74.46488,40.601998 -74.413773,40.641762") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34035, "countyName": "Somerset", "cityID": 3453280, "cityName": "North Plainfield" } }
+{ "create_at": datetime("2016-01-06T10:00:23.000Z"), "id": 684796669989404673, "text": "Got a point https://t.co/xDOw5YVMsP", "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": "Shaine", "screen_name": "SHAINEAVELI", "lang": "en", "location": "New Orleans, LA", "create_at": date("2011-06-10"), "description": "Founder of UNBOUND", "followers_count": 1093, "friends_count": 395, "statues_count": 87108 }, "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-06T10:00:23.000Z"), "id": 684796670056493056, "text": "Stop opening my messages , stay off my Twitter I don't fuck w you ✌��️ .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2878969744, "name": "NAAI❤️", "screen_name": "boythatsNAAI", "lang": "en", "location": "Trap $", "create_at": date("2014-11-15"), "description": "Sc: daddy_naai , #E4tank❤️", "followers_count": 420, "friends_count": 229, "statues_count": 26906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Belle Haven, VA", "id": "734825d1bfe33a28", "name": "Belle Haven", "place_type": "city", "bounding_box": rectangle("-77.081398,38.763609 -77.044219,38.792561") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5105928, "cityName": "Belle Haven" } }
+{ "create_at": datetime("2016-01-06T10:00:23.000Z"), "id": 684796670400409600, "text": "A message of long awaits change that the country needs to move toward a more happier nation.… https://t.co/1mEDlbaVKG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.002533,40.739779"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 395396732, "name": "Yahaira Hernandez", "screen_name": "itzyahaira", "lang": "en", "location": "NYC", "create_at": date("2011-10-21"), "description": "Filmmaker, Thespian & Professional Scribbler", "followers_count": 121, "friends_count": 309, "statues_count": 4712 }, "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-06T10:00:23.000Z"), "id": 684796670572281857, "text": "Want to work in #Waynesboro, TN? View our latest opening: https://t.co/tbDIaUW2o9 #Hospitality #SONIC #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.7554996,35.3256903"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Waynesboro", "Hospitality", "SONIC", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 88304630, "name": "TMJ-TN HRTA Jobs", "screen_name": "tmj_tn_hrta", "lang": "en", "location": "Tennessee", "create_at": date("2009-11-07"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Tennessee Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 351, "friends_count": 281, "statues_count": 1610 }, "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": 47181, "countyName": "Wayne", "cityID": 4778600, "cityName": "Waynesboro" } }
+{ "create_at": datetime("2016-01-06T10:00:23.000Z"), "id": 684796670601609216, "text": "Cloudy this afternoon, high 38 (3 C). Low 32 (0 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119751854, "name": "Chicago Weather", "screen_name": "Chicago_WX", "lang": "en", "location": "Chicago, IL", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Chicago, IL", "followers_count": 1062, "friends_count": 68, "statues_count": 6431 }, "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-06T10:00:23.000Z"), "id": 684796670740070400, "text": "Ghirardelli #BusinessMgmt #Job: Temporary Office Manager (#SanFrancisco, CA) https://t.co/azU88G29uS #management #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.4194155,37.7749295"), "retweet_count": 0, "lang": "it", "is_retweet": false, "hashtags": {{ "BusinessMgmt", "Job", "SanFrancisco", "management", "Jobs", "Hiring" }}, "user": { "id": 2228943546, "name": "Ghirardelli Careers", "screen_name": "ghirardellijobs", "lang": "en", "location": "null", "create_at": date("2013-12-03"), "description": "Ghirardelli invites you to explore our open #jobs and discover a place where your ideas come to life, as part of our delicious legacy.", "followers_count": 51, "friends_count": 1, "statues_count": 86 }, "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-06T10:00:23.000Z"), "id": 684796670912147458, "text": "@michaelsimeon @verified lol do they need finger prints or a blood sample lol", "in_reply_to_status": 684794087807950848, "in_reply_to_user": 39102845, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 39102845, 63796828 }}, "user": { "id": 37084847, "name": "elizabeth liberto", "screen_name": "lizzieliberto", "lang": "en", "location": "nashville tn.", "create_at": date("2009-05-01"), "description": "null", "followers_count": 84, "friends_count": 232, "statues_count": 9603 }, "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-01-06T10:00:23.000Z"), "id": 684796670949773312, "text": "#CareerArc #Labor #Job alert: Maintenance Team Member | BJs Wholesale Club | #Newnan, Georgia https://t.co/1LitJVG3Fd #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.7996573,33.3806716"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CareerArc", "Labor", "Job", "Newnan", "Jobs", "Hiring" }}, "user": { "id": 117968952, "name": "TMJ-GA Labor Jobs", "screen_name": "tmj_GA_LABOR", "lang": "en", "location": "Georgia", "create_at": date("2010-02-26"), "description": "Follow this account for geo-targeted General Labor job tweets in Georgia Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 335, "friends_count": 285, "statues_count": 87 }, "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-06T10:00:23.000Z"), "id": 684796670962339840, "text": "That's a case fed time. @ WTHR TV Channel 13 https://t.co/xHg3DAcpbd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.15758,39.78137"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 447308135, "name": "Leonard Lajuan Petty", "screen_name": "barberlife103", "lang": "en", "location": "Cleveland, OH", "create_at": date("2011-12-26"), "description": "null", "followers_count": 3568, "friends_count": 112, "statues_count": 32674 }, "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-01-06T10:00:23.000Z"), "id": 684796671008493568, "text": "Tilted Kilt: Kilt Girl Server, Bartender and Hostess (#SCOTTSDALE, AZ) https://t.co/sSBJQoL46M #Hospitality #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.9288904,33.6380585"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SCOTTSDALE", "Hospitality", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 99622969, "name": "TMJ-AZS HRTA Jobs", "screen_name": "tmj_AZS_HRTA", "lang": "en", "location": "Scottsdale, AZ", "create_at": date("2009-12-26"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Scottsdale, AZ. Need help? Tweet us at @CareerArc!", "followers_count": 318, "friends_count": 275, "statues_count": 650 }, "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-06T10:00:23.000Z"), "id": 684796671063126016, "text": "There's no reason for classroom windows to be open in January.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 636700426, "name": ".", "screen_name": "j_newlove", "lang": "en", "location": "NY", "create_at": date("2012-07-15"), "description": "Christ Conscious", "followers_count": 864, "friends_count": 751, "statues_count": 8155 }, "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-01-06T10:00:23.000Z"), "id": 684796671201554433, "text": "burger with fried egg #nom (@ The @CarolinaInn in Chapel Hill, NC) https://t.co/iRFgFu1xDj https://t.co/zKF0Ze8NNf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.054385,35.9097518"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "nom" }}, "user_mentions": {{ 36066235 }}, "user": { "id": 36964022, "name": "moon", "screen_name": "aforeffort", "lang": "en", "location": "Washington, DC", "create_at": date("2009-05-01"), "description": "'cause if your skin was soil, how long do you think before they start diggin'\nand if your life was gold, how long would you think you'd stay livin'", "followers_count": 129, "friends_count": 265, "statues_count": 3235 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chapel Hill, NC", "id": "23ebe811ea93fd41", "name": "Chapel Hill", "place_type": "city", "bounding_box": rectangle("-79.083153,35.869376 -78.968887,35.969507") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37135, "countyName": "Orange", "cityID": 3711800, "cityName": "Chapel Hill" } }
+{ "create_at": datetime("2016-01-06T10:00:23.000Z"), "id": 684796671209832448, "text": "Can you recommend anyone for this #job? Retail Store Shift Supervisor - https://t.co/OGvQGjwNnu #Jacksonville, FL #Retail #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.655651,30.3321838"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Jacksonville", "Retail", "Hiring" }}, "user": { "id": 28440291, "name": "TMJ-JAX Retail Jobs", "screen_name": "tmj_jax_retail", "lang": "en", "location": "Jacksonville, FL", "create_at": date("2009-04-02"), "description": "Follow this account for geo-targeted Retail job tweets in Jacksonville, FL. Need help? Tweet us at @CareerArc!", "followers_count": 399, "friends_count": 312, "statues_count": 736 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downtown Jacksonville, Jacksonville", "id": "3496512730330cc3", "name": "Downtown Jacksonville", "place_type": "neighborhood", "bounding_box": rectangle("-81.667342,30.320695 -81.6461,30.33406") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12031, "countyName": "Duval", "cityID": 1235000, "cityName": "Jacksonville" } }
+{ "create_at": datetime("2016-01-06T10:00:24.000Z"), "id": 684796671344152576, "text": "@jarednieters offering the best #cxnats coverage out there! Keep out there, keep up the great work!", "in_reply_to_status": -1, "in_reply_to_user": 21544197, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "cxnats" }}, "user_mentions": {{ 21544197 }}, "user": { "id": 34004343, "name": "Aero Tech Designs", "screen_name": "aerotechdesigns", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2009-04-21"), "description": "We are a bicycling apparel manufacture specializing in a wide variety of products and sizes! We sell direct to you, our customer! Made in the USA", "followers_count": 590, "friends_count": 803, "statues_count": 1887 }, "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-06T10:00:24.000Z"), "id": 684796671700541441, "text": "Chicken Marsala over Egg Noodles served with green beans & side salad @ Cain's Saloon https://t.co/i1VxFDO06d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.0409012,40.3897209"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 106827323, "name": "Cain's Saloon (PGH)", "screen_name": "Cainssaloon", "lang": "en", "location": "Pittsburgh, PA (Dormont)", "create_at": date("2010-01-20"), "description": "Pittsburgh's Premier Sports Bar & Eatery in the South Hills", "followers_count": 408, "friends_count": 192, "statues_count": 1064 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Lebanon, PA", "id": "7cd18caa8c0ebb9f", "name": "Mount Lebanon", "place_type": "city", "bounding_box": rectangle("-80.082398,40.350594 -80.018127,40.401888") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4219576, "cityName": "Dormont" } }
+{ "create_at": datetime("2016-01-06T10:00:24.000Z"), "id": 684796671725666304, "text": "This #Finance #job might be a great fit for you: Financial Analyst - https://t.co/UZgEOmQjwN #Sunrise, FL #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.3101684,26.1512497"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Finance", "job", "Sunrise", "Hiring", "CareerArc" }}, "user": { "id": 234650979, "name": "Florida Finance", "screen_name": "tmj_FL_finance", "lang": "en", "location": "Florida", "create_at": date("2011-01-05"), "description": "Follow this account for geo-targeted Finance job tweets in Florida Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 307, "friends_count": 233, "statues_count": 150 }, "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-06T10:00:24.000Z"), "id": 684796671725719552, "text": "I love her!!!�� https://t.co/EsGeCZjgC9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2922374279, "name": "R.I.P Grandmama", "screen_name": "briidgetteeee_", "lang": "en", "location": "Commerce, TX", "create_at": date("2014-12-14"), "description": "# TAMUC19 || Justin Bieber❣|| sc: x.nicolle", "followers_count": 415, "friends_count": 394, "statues_count": 7913 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hutchins, TX", "id": "2b75ff38a3cc185e", "name": "Hutchins", "place_type": "city", "bounding_box": rectangle("-96.729623,32.619374 -96.687632,32.670256") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4835612, "cityName": "Hutchins" } }
+{ "create_at": datetime("2016-01-06T10:00:24.000Z"), "id": 684796671801208832, "text": "Yall women gon be 50 still wearin tongue piercings.... Smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 476420777, "name": "Tre", "screen_name": "Tre_Potts", "lang": "en", "location": "Maryland✈️Schertz✈️The Army", "create_at": date("2012-01-27"), "description": "22, #Tamucc16, God | US Army future officer | LSU, UT & Saints | Spurs | tryna make it everyday", "followers_count": 1553, "friends_count": 1895, "statues_count": 99732 }, "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-01-06T10:00:24.000Z"), "id": 684796672040288256, "text": "@br3ezy24 Good!!:)", "in_reply_to_status": 684794355542970368, "in_reply_to_user": 585132025, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 585132025 }}, "user": { "id": 3409873452, "name": "Taylor Huss", "screen_name": "HaylorTuss", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-08-31"), "description": "R.I.P Roy Black 6/17/75-10/10/15 Always Loved Never Forgotten! Brittany Anteau ♡♡", "followers_count": 222, "friends_count": 461, "statues_count": 2827 }, "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-06T10:00:24.000Z"), "id": 684796672061222913, "text": "@MidweztMarauder @allEDGEdly @DannyFresh421 Lincoln wins again https://t.co/4hnhTmxWti", "in_reply_to_status": -1, "in_reply_to_user": 538718053, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 538718053, 22307882, 2508161862 }}, "user": { "id": 599725143, "name": "JDP", "screen_name": "jpsteelers43", "lang": "en", "location": "Lincoln NE", "create_at": date("2012-06-04"), "description": "Liquor/Beer/Wine Professional SteelerHead Snow Skiing StoopidHead", "followers_count": 108, "friends_count": 197, "statues_count": 5968 }, "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-06T10:00:24.000Z"), "id": 684796672472252416, "text": "New Agents and Expert Agents arr Getting back to basics #BairdWarner @ Baird & Warner - Naperville https://t.co/TK0euDueJp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.16217517,41.74729601"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BairdWarner" }}, "user": { "id": 31859742, "name": "Carrie Bey-Little", "screen_name": "CarrieBeyLittle", "lang": "en", "location": "United States, llinois", "create_at": date("2009-04-16"), "description": "MLS trainer, Licensed with Baird & Warner. Not your average agent, Social Media & Digital Marketing Strategist for the Real Estate Industry. #Bigdata", "followers_count": 1564, "friends_count": 1025, "statues_count": 13689 }, "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-01-06T10:00:24.000Z"), "id": 684796673487319040, "text": "@dccdestiny it will be ����❤️��", "in_reply_to_status": 684796530314772480, "in_reply_to_user": 880760360, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 880760360 }}, "user": { "id": 3002365756, "name": "p a y g e", "screen_name": "_paygelewis", "lang": "en", "location": "Victoria, TX", "create_at": date("2015-01-27"), "description": "18 | If I ever let my head down, it will be just to admire my shoes ✨ |", "followers_count": 419, "friends_count": 584, "statues_count": 4820 }, "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-06T10:00:24.000Z"), "id": 684796673525153793, "text": "@paradisefears also on that day. Love Miracle Day https://t.co/NQifDK4E7W", "in_reply_to_status": 684795577373077504, "in_reply_to_user": 18922329, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18922329 }}, "user": { "id": 241788572, "name": "Brianna", "screen_name": "_russobrianna", "lang": "en", "location": "Philly | La Salle University", "create_at": date("2011-01-22"), "description": "usually wearing cool socks", "followers_count": 1107, "friends_count": 279, "statues_count": 45904 }, "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-06T10:00:24.000Z"), "id": 684796673608908801, "text": "Interested in a #FacilitiesMgmt #job near #WestHaven, CT? This could be a great fit: https://t.co/q6Freh4n5b #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.9470471,41.2706527"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FacilitiesMgmt", "job", "WestHaven", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 173541362, "name": "CT Facility Mgmt.", "screen_name": "tmj_CT_facmgmt", "lang": "en", "location": "Connecticut", "create_at": date("2010-08-01"), "description": "Follow this account for geo-targeted Facilities Management job tweets in Connecticut Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 135, "friends_count": 125, "statues_count": 16 }, "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-01-06T10:00:24.000Z"), "id": 684796673621495808, "text": "Un JT High Tech avec un public qui interagit ? Yes ! >>>>\nhttps://t.co/e8vZNcKIyV\n@NRJ_GAMES @MaikiGeeky https://t.co/sPsZYD32T6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user_mentions": {{ 3187401453, 1174494830 }}, "user": { "id": 753793896, "name": "PP Garcia", "screen_name": "ppgarcia75", "lang": "fr", "location": "Paris ", "create_at": date("2012-08-12"), "description": "#Journaliste Rendre High Tech super cool @01netTV #POWER @NRJ_Games, ex @canalplus, @RMCinfo DQJMM, YouTube PP World, fan Star Wars. Déconneur, gamer, taré!", "followers_count": 12620, "friends_count": 1546, "statues_count": 51039 }, "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-06T10:00:24.000Z"), "id": 684796673697054720, "text": "You, Ammon Bundy, are no Rosa Parks. #OregonUnderAttack", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "OregonUnderAttack" }}, "user": { "id": 716668712, "name": "Steven Nelson", "screen_name": "snelsonus", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-07-25"), "description": "UCLA Professor of African and African American Art History, Director of the UCLA African Studies Center. I see lots of shiny objects in the world.", "followers_count": 1528, "friends_count": 1108, "statues_count": 8317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:24.000Z"), "id": 684796673898356736, "text": "Partly cloudy this afternoon, high 56 (13 C). Low 41 (5 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119753180, "name": "Jackson Weather", "screen_name": "JacksonWX", "lang": "en", "location": "Jackson, MS", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Jackson, MS", "followers_count": 295, "friends_count": 68, "statues_count": 6380 }, "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-06T10:00:24.000Z"), "id": 684796673973972992, "text": "Everytime I look at my box I cry like where the characters man https://t.co/Ac1nsSq9s3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 333539696, "name": "✨Hentai Bwoy✨", "screen_name": "RetroSenpai", "lang": "en", "location": "『RetroWorld』", "create_at": date("2011-07-11"), "description": "Gamer | Music Artist | Internet idol |", "followers_count": 3495, "friends_count": 2034, "statues_count": 152013 }, "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-06T10:00:24.000Z"), "id": 684796674204536832, "text": "Can you recommend anyone for this #Sales #job? https://t.co/2ZfHVz5Dlv #WoodlandHills, CA #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.6059197,34.1683386"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "job", "WoodlandHills", "Hiring" }}, "user": { "id": 4441474115, "name": "Join Lab Support", "screen_name": "JoinLabSupport", "lang": "en", "location": "null", "create_at": date("2015-12-10"), "description": "An #international leader in placing #science, #engineering, and #preclinical #professionals in #contract, contract-to-#hire, and direct hire opportunities.", "followers_count": 20, "friends_count": 108, "statues_count": 11 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:24.000Z"), "id": 684796674225537024, "text": "*Ring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2292947338, "name": "Jah-Ney ✨", "screen_name": "_NeyBellaa", "lang": "en", "location": "Cleveland, OH", "create_at": date("2014-01-19"), "description": "I Got Every Reason to feel like I'm that b**** || WHHS Varsity Cheerleader \nDiamond's World", "followers_count": 894, "friends_count": 897, "statues_count": 6598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warrensville Heights, OH", "id": "166b716425d5bb87", "name": "Warrensville Heights", "place_type": "city", "bounding_box": rectangle("-81.555545,41.423881 -81.488156,41.457349") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3980990, "cityName": "Warrensville Heights" } }
+{ "create_at": datetime("2016-01-06T10:00:24.000Z"), "id": 684796674749927424, "text": "@LaurenDavis123 ��", "in_reply_to_status": 684796246196785152, "in_reply_to_user": 128052906, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 128052906 }}, "user": { "id": 2327265610, "name": "Lexy Breen", "screen_name": "lexybreen", "lang": "en", "location": "null", "create_at": date("2014-02-05"), "description": "null", "followers_count": 569, "friends_count": 336, "statues_count": 1066 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newport, KY", "id": "25e7ede60b304d89", "name": "Newport", "place_type": "city", "bounding_box": rectangle("-84.504476,39.064988 -84.4611,39.102427") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21037, "countyName": "Campbell", "cityID": 2155884, "cityName": "Newport" } }
+{ "create_at": datetime("2016-01-06T10:00:24.000Z"), "id": 684796674850467842, "text": "when you see @epDannyEdge https://t.co/AgWU0BnTtJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25363826 }}, "user": { "id": 1065510265, "name": "hales .-.", "screen_name": "winant_haley", "lang": "en", "location": "Highland Village, TX", "create_at": date("2013-01-06"), "description": "mhs tennis|acting|valley creek", "followers_count": 74, "friends_count": 102, "statues_count": 1606 }, "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-01-06T10:00:24.000Z"), "id": 684796674850476032, "text": "Copywriting. Sometimes we write creative shit - sometimes we write, \"family fun is in the bag!\" https://t.co/ff0tcFM9bP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26904088, "name": "JD", "screen_name": "JacquelynDS", "lang": "en", "location": "United States", "create_at": date("2009-03-26"), "description": "This might offend my political connects.", "followers_count": 233, "friends_count": 226, "statues_count": 4102 }, "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-06T10:00:24.000Z"), "id": 684796674900946951, "text": "Happy to be back in the burgh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 393625359, "name": "Gabi Duarte", "screen_name": "gabbb_duarte", "lang": "en", "location": "null", "create_at": date("2011-10-18"), "description": "University of Pittsburgh '19", "followers_count": 398, "friends_count": 380, "statues_count": 3653 }, "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-06T10:00:24.000Z"), "id": 684796674963685377, "text": "@Stafturd hmmmm so wanna Netflix and chill tonight or nah?", "in_reply_to_status": 684796135114813440, "in_reply_to_user": 303586621, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 303586621 }}, "user": { "id": 85220249, "name": "Don Juan Dela Nooch", "screen_name": "stillill143", "lang": "en", "location": "ÜT: 45.52056,-122.66329", "create_at": date("2009-10-25"), "description": "Horny for @leespielman", "followers_count": 2617, "friends_count": 1969, "statues_count": 83625 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:24.000Z"), "id": 684796675043401728, "text": "Catching up. My goodness, if I never hear \"ROR trade\" it will be TO SOON GD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30802291, "name": "kristin herrington", "screen_name": "kris_herr", "lang": "en", "location": "san diego, ca", "create_at": date("2009-04-12"), "description": "figuring it out.", "followers_count": 248, "friends_count": 345, "statues_count": 9388 }, "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-06T10:00:24.000Z"), "id": 684796675152572416, "text": "@AvrahKaDabra_ when someone tells u bananas don't make good weapons https://t.co/IEyS4iXstt", "in_reply_to_status": -1, "in_reply_to_user": 28667135, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28667135 }}, "user": { "id": 12920542, "name": "#bizkev", "screen_name": "Kevin_T_Knight", "lang": "en", "location": "Chicago", "create_at": date("2008-01-31"), "description": "Programmer for the mighty Iron Galaxy Studios. I do crappy comics too. Sucker for cats and bunnies. I love video games. They're also for everyone.", "followers_count": 249, "friends_count": 208, "statues_count": 16447 }, "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-06T10:00:24.000Z"), "id": 684796675420991488, "text": "The \"Tupelo Honey Cafe\" in Knoxville is a fantastic cafe...lists of… https://t.co/RoG4wbeAzF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.919417,35.964658"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19825996, "name": "Darryl McDonald", "screen_name": "darryl_mcdonald", "lang": "en", "location": "Atlanta, Georgia", "create_at": date("2009-01-31"), "description": "Teradata's Chief Strategy Officer. Helping shape and drive the company’s strategic analytic and marketing applications business. Avid UGA fan, gadget geek & ΠΚΑ", "followers_count": 4685, "friends_count": 5145, "statues_count": 4971 }, "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-06T10:00:24.000Z"), "id": 684796675446013952, "text": "@latimes @joexg30 Heartburn is the least of there problem. Future Bypass patients", "in_reply_to_status": 684794127825924097, "in_reply_to_user": 16664681, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16664681, 1187505739 }}, "user": { "id": 1187505739, "name": "Joe Guillen", "screen_name": "joexg30", "lang": "en", "location": "null", "create_at": date("2013-02-16"), "description": "joexg30@gmail.com", "followers_count": 64, "friends_count": 71, "statues_count": 1093 }, "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-06T10:00:25.000Z"), "id": 684796675567779840, "text": "It’s super nice out today! I picked the right day to walk around the Pavilion", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 272705887, "name": "Poe_Cameron", "screen_name": "CMcGuire21", "lang": "en", "location": "Probably a Dunkin' Donuts or Chick-fil-A drive thru", "create_at": date("2011-03-26"), "description": "Doing my best to reflect the grace and love that Christ gave to me through His sacrifice on Calvary! (John 9:2-3)", "followers_count": 226, "friends_count": 1353, "statues_count": 15189 }, "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-01-06T10:00:25.000Z"), "id": 684796675664293889, "text": "Tattoos, piercings and she just learned to twerk������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 158894013, "name": "Bo Diddley", "screen_name": "damnnnn_homie", "lang": "en", "location": "42 Wallaby Way", "create_at": date("2010-06-23"), "description": "#UABidk. Materials Science and Engineering major, Spanish minor. NSBE. #ILookLikeAnEngineer.", "followers_count": 1108, "friends_count": 853, "statues_count": 185763 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shelby, MI", "id": "5c8c81a4fef94364", "name": "Shelby", "place_type": "city", "bounding_box": rectangle("-83.095428,42.623997 -82.973298,42.715777") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26127, "countyName": "Oceana", "cityID": 2672840, "cityName": "Shelby" } }
+{ "create_at": datetime("2016-01-06T10:00:25.000Z"), "id": 684796675974496260, "text": "Why can't I sleep more on my day off?:(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 95316526, "name": "에리카", "screen_name": "_erikawaii", "lang": "en", "location": "null", "create_at": date("2009-12-07"), "description": "Lover of anime, K-Pop, & Doctor Who | I have a slight obsession with pugs |", "followers_count": 482, "friends_count": 281, "statues_count": 69855 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:25.000Z"), "id": 684796676465364992, "text": "He's annoying �� https://t.co/Q1BXuat0pe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 49177757, "name": "Kayla", "screen_name": "AlwaysKayla_", "lang": "en", "location": "null", "create_at": date("2009-06-20"), "description": "The anti social butterfly. Love you A.", "followers_count": 679, "friends_count": 236, "statues_count": 31776 }, "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-06T10:00:25.000Z"), "id": 684796676477972481, "text": "If i fail geometry I put all blame on my teacher lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2369533206, "name": "AT", "screen_name": "tamminga_", "lang": "en", "location": "null", "create_at": date("2014-03-02"), "description": "$$Respect My Mind$$ #SMG #BG", "followers_count": 1724, "friends_count": 248, "statues_count": 20383 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Imperial, MO", "id": "01e9c4ca710a63f5", "name": "Imperial", "place_type": "city", "bounding_box": rectangle("-90.437062,38.334503 -90.347229,38.41148") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29099, "countyName": "Jefferson", "cityID": 2934354, "cityName": "Imperial" } }
+{ "create_at": datetime("2016-01-06T10:00:25.000Z"), "id": 684796676511416321, "text": "Afternoon rain this afternoon, high 51 (11 C). Low 47 (8 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119754993, "name": "Waco Weather", "screen_name": "WacoWeather", "lang": "en", "location": "Waco, TX", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Waco, TX", "followers_count": 818, "friends_count": 68, "statues_count": 6361 }, "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.069786,31.639828") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4876000, "cityName": "Waco" } }
+{ "create_at": datetime("2016-01-06T10:00:25.000Z"), "id": 684796676607885312, "text": "I need someone like this in my life. Romantic or not omg. https://t.co/Tmaxd5HDJW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 319919859, "name": "jenna the feminist", "screen_name": "JennaGagnon1", "lang": "en", "location": "oregon", "create_at": date("2011-06-18"), "description": "#kindisthenewcute and i like puppers", "followers_count": 368, "friends_count": 262, "statues_count": 6801 }, "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-06T10:00:25.000Z"), "id": 684796676968689664, "text": "@GerDaJuiceMan https://t.co/iYOPHnekW2", "in_reply_to_status": 684796604960866304, "in_reply_to_user": 433153935, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 433153935 }}, "user": { "id": 433153935, "name": "G-Baby", "screen_name": "GerDaJuiceMan", "lang": "en", "location": "Townsville", "create_at": date("2011-12-09"), "description": "Ger (Jair)", "followers_count": 1095, "friends_count": 905, "statues_count": 40670 }, "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-06T10:00:25.000Z"), "id": 684796676997918721, "text": "St. Louis Rams' relocation bid contends they have the best plan for L.A. and NFL https://t.co/5jtvpuwj8c https://t.co/0hKcFyDsj2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17036898, "name": "L.A. Times: L.A. Now", "screen_name": "LANow", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2008-10-28"), "description": "Southern California -- this just in. News from L.A. and beyond from the Los Angeles Times. See @latimes for more news.", "followers_count": 224579, "friends_count": 589, "statues_count": 84608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:25.000Z"), "id": 684796677065019393, "text": "Flash Gordon RB#8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 434604069, "name": "Sharky McStevenson", "screen_name": "itstrevy7", "lang": "en", "location": "null", "create_at": date("2011-12-11"), "description": "@mikahelizabethh", "followers_count": 747, "friends_count": 420, "statues_count": 39842 }, "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-06T10:00:25.000Z"), "id": 684796677081808896, "text": "Join the CVS Health team! See our latest #Retail #job opening here: https://t.co/BR417lLIaa #ColonialHeights, VA #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.4102607,37.244039"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "ColonialHeights", "Hiring", "CareerArc" }}, "user": { "id": 28708797, "name": "TMJ-RCH Retail Jobs", "screen_name": "tmj_rch_retail", "lang": "en", "location": "Richmond, VA", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Richmond, VA. Need help? Tweet us at @CareerArc!", "followers_count": 377, "friends_count": 313, "statues_count": 297 }, "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-06T10:00:25.000Z"), "id": 684796677253804032, "text": "dang, I have to get out 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": 1565266201, "name": "therealbari ❤️", "screen_name": "JDolluz_", "lang": "en", "location": "Colby, KS", "create_at": date("2013-07-03"), "description": "nature's child, born wild. ✝ only a few care, the rest are just curious. ☮ the hood love me. ❤️", "followers_count": 569, "friends_count": 768, "statues_count": 21762 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colby, KS", "id": "69affbaa7226ae8e", "name": "Colby", "place_type": "city", "bounding_box": rectangle("-101.071619,39.359586 -101.017668,39.402741") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20193, "countyName": "Thomas", "cityID": 2014650, "cityName": "Colby" } }
+{ "create_at": datetime("2016-01-06T10:00:25.000Z"), "id": 684796677316673540, "text": "Reporting live from the gutta", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1700480280, "name": "jacob", "screen_name": "J_gambill", "lang": "en", "location": "TXST", "create_at": date("2013-08-25"), "description": "TXST KA", "followers_count": 345, "friends_count": 359, "statues_count": 10463 }, "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-01-06T10:00:25.000Z"), "id": 684796677488771073, "text": "I should be 500 pounds https://t.co/nA2WBeh6md", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 523735246, "name": "Tori D'Ancona", "screen_name": "tdancona", "lang": "en", "location": "New Jersey", "create_at": date("2012-03-13"), "description": "122714", "followers_count": 2289, "friends_count": 816, "statues_count": 69958 }, "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-06T10:00:25.000Z"), "id": 684796677526441984, "text": "Ima gone & let David start my sleeve today ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 625235883, "name": "michaela. ✨", "screen_name": "SheSo_Eylau", "lang": "en", "location": "Roszay ❤️✨", "create_at": date("2012-07-02"), "description": "Understand The Game Or Be A Victim. ✨ Rest Easy Big Ced. ❤️", "followers_count": 1725, "friends_count": 1527, "statues_count": 41430 }, "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.355819 -93.968792,33.507099") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48037, "countyName": "Bowie", "cityID": 4872368, "cityName": "Texarkana" } }
+{ "create_at": datetime("2016-01-06T10:00:25.000Z"), "id": 684796677790679040, "text": "Want to work in #SantaFeSprings, CA? View our latest opening: https://t.co/F7UTFHV3Iz #Finance #EngagePT #transportation #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.0853451,33.9472359"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SantaFeSprings", "Finance", "EngagePT", "transportation", "Job", "Jobs", "Hiring" }}, "user": { "id": 3271973436, "name": "PerformanceTeam Jobs", "screen_name": "PTGTCareers", "lang": "en", "location": "null", "create_at": date("2015-07-08"), "description": "PT is an industry leading 3PL with over 28 years of experience in warehousing, distribution, e-commerce fulfillment, transportation and supply chain logistics.", "followers_count": 24, "friends_count": 114, "statues_count": 33 }, "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-01-06T10:00:25.000Z"), "id": 684796678046617602, "text": "If you got them bandz then she might fuck . . ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4149198274, "name": "Caramel'Mocha .", "screen_name": "Dahtred", "lang": "en", "location": "null", "create_at": date("2015-11-09"), "description": "God Above Anything &; Everything • 21♐ • A Mother First ❤ • iGrind For US . . #PrettySmileWithAPrettyFace", "followers_count": 178, "friends_count": 176, "statues_count": 9821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Apopka, FL", "id": "18ff21ef9765522d", "name": "Apopka", "place_type": "city", "bounding_box": rectangle("-81.579909,28.664067 -81.479167,28.757424") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1201700, "cityName": "Apopka" } }
+{ "create_at": datetime("2016-01-06T10:00:25.000Z"), "id": 684796678130515968, "text": "Officially going to Coachella 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": 386290959, "name": "Cassidy Byrnes", "screen_name": "sassycassy0000", "lang": "en", "location": "null", "create_at": date("2011-10-06"), "description": "null", "followers_count": 708, "friends_count": 905, "statues_count": 14402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montebello, NY", "id": "f438230c852518a1", "name": "Montebello", "place_type": "city", "bounding_box": rectangle("-74.142947,41.112235 -74.088961,41.151183") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36087, "countyName": "Rockland", "cityID": 3648090, "cityName": "Montebello" } }
+{ "create_at": datetime("2016-01-06T10:00:25.000Z"), "id": 684796678185054208, "text": "If the iPhone 7 doesn't have a head phone jack that just mean their moving to Bluetooth headphones", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 450737591, "name": "malik", "screen_name": "B1GLEEK", "lang": "en", "location": "null", "create_at": date("2011-12-30"), "description": "Philippians 4:13 I can do anything i put my mind to! it's all love and nobody ever hated.", "followers_count": 1167, "friends_count": 1111, "statues_count": 42982 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scotchtown, NY", "id": "d6e3868f1e64c0f7", "name": "Scotchtown", "place_type": "city", "bounding_box": rectangle("-74.405271,41.4573 -74.344193,41.518179") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36071, "countyName": "Orange", "cityID": 3665882, "cityName": "Scotchtown" } }
+{ "create_at": datetime("2016-01-06T10:00:25.000Z"), "id": 684796678206009344, "text": "Scorpio �� https://t.co/iRhnmHjMo8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 527925563, "name": "JaJa Bank$", "screen_name": "_SkinnyJay_", "lang": "en", "location": "null", "create_at": date("2012-03-17"), "description": "Sc: kingjigsaw", "followers_count": 437, "friends_count": 302, "statues_count": 10457 }, "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-01-06T10:00:25.000Z"), "id": 684796678243659776, "text": "\" Un 6 de Enero de 1.883, nace en Becharre, Libano el Poeta, Filosofo y Pintor, Kahill Gibran.\" RAE. https://t.co/GtBnKod4ye", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 267483435, "name": "RAE", "screen_name": "caiman52", "lang": "es", "location": "San Jose, CA", "create_at": date("2011-03-16"), "description": "null", "followers_count": 10258, "friends_count": 10082, "statues_count": 77587 }, "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-06T10:00:25.000Z"), "id": 684796678373830656, "text": "@Myko35 canım benim eksik olmaya gör öpüyorum.", "in_reply_to_status": 684795101231595520, "in_reply_to_user": 214908799, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "user_mentions": {{ 214908799 }}, "user": { "id": 154339467, "name": "Ted Flu", "screen_name": "CelilAkdemir", "lang": "tr", "location": "null", "create_at": date("2010-06-10"), "description": "Hip Hop Canım Benim", "followers_count": 338, "friends_count": 77, "statues_count": 12521 }, "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-06T10:00:25.000Z"), "id": 684796678449295360, "text": "@funkykitty will do����", "in_reply_to_status": 684796517555671040, "in_reply_to_user": 99209518, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 99209518 }}, "user": { "id": 322348563, "name": "Tÿsh La Flame", "screen_name": "TyshTheComedian", "lang": "en", "location": "Funkadelic Ave. Chicago, IL", "create_at": date("2011-06-22"), "description": "20 & chasin acting too ugly to model. Writer. Positive vibes. MSU has my heart. La Flame Season. Wub (^^^) #SharkSquad #GRiZFAM #ForestFam #BassHead", "followers_count": 1048, "friends_count": 820, "statues_count": 43121 }, "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-06T10:00:25.000Z"), "id": 684796678457528320, "text": "#Novi, MI #Automotive #Job: Product Development Engineer at Aerotek https://t.co/JnhK5LroIX #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.4754913,42.48059"), "retweet_count": 0, "lang": "nl", "is_retweet": false, "hashtags": {{ "Novi", "Automotive", "Job", "Jobs", "Hiring" }}, "user": { "id": 180059944, "name": "Michigan Automotive", "screen_name": "tmj_MI_auto", "lang": "en", "location": "Michigan", "create_at": date("2010-08-18"), "description": "Follow this account for geo-targeted Automotive job tweets in Michigan Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 185, "friends_count": 122, "statues_count": 125 }, "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-01-06T10:00:25.000Z"), "id": 684796678516387841, "text": "@MEPFuller he will likely stuff some food in his pockets for later", "in_reply_to_status": 684784711084949504, "in_reply_to_user": 398088661, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 398088661 }}, "user": { "id": 14611587, "name": "Dean C ن", "screen_name": "constans", "lang": "en", "location": "New York, NY", "create_at": date("2008-05-01"), "description": "hard scrabble, yet privileged, NJ flotsam\n``Steampunk Belzer'' - Kurt Schlichter", "followers_count": 349, "friends_count": 937, "statues_count": 5342 }, "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-06T10:00:25.000Z"), "id": 684796678751178753, "text": "Very excited to announce @RubyRose's first #Miami show this Saturday for the Grand Opening of… https://t.co/sejEOOMPHA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.1318413,25.7834783"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Miami" }}, "user_mentions": {{ 23741531 }}, "user": { "id": 885617828, "name": "New World Events LLC", "screen_name": "NewWorldEvents", "lang": "en", "location": "Miami Beach FL", "create_at": date("2012-10-16"), "description": "VIP Concierge Services, Event Planning, and Event Promotions", "followers_count": 90, "friends_count": 38, "statues_count": 1468 }, "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-06T10:00:25.000Z"), "id": 684796678927302656, "text": "#yup #artseshApproved https://t.co/C8KKTWkJI2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "yup", "artseshApproved" }}, "user": { "id": 4557847993, "name": "Arturo Martinez", "screen_name": "artsesh", "lang": "en", "location": "Visalia, CA", "create_at": date("2015-12-14"), "description": "Purpose is to life what the skeleton is to the body. - Ravi Zacharias", "followers_count": 18, "friends_count": 38, "statues_count": 65 }, "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-06T10:00:25.000Z"), "id": 684796678931640322, "text": "my hair is so frustrating idec anymore", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3249519047, "name": "petty flacko", "screen_name": "pettyprincesa", "lang": "en", "location": "null", "create_at": date("2015-05-12"), "description": "woah", "followers_count": 122, "friends_count": 82, "statues_count": 1221 }, "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-06T10:00:25.000Z"), "id": 684796679027990529, "text": "Want to work at Menorah Park Center for Senior L...? We're #hiring in #Beachwood, OH! Click for details: https://t.co/itzAygcLZj #Job #STNA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4899136,41.5011199"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Beachwood", "Job", "STNA" }}, "user": { "id": 214122801, "name": "MenorahParkJobs", "screen_name": "MenorahParkJobs", "lang": "en", "location": "Beachwood, Ohio", "create_at": date("2010-11-10"), "description": "Menorah Park Center for Senior Living is one of the largest senior living communities in the state of Ohio.", "followers_count": 133, "friends_count": 82, "statues_count": 1385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beachwood, OH", "id": "fa5bd676191716ee", "name": "Beachwood", "place_type": "city", "bounding_box": rectangle("-81.532635,41.449669 -81.48037,41.501448") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3904500, "cityName": "Beachwood" } }
+{ "create_at": datetime("2016-01-06T10:00:25.000Z"), "id": 684796679099293696, "text": "Cloudy this afternoon, high 44 (7 C). Low 27 (-3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119755728, "name": "Louisville Weather", "screen_name": "Louisville_WX", "lang": "en", "location": "Louisville, KY", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Louisville, KY", "followers_count": 1311, "friends_count": 68, "statues_count": 6387 }, "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-01-06T10:00:25.000Z"), "id": 684796679367688192, "text": "@nporrasfalconio it's brown... Middle eastern...", "in_reply_to_status": 684794488393342977, "in_reply_to_user": 2877941533, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2877941533 }}, "user": { "id": 409702860, "name": "Marta Audisho", "screen_name": "MartaaAudisho", "lang": "en", "location": "null", "create_at": date("2011-11-10"), "description": "All that you ask for in prayer, believe that you will receive it and it shall be yours. Mk. 11:24", "followers_count": 561, "friends_count": 393, "statues_count": 29127 }, "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-06T10:00:25.000Z"), "id": 684796679405584387, "text": "Damn.. 9 more years? https://t.co/uIHzIge5Md", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.058093,42.356175"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119458432, "name": "NAS", "screen_name": "NAS_____", "lang": "en", "location": "Boston", "create_at": date("2010-03-03"), "description": "CEO of Vandalay Industries. \n\nDo your part and please have @Yung_Messiah reported for spam.\n\nPats, C's, Bruins, Red Sox.", "followers_count": 946, "friends_count": 767, "statues_count": 144530 }, "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-06T10:00:25.000Z"), "id": 684796679459962880, "text": "I hate when it rains ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 396989854, "name": "adamjoland", "screen_name": "JustCallMe_Adam", "lang": "en", "location": "null", "create_at": date("2011-10-23"), "description": "UCLA. ΦΔΘ.", "followers_count": 251, "friends_count": 233, "statues_count": 8278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:25.000Z"), "id": 684796679598534658, "text": "these are my alphabet bitches", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1145531305, "name": "gabi", "screen_name": "gabi_oconnor", "lang": "en", "location": "null", "create_at": date("2013-02-03"), "description": "shs cheerleader, senior, united states air force", "followers_count": 1050, "friends_count": 487, "statues_count": 16948 }, "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-06T10:00:26.000Z"), "id": 684796679707561984, "text": "Gunna watch prison break then put up my clothes for the 97th time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 618419938, "name": "вℓѧкє мσяяσω™", "screen_name": "Blakeee_M", "lang": "en", "location": "Bessemer City, NC", "create_at": date("2012-06-25"), "description": "| Instagram: Blakeeem | If you aint first,ur last|", "followers_count": 608, "friends_count": 540, "statues_count": 8761 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bessemer City, NC", "id": "bc41bc174fd0fd20", "name": "Bessemer City", "place_type": "city", "bounding_box": rectangle("-81.321137,35.265524 -81.247907,35.30534") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37071, "countyName": "Gaston", "cityID": 3705220, "cityName": "Bessemer City" } }
+{ "create_at": datetime("2016-01-06T10:00:26.000Z"), "id": 684796679946555393, "text": "@atomicbird Yo dawg, I hear you like constraints on your constraints.", "in_reply_to_status": 684795854377463809, "in_reply_to_user": 682823, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 682823 }}, "user": { "id": 15884617, "name": "Bill Lloyd", "screen_name": "wklj", "lang": "en", "location": "California", "create_at": date("2008-08-17"), "description": "This space for rent.\n\nhttp://www.strava.com/athletes/347", "followers_count": 351, "friends_count": 264, "statues_count": 8152 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio Rengstorff Area, Mountain View", "id": "59bcebcc59e5486e", "name": "San Antonio Rengstorff Area", "place_type": "neighborhood", "bounding_box": rectangle("-122.116865,37.393136 -122.091646,37.409674") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 649670, "cityName": "Mountain View" } }
+{ "create_at": datetime("2016-01-06T10:00:26.000Z"), "id": 684796680131194881, "text": "happy 18th ���� @caitcowdrey https://t.co/2VyCCljw6J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 448012810 }}, "user": { "id": 448549412, "name": "trace", "screen_name": "traceygrabowski", "lang": "en", "location": "null", "create_at": date("2011-12-27"), "description": "null", "followers_count": 1368, "friends_count": 1233, "statues_count": 24505 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jenison, MI", "id": "49e7adaebc60092a", "name": "Jenison", "place_type": "city", "bounding_box": rectangle("-85.921923,42.885184 -85.781818,42.936233") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26139, "countyName": "Ottawa", "cityID": 2641680, "cityName": "Jenison" } }
+{ "create_at": datetime("2016-01-06T10:00:26.000Z"), "id": 684796680395427840, "text": "Also, they never say who moved the cheese! What malevolent supernatural beings run this cheese maze? Oh God the uncertainty...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 44046021, "name": "Trey Braun", "screen_name": "T_Braun78", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-06-01"), "description": "Husband. Son. Used to play #GTfootball but I'll always be a #GeorgiaTech student.", "followers_count": 948, "friends_count": 562, "statues_count": 4129 }, "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-06T10:00:26.000Z"), "id": 684796680756047872, "text": "#Engineering #Job in #Carlisle, PA: Maintenance Assistant at CBRE https://t.co/zNEB9wthtN #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.195,40.2025"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "Engineering", "Job", "Carlisle", "Jobs", "Hiring" }}, "user": { "id": 2960014897, "name": "CBRE Jobs", "screen_name": "CBREJobs", "lang": "en", "location": "null", "create_at": date("2015-01-05"), "description": "A globally integrated commercial real estate and investment services firm that transforms real estate into real advantage.", "followers_count": 351, "friends_count": 6, "statues_count": 1533 }, "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-01-06T10:00:26.000Z"), "id": 684796680756137984, "text": "HAP BIRTH to my bestie. out of the hundreds we have, this is the only photo that matters. ��✨����✨������ @whichcarly https://t.co/oaDTCajvmE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4097916133 }}, "user": { "id": 427987696, "name": "hanna", "screen_name": "hannuh_lee", "lang": "en", "location": "elsewhere", "create_at": date("2011-12-03"), "description": "pretty witchy", "followers_count": 1454, "friends_count": 920, "statues_count": 27133 }, "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-01-06T10:00:26.000Z"), "id": 684796681448062978, "text": "Wish Erykah Badu was performing instead! @Micropixie @fatbellybella", "in_reply_to_status": 684796415009132544, "in_reply_to_user": 44987387, "favorite_count": 0, "coordinate": point("-122.268288,37.809974"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 44987387, 18278629 }}, "user": { "id": 44987387, "name": "Single Beige Female", "screen_name": "Micropixie", "lang": "en", "location": "San Francisco", "create_at": date("2009-06-05"), "description": "Extraterrestrial artist (http://micropixie.com) trying on the earthly experience; often moonlights as full-time human being (http://singlebeigefemale.com)", "followers_count": 824, "friends_count": 542, "statues_count": 7972 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paramount Theatre", "id": "07d9d21114c80002", "name": "Paramount Theatre", "place_type": "poi", "bounding_box": rectangle("-122.26828809999999,37.809973899999996 -122.268288,37.809974") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-01-06T10:00:26.000Z"), "id": 684796681536274432, "text": "@TravsTakes4 @jaysonst if you going to use it at all HOF is far better than MVP. WAR enables comparison across eras.", "in_reply_to_status": 684794992288743425, "in_reply_to_user": 449345239, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 449345239, 19735580 }}, "user": { "id": 236862139, "name": "Robert P Brincheck", "screen_name": "rbrincheck", "lang": "en", "location": "Southfield MI USA", "create_at": date("2011-01-11"), "description": "Husband and Labradoodle owner who has been working in and around the Auto industry since the days of Roger B. Mass consumer of IP addresses and all things Apple", "followers_count": 72, "friends_count": 136, "statues_count": 2521 }, "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-01-06T10:00:26.000Z"), "id": 684796681590706176, "text": "@princessriaah_ @Chynna_Maay im in class crackin up ����������", "in_reply_to_status": 684796590754676736, "in_reply_to_user": 3743586133, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3743586133, 4159835779 }}, "user": { "id": 3696032713, "name": "January 8️⃣th ♑️", "screen_name": "NyMariee", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-09-26"), "description": "FreeMyTwin ❣ SirdyWorld ❤️✊ SC:Amour.Sierra", "followers_count": 79, "friends_count": 116, "statues_count": 134 }, "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-06T10:00:26.000Z"), "id": 684796681594900481, "text": "@digiphile Yep. And I suppose many fellow passengers are just getting ready for work. (I'm on a train that's headed toward the Valley.)", "in_reply_to_status": 684795955519029248, "in_reply_to_user": 1175221, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1175221 }}, "user": { "id": 16868756, "name": "Dan Diamond", "screen_name": "ddiamond", "lang": "en", "location": "Washington, D.C.", "create_at": date("2008-10-20"), "description": "Ex-executive editor of @daily_briefing. Happily host @weekly_briefing. Frequently found @Forbes, Vox, ESPN @TrueHoopNetwork. Starting soon @POLITICO.", "followers_count": 10679, "friends_count": 602, "statues_count": 24256 }, "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-06T10:00:26.000Z"), "id": 684796682098221056, "text": "Heellllllllll nah to the nah nah nah https://t.co/OkhjpxOvJD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 2337095740, "name": "jerome", "screen_name": "icantdribble", "lang": "en", "location": "478✈️903 ", "create_at": date("2014-02-10"), "description": "booty, basketball, and babes. Life is locomotion, if you're not moving, you're not living - Barry Allen", "followers_count": 257, "friends_count": 299, "statues_count": 18462 }, "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-06T10:00:26.000Z"), "id": 684796682123358209, "text": "@MonsterProducts how do I go about tracking a package?\nI didn't register, just signed up for emails", "in_reply_to_status": -1, "in_reply_to_user": 5528562, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 5528562 }}, "user": { "id": 42834044, "name": "K.c.", "screen_name": "Fifty5Freak", "lang": "en", "location": "California", "create_at": date("2009-05-27"), "description": "life, music, dogs.", "followers_count": 43, "friends_count": 329, "statues_count": 710 }, "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-06T10:00:26.000Z"), "id": 684796682211491840, "text": "Happy Birthday Nick! I hope you have a great day������@nick_doan2 https://t.co/TYy63d3z1G", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 377820183 }}, "user": { "id": 2307585558, "name": "Jazzlyn Beam", "screen_name": "JazzlynBeam", "lang": "en", "location": "null", "create_at": date("2014-01-23"), "description": "null", "followers_count": 208, "friends_count": 202, "statues_count": 170 }, "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-06T10:00:26.000Z"), "id": 684796682249310209, "text": "@RTarabic \nسوريا سحب الروس البساط من تحت قدميكم ولو تنسحبوا من المفاوضات او يتم طردكم لكان افضل وكفى ابتزاز للعالم https://t.co/gmWUy22jtC", "in_reply_to_status": 684795349597171712, "in_reply_to_user": 71516564, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user_mentions": {{ 71516564 }}, "user": { "id": 1270543177, "name": "سلمت يمينك ياسلمان", "screen_name": "R11U1", "lang": "ar", "location": "Orlando, FL", "create_at": date("2013-03-15"), "description": "‏لا يقتصرالشرف على الجنس فكل خائن لبلاده عديم شرف ووضيع نفس وقدر.وقت الحرب لاحياد (دامت راية التوحيد خفاقة فوق ترابك ياوطني) ،لاعب منتخبي كرة اليد والجودو سابقا", "followers_count": 4529, "friends_count": 3477, "statues_count": 46221 }, "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-06T10:00:26.000Z"), "id": 684796682337292288, "text": "@BarcaGP @Disneyland I find your lack of faith disturbing.", "in_reply_to_status": 684794198520758272, "in_reply_to_user": 26634860, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 26634860, 12378512 }}, "user": { "id": 47536386, "name": "Sam", "screen_name": "BaconBikerSam", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-06-15"), "description": "null", "followers_count": 234, "friends_count": 277, "statues_count": 16159 }, "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-06T10:00:26.000Z"), "id": 684796682647781376, "text": "It's so important to NOT recite Quraan 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": 135368295, "name": "Yasmin Begum", "screen_name": "KraziiKiYaaR3", "lang": "en", "location": "New York", "create_at": date("2010-04-20"), "description": "But perhaps you hate a thing & it is good for you; & perhaps you love a thing & it is bad for you. And Allah knows, while you know not. 2:216", "followers_count": 98, "friends_count": 102, "statues_count": 2193 }, "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-06T10:00:26.000Z"), "id": 684796683096571905, "text": "me & ree talk just about everyday .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2239286229, "name": "savagelife", "screen_name": "alyakjackson29", "lang": "en", "location": "null", "create_at": date("2013-12-22"), "description": "harmonyMarie❤️", "followers_count": 1420, "friends_count": 853, "statues_count": 54782 }, "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-06T10:00:26.000Z"), "id": 684796683167739904, "text": "Souf Philly tonight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 386359355, "name": "Kenny Holsuttin", "screen_name": "SlobSmith", "lang": "en", "location": "Norfeast Philly Fox Chase", "create_at": date("2011-10-06"), "description": "#ILMOPC \nHartel and Rockwell", "followers_count": 236, "friends_count": 352, "statues_count": 10755 }, "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-06T10:00:26.000Z"), "id": 684796683218194435, "text": "@TweetItHowULive I'm so excited", "in_reply_to_status": 684796373380771841, "in_reply_to_user": 353135573, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 353135573 }}, "user": { "id": 117615159, "name": "The Ląшγεг Girl", "screen_name": "KXIVB_", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2010-02-25"), "description": "Molding The Next Generation | Temple U.♕♕ I ♥ politics", "followers_count": 1085, "friends_count": 686, "statues_count": 64968 }, "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-06T10:00:26.000Z"), "id": 684796683603951616, "text": "Can you recommend anyone for this #Labor #job? https://t.co/rmUEz8ad7k #Novi, MI #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.4754913,42.48059"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Labor", "job", "Novi", "Hiring", "CareerArc" }}, "user": { "id": 22740983, "name": "TMJ-DTW Labor Jobs", "screen_name": "tmj_dtw_labor", "lang": "en", "location": "Detroit, MI", "create_at": date("2009-03-03"), "description": "Follow this account for geo-targeted General Labor job tweets in Detroit, MI. Need help? Tweet us at @CareerArc!", "followers_count": 339, "friends_count": 292, "statues_count": 19 }, "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-01-06T10:00:26.000Z"), "id": 684796683629236225, "text": "@CloudKlaus Fucking hell", "in_reply_to_status": 684796617942253568, "in_reply_to_user": 1891254613, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1891254613 }}, "user": { "id": 2725381740, "name": "Pkmn Evolutionaries", "screen_name": "PkmnProfessorK", "lang": "en", "location": "Florida", "create_at": date("2014-08-11"), "description": "Verified PokéTuber with over 15,000 amazing subscribers! Pokémon TCG Player & Professor Contact- PokemonEvolutionaries@gmail.com", "followers_count": 1896, "friends_count": 322, "statues_count": 29495 }, "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-06T10:00:27.000Z"), "id": 684796684421840897, "text": "Accident, two lanes blocked in #Bellevue on I-405 NB approaching WA-520, stop and go traffic back to Lk Washington Blvd, delay of 13 mins", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.1878,47.63216"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Bellevue" }}, "user": { "id": 249879158, "name": "Total Traffic SEA", "screen_name": "TotalTrafficSEA", "lang": "en", "location": "Seattle, WA", "create_at": date("2011-02-09"), "description": "Watching and reporting traffic and transit in Seattle, Tacoma, Bellevue, Everett, and all across Western Washington!", "followers_count": 1687, "friends_count": 347, "statues_count": 90335 }, "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-06T10:00:27.000Z"), "id": 684796684467965954, "text": "SPOILER ALERT: We have an \"alternative\" version of the 3 Kings Story to perform for you later...video to come... ::wink::", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1414372314, "name": "BeingNOLA", "screen_name": "BeingNOLA", "lang": "en", "location": "New Orleans ", "create_at": date("2013-05-08"), "description": "I'm Robin McDowell (@timesnewrobin) I live in Bayou St. John. This is my life. / A new New Orleanian every week.", "followers_count": 8304, "friends_count": 6778, "statues_count": 54396 }, "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-06T10:00:27.000Z"), "id": 684796684547833856, "text": "��❣��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 499850713, "name": "D e m i", "screen_name": "demismith1995", "lang": "en", "location": "null", "create_at": date("2012-02-22"), "description": "You have to get lost before you find yourself", "followers_count": 491, "friends_count": 446, "statues_count": 7306 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Archdale, NC", "id": "0044c2f4709633a2", "name": "Archdale", "place_type": "city", "bounding_box": rectangle("-80.015734,35.83584 -79.885719,35.925449") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37151, "countyName": "Randolph", "cityID": 3701720, "cityName": "Archdale" } }
+{ "create_at": datetime("2016-01-06T10:00:27.000Z"), "id": 684796684845592576, "text": "#jamaicanjerkchicken tomorrow. https://t.co/7qtm8zUOiA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "jamaicanjerkchicken" }}, "user": { "id": 3044849384, "name": "TheBen Randall", "screen_name": "InTheWeedsWBR", "lang": "en", "location": "Chicago", "create_at": date("2015-02-26"), "description": "Official twitter of the In The Weeds Podcast. Kitchen life stories, cheffing all over the place. Listen to my life.", "followers_count": 211, "friends_count": 378, "statues_count": 2183 }, "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-06T10:00:27.000Z"), "id": 684796684891721728, "text": "People share too many intimate details of their lives 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": 104623775, "name": "Dakota Rector", "screen_name": "dakotarector", "lang": "en", "location": "Castle Shannon, PA", "create_at": date("2010-01-13"), "description": "Southern heart. Old soul. Cat lady.", "followers_count": 212, "friends_count": 153, "statues_count": 10137 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carnegie, PA", "id": "926c2b6b7aac446c", "name": "Carnegie", "place_type": "city", "bounding_box": rectangle("-80.099975,40.398563 -80.067374,40.420078") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4211336, "cityName": "Carnegie" } }
+{ "create_at": datetime("2016-01-06T10:00:27.000Z"), "id": 684796685185323008, "text": ".@davidwalshblog TV being the last proprietary device platform we won't see the best days of #FirefoxOS until we can build our own sets", "in_reply_to_status": 684767203044933632, "in_reply_to_user": 15759583, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FirefoxOS" }}, "user_mentions": {{ 15759583 }}, "user": { "id": 3783683481, "name": "Clinton Gallagher", "screen_name": "screenDepotTV", "lang": "en", "location": "Milwaukee County, WI", "create_at": date("2015-09-26"), "description": "Saving Our World One Screen At A Time.\n\nServing mankind w/ digital signage, connected TV, mobile, WWW design-build software, hardware + services.", "followers_count": 61, "friends_count": 89, "statues_count": 277 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wauwatosa, WI", "id": "d378ed20e9e8e270", "name": "Wauwatosa", "place_type": "city", "bounding_box": rectangle("-88.067098,43.030534 -87.983987,43.104541") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5584675, "cityName": "Wauwatosa" } }
+{ "create_at": datetime("2016-01-06T10:00:27.000Z"), "id": 684796685390753793, "text": "I can't stop laughing ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2610050369, "name": "SMALLZZ ☀︎", "screen_name": "caaiilaaa_", "lang": "en", "location": "Riverside, CA & Newport ", "create_at": date("2014-06-14"), "description": "I have bad luck☹ re-born 03 | 12 | 15 ॐ", "followers_count": 602, "friends_count": 400, "statues_count": 9088 }, "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-06T10:00:27.000Z"), "id": 684796685399109632, "text": "It's rendering ! @ZooNajihs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2467760680 }}, "user": { "id": 3198782968, "name": "Prex", "screen_name": "Prex23rd", "lang": "en", "location": "Florida, USA", "create_at": date("2015-04-23"), "description": "Bo1/Bo3 player . 16 years of age. PS4 only", "followers_count": 765, "friends_count": 316, "statues_count": 10549 }, "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-06T10:00:27.000Z"), "id": 684796686007402496, "text": "Stressed is a serious understatement 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": 626818891, "name": "Ragan Fowler", "screen_name": "ragan_fowler", "lang": "en", "location": "null", "create_at": date("2012-07-04"), "description": "let your faith be greater than your fear | jeremiah 29:11 |", "followers_count": 1634, "friends_count": 1229, "statues_count": 27372 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valley Falls, SC", "id": "83f69050c89a6a95", "name": "Valley Falls", "place_type": "city", "bounding_box": rectangle("-82.024291,34.989317 -81.937438,35.028922") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45083, "countyName": "Spartanburg", "cityID": 4573465, "cityName": "Valley Falls" } }
+{ "create_at": datetime("2016-01-06T10:00:27.000Z"), "id": 684796686145839106, "text": "@superstar_beat @johnrobb_1968 @Figster017 @VeryCarefulGirl @momma_rocker @bap31567 @1carolinagirl sweet enjoy������", "in_reply_to_status": 684796341147533313, "in_reply_to_user": 2336867165, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2336867165, 2769464967, 2279739011, 2884849539, 2329189534, 1310670458, 23000618 }}, "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": 665, "friends_count": 532, "statues_count": 20264 }, "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-06T10:00:27.000Z"), "id": 684796686233911296, "text": "Well the word on the street is that my crew on the list of new labels to watch for 2016 - VBE https://t.co/FvQR8EiHGg @DOSS_ROLLIE @WikidRL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 266731847, 2659866157 }}, "user": { "id": 319599507, "name": "VBlockEnt", "screen_name": "CelebrityVBlock", "lang": "en", "location": ".facebook.com/vblockent", "create_at": date("2011-06-18"), "description": "El Presidente drops 01/01/16. \nBook Do$$ Rollie for features and performances: dossrollie@gmail.com", "followers_count": 3849, "friends_count": 1405, "statues_count": 3933 }, "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-06T10:00:27.000Z"), "id": 684796686934265856, "text": "@okeechobeefest is gonna be 4 days....oh shit", "in_reply_to_status": -1, "in_reply_to_user": 3015897352, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3015897352 }}, "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...", "followers_count": 592, "friends_count": 922, "statues_count": 25725 }, "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-06T10:00:27.000Z"), "id": 684796687005532160, "text": "My friends gave me a ride to class and it made my day 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": 518805398, "name": "Angela Griffith", "screen_name": "TxJoGriff", "lang": "en", "location": "TexasUCLA", "create_at": date("2012-03-08"), "description": "null", "followers_count": 361, "friends_count": 56, "statues_count": 24536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:27.000Z"), "id": 684796687433375745, "text": "Want to work in #NewOrleans, LA? View our latest opening: https://t.co/n0t159rjLA #Clerical #Veterans #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.0705556,29.9647222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NewOrleans", "Clerical", "Veterans", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 31625510, "name": "TMJ-LAN Cleric. Jobs", "screen_name": "tmj_lan_cler", "lang": "en", "location": "New Orleans, LA", "create_at": date("2009-04-15"), "description": "Follow this account for geo-targeted Office/Clerical job tweets in New Orleans, LA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 239, "friends_count": 204, "statues_count": 17 }, "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-06T10:00:27.000Z"), "id": 684796687634726912, "text": "See our latest #Brisbane, CA #job and click to apply: Residential Resale Representative - https://t.co/AFcEvUeBfE #Sales #Veterans #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.3999715,37.6807661"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Brisbane", "job", "Sales", "Veterans", "Hiring" }}, "user": { "id": 20827204, "name": "SF Sales Jobs", "screen_name": "tmj_sfo_sales", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-02-13"), "description": "Follow this account for geo-targeted Sales job tweets in San Francisco, CA. Need help? Tweet us at @CareerArc!", "followers_count": 572, "friends_count": 372, "statues_count": 655 }, "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-01-06T10:00:27.000Z"), "id": 684796687756312576, "text": "Damn all this is hitting me so fast. 1st game Friday 7pm @ Orinda 2nd game Sunday 8am @ Pinole valley and then playoff game 7:20 pm .��", "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": 164, "friends_count": 578, "statues_count": 18783 }, "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-06T10:00:28.000Z"), "id": 684796688112943104, "text": "\"Our calling is not to stay alive, but to stay in love with Jesus\"\nTY John Piper. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 549459599, "name": "anna shaw", "screen_name": "annashaww", "lang": "en", "location": "Dahlonega, GA", "create_at": date("2012-04-09"), "description": "[made for more]", "followers_count": 415, "friends_count": 491, "statues_count": 4180 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakwood, GA", "id": "75038a7c3f62b7d5", "name": "Oakwood", "place_type": "city", "bounding_box": rectangle("-83.908273,34.191852 -83.845287,34.25539") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13139, "countyName": "Hall", "cityID": 1357260, "cityName": "Oakwood" } }
+{ "create_at": datetime("2016-01-06T10:00:28.000Z"), "id": 684796688461086720, "text": "Temp 42.2°F Wind Chill 42.2°F RH 23% Wind 1.6 ESE Gust 6.0 ESE SLP 30.484 in Falling quickly Rain 0.00 in Solar 541 UV 2.6 #VAWX #SWVAWX #WV", "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", "WV" }}, "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": 102, "friends_count": 65, "statues_count": 32329 }, "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-06T10:00:28.000Z"), "id": 684796689069293568, "text": "@wingoz Don't forget a 5-0 team didn't make it", "in_reply_to_status": 684789212407803905, "in_reply_to_user": 31447081, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 31447081 }}, "user": { "id": 2169357822, "name": "Greg Chiado", "screen_name": "CoachChiado", "lang": "en", "location": "null", "create_at": date("2013-11-01"), "description": "16 year teaching veteran at WWMS and girls assistant basketball coach at Terre Haute North. Father of 2 and husband of a saint.", "followers_count": 244, "friends_count": 417, "statues_count": 5698 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Terre Haute, IN", "id": "8a61588aff8b0577", "name": "Terre Haute", "place_type": "city", "bounding_box": rectangle("-87.460551,39.394564 -87.303557,39.520714") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18167, "countyName": "Vigo", "cityID": 1875428, "cityName": "Terre Haute" } }
+{ "create_at": datetime("2016-01-06T10:00:28.000Z"), "id": 684796689144610816, "text": "@shobha_malik @merikalamse @AnoorvaSinha @aatishmizaj shukriiiyaa mikki baaji.....", "in_reply_to_status": 684796364312608768, "in_reply_to_user": 3978706034, "favorite_count": 0, "retweet_count": 0, "lang": "hi", "is_retweet": false, "user_mentions": {{ 3978706034, 283505314, 2269126382, 145351263 }}, "user": { "id": 611616646, "name": "binish", "screen_name": "binishraza", "lang": "en", "location": "Detroit, MI", "create_at": date("2012-06-18"), "description": "LOVE STRETCHES YOUR HEART AND MAKE'S YOU BIG INSIDE....\n\n15 Dec ki ek shaam ko naazil hua tha.....Sagittarius kahte hai'n log..", "followers_count": 177, "friends_count": 11, "statues_count": 7636 }, "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-06T10:00:28.000Z"), "id": 684796689425788929, "text": "@AndreasAnthros I only know her through twitter.", "in_reply_to_status": 684795400168050701, "in_reply_to_user": 4079681326, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4079681326 }}, "user": { "id": 20291231, "name": "Ellen Martel", "screen_name": "emarty", "lang": "en", "location": "Chicago, IL", "create_at": date("2009-02-06"), "description": "On @emarty: Every word its own scratchy patch, yet the sum is seamless and fluffy. —Aleksandar Hemon, Author, of Books.", "followers_count": 748, "friends_count": 953, "statues_count": 21052 }, "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-06T10:00:28.000Z"), "id": 684796689467703296, "text": "@yarzthebarz he really is lmao hate his music", "in_reply_to_status": 684794958096773122, "in_reply_to_user": 1631351586, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1631351586 }}, "user": { "id": 142033342, "name": "Dan Tha Man", "screen_name": "NetsFan16", "lang": "en", "location": "null", "create_at": date("2010-05-09"), "description": "Palestine \nPharmD.", "followers_count": 110, "friends_count": 51, "statues_count": 13112 }, "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-06T10:00:28.000Z"), "id": 684796689601982464, "text": "' I hate niggas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 286348548, "name": "BipolarBitch;)", "screen_name": "_TatiyanaaFaith", "lang": "en", "location": "New London, CT", "create_at": date("2011-04-22"), "description": "null", "followers_count": 345, "friends_count": 332, "statues_count": 11633 }, "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-06T10:00:28.000Z"), "id": 684796689916399616, "text": "I need someone to talk to but 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": 396818213, "name": "denise", "screen_name": "badffish", "lang": "en", "location": "null", "create_at": date("2011-10-23"), "description": "5.23❣", "followers_count": 2024, "friends_count": 893, "statues_count": 67497 }, "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-06T10:00:28.000Z"), "id": 684796689933283328, "text": "im scared to call my dad and ask him to pay my tuition fees today lololol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3290341744, "name": "gold hoopz mami", "screen_name": "down2marsgal", "lang": "en", "location": "bay area // pdx", "create_at": date("2015-05-19"), "description": "21//riding the outer ring of my own private saturn", "followers_count": 176, "friends_count": 208, "statues_count": 3753 }, "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-06T10:00:28.000Z"), "id": 684796689941540864, "text": "@kell happy #twitterversary! life wouldn't be the same w/o that face ���� photo credits--- @ged https://t.co/w6rcq9nDNG", "in_reply_to_status": -1, "in_reply_to_user": 133035973, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "twitterversary" }}, "user_mentions": {{ 133035973, 14619572 }}, "user": { "id": 15349556, "name": "diana", "screen_name": "namnam", "lang": "en", "location": "los angeles", "create_at": date("2008-07-07"), "description": "happy @TwitterLA. \nliving a more authentic life, indulging in hot cheetos and too many shoes.", "followers_count": 1566, "friends_count": 815, "statues_count": 1238 }, "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-06T10:00:28.000Z"), "id": 684796690042204160, "text": "@Ms25Lighters I don't wanna hear that bullshit how you slip into some pussy head ass.", "in_reply_to_status": 684796423905226752, "in_reply_to_user": 186976811, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 186976811 }}, "user": { "id": 26179647, "name": "Mse: TEXANS vs Kc.", "screen_name": "_BoobieJohnson", "lang": "en", "location": "MSEGANG, Tx", "create_at": date("2009-03-23"), "description": "love loyalty & life.\n#Mse clothing available for purchase. \nsupport the Mafia.", "followers_count": 1694, "friends_count": 880, "statues_count": 257249 }, "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-01-06T10:00:28.000Z"), "id": 684796690285637632, "text": "I wish Kylum wasn't at school .. I woulda made him take that ride 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": 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": 3807, "friends_count": 2781, "statues_count": 103529 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wallace, LA", "id": "0058c8ba4e9ee92a", "name": "Wallace", "place_type": "city", "bounding_box": rectangle("-90.687578,30.029095 -90.639209,30.046796") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22095, "countyName": "St. John the Baptist", "cityID": 2279275, "cityName": "Wallace" } }
+{ "create_at": datetime("2016-01-06T10:00:28.000Z"), "id": 684796690403033088, "text": "@JonahTulsNFL can we all sit in a circle and listen to Hello and heave cry?", "in_reply_to_status": 684796226110263297, "in_reply_to_user": 2156167098, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2156167098 }}, "user": { "id": 64570054, "name": "Josh Mensch", "screen_name": "JoshMenschNFL", "lang": "en", "location": "Minnetonka, MN", "create_at": date("2009-08-10"), "description": "Midwest #NFLScout for http://www.NFLdraftsquad.com and Scouting Academy student. #Vikings or #NFL q's ask! #drafttwitter #twins #twolves", "followers_count": 650, "friends_count": 610, "statues_count": 13307 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minnetonka, MN", "id": "555209e7a34626a2", "name": "Minnetonka", "place_type": "city", "bounding_box": rectangle("-93.523339,44.890915 -93.398853,44.978956") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743252, "cityName": "Minnetonka" } }
+{ "create_at": datetime("2016-01-06T10:00:28.000Z"), "id": 684796690520358913, "text": "All I do is sit in class & read 50 Shades of Grey lmaoo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4030792332, "name": "pris", "screen_name": "_prissyypriss", "lang": "en", "location": "null", "create_at": date("2015-10-26"), "description": "null", "followers_count": 195, "friends_count": 166, "statues_count": 1025 }, "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-06T10:00:28.000Z"), "id": 684796690675576832, "text": "That's what I thought����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2747983381, "name": "Derrick Alexander", "screen_name": "derrick_73", "lang": "en", "location": "Kaufman, TX", "create_at": date("2014-08-19"), "description": "Kaufman TX- UTPB \nsc: derrick_steven", "followers_count": 149, "friends_count": 377, "statues_count": 428 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kaufman, TX", "id": "1095688d04c9b251", "name": "Kaufman", "place_type": "city", "bounding_box": rectangle("-96.324625,32.556042 -96.279528,32.597534") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48257, "countyName": "Kaufman", "cityID": 4838488, "cityName": "Kaufman" } }
+{ "create_at": datetime("2016-01-06T10:00:28.000Z"), "id": 684796690767818752, "text": "I JUST WANT IT TO BE SUMMER TIME. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467015442, "name": "Kelsie", "screen_name": "Kelsie_Raignbow", "lang": "en", "location": "San Antonio, TX", "create_at": date("2012-01-17"), "description": "Let your mind grow and let the good times roll #TXST19", "followers_count": 793, "friends_count": 676, "statues_count": 20011 }, "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-06T10:00:28.000Z"), "id": 684796690944098305, "text": "@dukeofberlin were the bottles of wine empty and recently drunk? maybe that was your problem ;)", "in_reply_to_status": 684796515861282816, "in_reply_to_user": 21734177, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21734177 }}, "user": { "id": 18210275, "name": "Justine Arreche", "screen_name": "SaltineJustine", "lang": "en", "location": "Where the Wild Things Are", "create_at": date("2008-12-17"), "description": "I never met a deep fryer I didn't like — Lead Clipart Strategist at @travisci.", "followers_count": 2245, "friends_count": 235, "statues_count": 23343 }, "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-06T10:00:28.000Z"), "id": 684796690985926656, "text": "@Giggggss maybe if u weren't so into your pie when I'm in a crisis", "in_reply_to_status": 684796468788621312, "in_reply_to_user": 1169146080, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1169146080 }}, "user": { "id": 376299222, "name": "Melissa Wap", "screen_name": "bunnyofthesnow", "lang": "en", "location": "Toms River, NJ", "create_at": date("2011-09-19"), "description": "let your heart guide you", "followers_count": 331, "friends_count": 250, "statues_count": 13717 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, NJ", "id": "a10f50910f169b2f", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-74.259004,40.038816 -74.15081,40.120394") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean", "cityID": 3438580, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-01-06T10:00:28.000Z"), "id": 684796691107586048, "text": "hit da Quan", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 2707951062, "name": "PRL", "screen_name": "glotevo", "lang": "en", "location": "Spring, TX", "create_at": date("2014-08-04"), "description": "WHS 17' golden state warriors", "followers_count": 2182, "friends_count": 1421, "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-01-06T10:00:28.000Z"), "id": 684796691120181248, "text": "@katie_carley7 haha at least I found you guys ��", "in_reply_to_status": 684796621767315456, "in_reply_to_user": 1629471937, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1629471937 }}, "user": { "id": 2938924603, "name": "قصير وكبير", "screen_name": "_xeliax_", "lang": "en", "location": "c/o 2016 Lhs", "create_at": date("2014-12-21"), "description": "❤️ Mi ojitos cafésitos @shakiraprieto26❤️", "followers_count": 393, "friends_count": 437, "statues_count": 8540 }, "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-06T10:00:28.000Z"), "id": 684796691241897984, "text": "My time up. Its back to the drawing board. https://t.co/CLyfmeb9JG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 120562342, "name": "Narco Reひs", "screen_name": "UncleTrini", "lang": "en", "location": "Greensboro, NC", "create_at": date("2010-03-06"), "description": "null", "followers_count": 2844, "friends_count": 965, "statues_count": 480875 }, "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-06T10:00:28.000Z"), "id": 684796691241930752, "text": "I want this �� https://t.co/g7szuNRX3e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1598931295, "name": "a∽h", "screen_name": "ashlee_helm7", "lang": "en", "location": "null", "create_at": date("2013-07-16"), "description": "null", "followers_count": 577, "friends_count": 131, "statues_count": 17224 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elizabethtown, KY", "id": "998c5b6035f9b5a4", "name": "Elizabethtown", "place_type": "city", "bounding_box": rectangle("-85.952048,37.626084 -85.789831,37.771861") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21093, "countyName": "Hardin", "cityID": 2124274, "cityName": "Elizabethtown" } }
+{ "create_at": datetime("2016-01-06T10:00:28.000Z"), "id": 684796691397111809, "text": "@MjBdatGuy why you inviting these hoes to come smoke w you", "in_reply_to_status": 684796163564908544, "in_reply_to_user": 347680150, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 347680150 }}, "user": { "id": 269363936, "name": "sarah", "screen_name": "sarahashleyx_", "lang": "en", "location": "NBK", "create_at": date("2011-03-20"), "description": "null", "followers_count": 506, "friends_count": 185, "statues_count": 30918 }, "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-01-06T10:00:28.000Z"), "id": 684796691560710145, "text": "���� https://t.co/M70mQNZ5eL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 415915050, "name": "★☆", "screen_name": "LauraWtfsky", "lang": "en", "location": "☆TG☯D★ ", "create_at": date("2011-11-18"), "description": "NY ✿ Long Live W♾LFE", "followers_count": 2358, "friends_count": 911, "statues_count": 34284 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bay Shore, NY", "id": "1711fcf6c5dcde3c", "name": "Bay Shore", "place_type": "city", "bounding_box": rectangle("-73.285247,40.705724 -73.223943,40.752091") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3604935, "cityName": "Bay Shore" } }
+{ "create_at": datetime("2016-01-06T10:00:28.000Z"), "id": 684796691573112832, "text": "Westlake Park Wednesday's! 11-6. Get your Taiyaki on! New flavors!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1348196868, "name": "BeanFish Food Truck", "screen_name": "BeanFishTaiyaki", "lang": "en", "location": "Streets of Seattle", "create_at": date("2013-04-12"), "description": "TAIYAKI - Sweet & savory stuffed fish-shaped waffles. Made the old school way using cast iron, flame and homemade batter. Search #beanfish to see the goods.", "followers_count": 569, "friends_count": 739, "statues_count": 779 }, "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-06T10:00:28.000Z"), "id": 684796691615059969, "text": "@madddiexo @NickerPlzz sorry that was way harsh", "in_reply_to_status": 684794022234329088, "in_reply_to_user": 399142284, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 62684616, 491999779 }}, "user": { "id": 399142284, "name": "Krystalle Eid", "screen_name": "KrystalleEid", "lang": "en", "location": "♍️", "create_at": date("2011-10-26"), "description": "And the rest is rust and stardust", "followers_count": 307, "friends_count": 308, "statues_count": 8616 }, "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-01-06T10:00:28.000Z"), "id": 684796691648778240, "text": "Wind 2.0 mph SSW. Barometer 30.347 in, Falling. Temperature 38.2 °F. Rain today 0.00 in. Humidity 55%", "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": 287, "statues_count": 7303 }, "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-06T10:00:28.000Z"), "id": 684796691707396096, "text": "Roll my weed up keep my D up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.969998,34.055416"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 304851751, "name": "Michael", "screen_name": "Mike14OPM", "lang": "en", "location": "Land Of No Religion", "create_at": date("2011-05-24"), "description": "23/typical lonely stoner", "followers_count": 1022, "friends_count": 1378, "statues_count": 18953 }, "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-01-06T10:00:28.000Z"), "id": 684796691959033856, "text": "get your tickets now before my parents buy them all @Illeanarama https://t.co/gx3KLdozgq @wifeytv @sundancefest #theskinnytv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "theskinnytv" }}, "user_mentions": {{ 107325601, 1620771378, 12369372 }}, "user": { "id": 212005789, "name": "Jessie Kahnweiler", "screen_name": "shegotchutzpah", "lang": "en", "location": "null", "create_at": date("2010-11-04"), "description": "i cant afford therapy so i make films. i live in LA with my plants.", "followers_count": 3711, "friends_count": 941, "statues_count": 3673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:28.000Z"), "id": 684796692022071297, "text": "Hell no. https://t.co/kgInYkYS1u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2863154052, "name": "tiara ✨", "screen_name": "_tiaramonet", "lang": "en", "location": "Asbury Park, NJ", "create_at": date("2014-10-18"), "description": "Instagram: @_monettiara Snapchat: tmonet97 Temple University '19", "followers_count": 662, "friends_count": 479, "statues_count": 7183 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Asbury Park, NJ", "id": "68f46439c0f4a207", "name": "Asbury Park", "place_type": "city", "bounding_box": rectangle("-74.027125,40.213215 -73.995775,40.231329") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3401960, "cityName": "Asbury Park" } }
+{ "create_at": datetime("2016-01-06T10:00:28.000Z"), "id": 684796692219088896, "text": "Ppl need to stop telling me I look like the dude in the backseat of that vine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 67841880, "name": "Christopher Ray", "screen_name": "cmacbufu", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2009-08-22"), "description": "22. Instagram: cmacbufu Snapchat: cmacbufu #PartGang", "followers_count": 1432, "friends_count": 326, "statues_count": 19879 }, "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-06T10:00:29.000Z"), "id": 684796692470734848, "text": "Customer Service Representative II - Sunshine Health: (#Sunrise, FL) https://t.co/lLtGJcf5cx #CustomerService #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.2862256,26.1571743"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sunrise", "CustomerService", "Job", "Jobs", "Hiring" }}, "user": { "id": 72372511, "name": "TMJ-FLF CstSrv Jobs", "screen_name": "tmj_flf_cstsrv", "lang": "en", "location": "Ft. Lauderdale, FL", "create_at": date("2009-09-07"), "description": "Follow this account for geo-targeted Customer Service job tweets in Ft. Lauderdale, FL from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 293, "friends_count": 264, "statues_count": 74 }, "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-06T10:00:29.000Z"), "id": 684796692722483201, "text": "Plies is everyday folk man���� sound like my mama the other day complaining https://t.co/JLO0aA2wEa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 143182388, "name": "Ash✨", "screen_name": "ashlaylay__", "lang": "en", "location": "null", "create_at": date("2010-05-12"), "description": "your presence is a treasure at all times. #wssu18 #ripauntie", "followers_count": 1910, "friends_count": 1105, "statues_count": 212793 }, "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-06T10:00:29.000Z"), "id": 684796692751777794, "text": "The Feds they picking up this case...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 435328217, "name": "2Focused2Care", "screen_name": "ThereGoesLance", "lang": "en", "location": "Washington, USA", "create_at": date("2011-12-12"), "description": "#VibeWithUsENT", "followers_count": 1095, "friends_count": 425, "statues_count": 69661 }, "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-01-06T10:00:29.000Z"), "id": 684796692953051136, "text": "@Jacob_B_Shaver, oh, you're a comic. Where do you perform? I'm from #Chicago. Look up corruption and it reads: #TheChicagoDemocratMachine!", "in_reply_to_status": 684796186943881216, "in_reply_to_user": 2502397874, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Chicago", "TheChicagoDemocratMachine" }}, "user_mentions": {{ 2502397874 }}, "user": { "id": 22549562, "name": "Stephanie Trussell", "screen_name": "ReaganMom", "lang": "en", "location": "Sun Valley", "create_at": date("2009-03-02"), "description": "Heard Sundays, 9p-12a CST WLS 890 AM", "followers_count": 1324, "friends_count": 1910, "statues_count": 10337 }, "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-06T10:00:29.000Z"), "id": 684796693074714624, "text": "Did anyone read the Huffpost today?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4028916440, "name": "Andrea L. Sherrell", "screen_name": "lois620", "lang": "en", "location": "Des Moines, WA", "create_at": date("2015-10-26"), "description": "Paralegal Student-Highline", "followers_count": 7, "friends_count": 42, "statues_count": 155 }, "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-06T10:00:29.000Z"), "id": 684796693204750336, "text": "SkyScrape + Guest Posting = Links #SEO https://t.co/SyXMCsU9b8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SEO" }}, "user": { "id": 3331546606, "name": "August Noble", "screen_name": "AugustNoble", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-06-17"), "description": "SEO & Content @SnackNation - (http://backlinko.com/white-hat-seo) | Entrepreneurship | Millennials | Get Better Everyday", "followers_count": 392, "friends_count": 455, "statues_count": 371 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:29.000Z"), "id": 684796693489913856, "text": "@HarneyCoSheriff Rep Greg Waldon speaking from the State of House Floor. My GOD what's going on in Oregon? \nhttps://t.co/rXOH4r7Cas", "in_reply_to_status": -1, "in_reply_to_user": 4709625230, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4709625230 }}, "user": { "id": 3578738958, "name": "MoniQue", "screen_name": "MoniQueMoniCat", "lang": "en", "location": "null", "create_at": date("2015-09-15"), "description": "I support Trump & Matteo Salvini. La lince magica delle alpi, è tornato! Home of the edelweiss, majestic mountains of might, the spectacular Italian Alps.", "followers_count": 1403, "friends_count": 1087, "statues_count": 9423 }, "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-06T10:00:29.000Z"), "id": 684796693532020736, "text": "@TrevonSimmons4 oh my�� https://t.co/wo3CD987hS", "in_reply_to_status": -1, "in_reply_to_user": 828160232, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 828160232 }}, "user": { "id": 451626855, "name": "Raider Alumni✨", "screen_name": "Bishh_Relax", "lang": "en", "location": "•XI.XXII•", "create_at": date("2011-12-31"), "description": "OVOXO", "followers_count": 369, "friends_count": 465, "statues_count": 4934 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Rockingham, NC", "id": "7d7de27c2e7b37fd", "name": "East Rockingham", "place_type": "city", "bounding_box": rectangle("-79.794806,34.891032 -79.734267,34.930784") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37153, "countyName": "Richmond", "cityID": 3719800, "cityName": "East Rockingham" } }
+{ "create_at": datetime("2016-01-06T10:00:29.000Z"), "id": 684796693989036032, "text": "Aye bruh I used to go to beacon light before the storm I know how they get down.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 236156697, "name": "Kenny", "screen_name": "KennyWitTheKoke", "lang": "en", "location": "318225", "create_at": date("2011-01-09"), "description": "Excuse the Rant | The Last Cocaine Super Hero | #EnginEars | Inquirie$ : excusetherant@gmail.com |Come in my mentions sideways you getting Mutombo'd", "followers_count": 633, "friends_count": 314, "statues_count": 88632 }, "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-01-06T10:00:29.000Z"), "id": 684796694022635520, "text": "@elidourado Power surge this morning screwed up my desktop PC *again*. Something needs to be done about the squirrel menace.", "in_reply_to_status": 684795347256897536, "in_reply_to_user": 851361, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 851361 }}, "user": { "id": 9218762, "name": "Franklin Harris", "screen_name": "FranklinH3000", "lang": "en", "location": "Alabama", "create_at": date("2007-10-02"), "description": "Love child of Pauline Kael and Joe Bob Briggs. Retweets aren't endorsements except when they are. Even more annoying in person. #critic #gadfly #knowitall", "followers_count": 1416, "friends_count": 1196, "statues_count": 69295 }, "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-06T10:00:29.000Z"), "id": 684796694198796290, "text": "If you travel by car you know sometimes you get no radio or data service while on the road...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 816497430, "name": "Morales", "screen_name": "JmRockstarlife", "lang": "en", "location": "null", "create_at": date("2012-09-10"), "description": "null", "followers_count": 229, "friends_count": 297, "statues_count": 19912 }, "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-06T10:00:29.000Z"), "id": 684796694391750656, "text": "I wonder what Kevin Walter is doing 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": 106841648, "name": "Dakota", "screen_name": "codyofamerica", "lang": "en", "location": "we call it Houston now ", "create_at": date("2010-01-20"), "description": "a product of my surrounding and ima always rep the south east. I'm gonna get to where I'm going. #onelove", "followers_count": 275, "friends_count": 343, "statues_count": 11807 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-06T10:00:29.000Z"), "id": 684796694748200960, "text": "�� https://t.co/D6nVhkWCM4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 457884277, "name": "Jeremy Snow", "screen_name": "Jeremysnow_33", "lang": "en", "location": "Pittsburg, KS", "create_at": date("2012-01-07"), "description": "PSU 19' #MakeAmericaGreatAgain", "followers_count": 686, "friends_count": 684, "statues_count": 14314 }, "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-06T10:00:29.000Z"), "id": 684796694966370304, "text": "@steimer who do you work for? Congrats", "in_reply_to_status": 684793619971063808, "in_reply_to_user": 18501326, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18501326 }}, "user": { "id": 2973721354, "name": "Enrique Limon", "screen_name": "EnriqueLimon619", "lang": "en", "location": "Oceanside, CA", "create_at": date("2015-01-11"), "description": "Loves Games, Wingz, Beer, and... oh yeah, Wife", "followers_count": 12, "friends_count": 17, "statues_count": 332 }, "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-06T10:00:29.000Z"), "id": 684796695457214464, "text": "I spy cute Starbucks cups with xo's on them and I need one", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 81291136, "name": "bstan", "screen_name": "b___unit", "lang": "en", "location": "419, Ohio", "create_at": date("2009-10-09"), "description": "@ctvbleezy's bbgirl || bist du bekifft?", "followers_count": 622, "friends_count": 316, "statues_count": 30140 }, "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-06T10:00:29.000Z"), "id": 684796695536795649, "text": "@mayitoalvarado1 listo ����", "in_reply_to_status": 684796470789144577, "in_reply_to_user": 1880489407, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 1880489407 }}, "user": { "id": 631421206, "name": "Laura M. Gómez", "screen_name": "LauraGomezNews", "lang": "en", "location": "Houston, Texas", "create_at": date("2012-07-09"), "description": "Periodista Deportiva,Bilingüe, Sports,Chocolate&Coffee addict✌️⚽️#TeamNoSleep", "followers_count": 415, "friends_count": 253, "statues_count": 2158 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-06T10:00:29.000Z"), "id": 684796695570321409, "text": "@plarson99 thanks fam ❤️��", "in_reply_to_status": 684794900093644801, "in_reply_to_user": 2953920338, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2953920338 }}, "user": { "id": 863712260, "name": "Noga", "screen_name": "MichaelNoga6", "lang": "en", "location": "Andover Highschool - 17'", "create_at": date("2012-10-05"), "description": "BANG BROS | Baseball/Football #6️⃣. You Don't Score Until You Score", "followers_count": 1485, "friends_count": 933, "statues_count": 8407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Andover, MN", "id": "219ccc645d458eee", "name": "Andover", "place_type": "city", "bounding_box": rectangle("-93.40681,45.21178 -93.265395,45.304142") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27003, "countyName": "Anoka", "cityID": 2701486, "cityName": "Andover" } }
+{ "create_at": datetime("2016-01-06T10:00:29.000Z"), "id": 684796695587106816, "text": "@ogudino lol Tells you I be on my own shit I ain't like errrbody", "in_reply_to_status": 684793910544039936, "in_reply_to_user": 45289061, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 45289061 }}, "user": { "id": 354936969, "name": "ᏟhᎾᏟᎾᏞᎪᏆᎬ ᏢᎪᏢᎥ", "screen_name": "TwanHefner_", "lang": "en", "location": "Isla Vista, CA", "create_at": date("2011-08-14"), "description": "I may be single but at least I'm not getting cheated on! Inglewood ~ Corona ~ Santa Barbara", "followers_count": 1420, "friends_count": 803, "statues_count": 132316 }, "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-06T10:00:29.000Z"), "id": 684796696073756672, "text": "Books books books I need so many books", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 762779606, "name": "Evan Lee", "screen_name": "HistorybyMrLee", "lang": "en", "location": "Raleigh", "create_at": date("2012-08-16"), "description": "Mr. Lee's former classroom twitter for news, assignments, due dates & bad jokes. Current commentary on life in law school. Find me in library near you!", "followers_count": 86, "friends_count": 185, "statues_count": 825 }, "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-06T10:00:29.000Z"), "id": 684796696094588928, "text": "@tyannaalissaa ?????", "in_reply_to_status": 684796411469119488, "in_reply_to_user": 1138283378, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1138283378 }}, "user": { "id": 420825817, "name": "olive", "screen_name": "Olivia_Oropesa", "lang": "en", "location": "null", "create_at": date("2011-11-24"), "description": "26E4U", "followers_count": 657, "friends_count": 601, "statues_count": 10772 }, "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-06T10:00:29.000Z"), "id": 684796696258326531, "text": "@hgruza lol doing mine saturday", "in_reply_to_status": 684790128464793600, "in_reply_to_user": 2245147666, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2245147666 }}, "user": { "id": 1204862030, "name": "Fante", "screen_name": "fantethecreator", "lang": "en", "location": "Orlando Green", "create_at": date("2013-02-21"), "description": "It is what it is @kelseyann_12\n\nEpsilon - Eta KS GP", "followers_count": 283, "friends_count": 165, "statues_count": 16850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wedgefield, FL", "id": "013cdd1d063c3e46", "name": "Wedgefield", "place_type": "city", "bounding_box": rectangle("-81.101517,28.468354 -81.062402,28.518033") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1275612, "cityName": "Wedgefield" } }
+{ "create_at": datetime("2016-01-06T10:00:29.000Z"), "id": 684796696388354048, "text": "@BHPhotoVideo #CES2016 any price info on the new Kodak camera?", "in_reply_to_status": 684793899370467328, "in_reply_to_user": 10422602, "favorite_count": 0, "coordinate": point("-81.39211733,28.52252252"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CES2016" }}, "user_mentions": {{ 10422602 }}, "user": { "id": 37599957, "name": "Corey Steib", "screen_name": "corey_steib", "lang": "en", "location": "Orlando,FL", "create_at": date("2009-05-03"), "description": "Professional Filmmaker going on 12 years. Works with @CreativeInlet @BLUESHAPE_IT Contributor to the The 6th Edition of the Camera Assistant Manuel.", "followers_count": 737, "friends_count": 577, "statues_count": 5735 }, "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-06T10:00:30.000Z"), "id": 684796696677621760, "text": "We're #hiring! Read about our latest #job opening here: Bartender - https://t.co/1KDLBluXXQ #Arcadia, CA #Hospitality #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.022293,34.139698"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Arcadia", "Hospitality", "CareerArc" }}, "user": { "id": 88003804, "name": "TMJ-CA HRTA Jobs", "screen_name": "tmj_ca_hrta", "lang": "en", "location": "California", "create_at": date("2009-11-06"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 431, "friends_count": 297, "statues_count": 10867 }, "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-01-06T10:00:30.000Z"), "id": 684796696887455744, "text": "@BiggBoss @princenarula88 Yes he's Mr Task����", "in_reply_to_status": 684782291650912257, "in_reply_to_user": 1652516689, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1652516689, 3069293665 }}, "user": { "id": 2888695493, "name": "Sumen Gill", "screen_name": "GillSumen", "lang": "en", "location": "null", "create_at": date("2014-11-22"), "description": "null", "followers_count": 40, "friends_count": 68, "statues_count": 3416 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coram, NY", "id": "92e1e697abf56722", "name": "Coram", "place_type": "city", "bounding_box": rectangle("-73.039115,40.837693 -72.972416,40.921065") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3618157, "cityName": "Coram" } }
+{ "create_at": datetime("2016-01-06T10:00:30.000Z"), "id": 684796696904122368, "text": "Can you recommend anyone for this #Hospitality #job? https://t.co/Gqfz85t0sB #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.8857595,41.1583556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 22453816, "name": "Chicago Hospitality", "screen_name": "tmj_chi_hrta", "lang": "en", "location": "Chicago, IL", "create_at": date("2009-03-01"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Chicago, IL. Need help? Tweet us at @CareerArc!", "followers_count": 549, "friends_count": 299, "statues_count": 1354 }, "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-01-06T10:00:30.000Z"), "id": 684796697008947202, "text": "Happy birthday to Alex Turner ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2392093308, "name": "Sarah Luna.", "screen_name": "seasideluna", "lang": "en", "location": "Los Angeles, CA ", "create_at": date("2014-03-15"), "description": "making music that is decent ♡ I love indie/rock/alt/punk xx (http://themoonghost.com)", "followers_count": 1446, "friends_count": 1422, "statues_count": 5521 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:30.000Z"), "id": 684796697088622592, "text": "@Beatsbyedwin nah that's just t-mobile for you lol", "in_reply_to_status": 684795779664457728, "in_reply_to_user": 389513848, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 389513848 }}, "user": { "id": 2310407688, "name": "Christian Marquez", "screen_name": "summthing_nice", "lang": "en", "location": "null", "create_at": date("2014-01-25"), "description": "snapchat: christian_nnnnn", "followers_count": 339, "friends_count": 412, "statues_count": 1776 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oregon, IL", "id": "4d7a49e983a1f416", "name": "Oregon", "place_type": "city", "bounding_box": rectangle("-89.356693,41.999369 -89.310518,42.027207") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17141, "countyName": "Ogle", "cityID": 1756484, "cityName": "Oregon" } }
+{ "create_at": datetime("2016-01-06T10:00:30.000Z"), "id": 684796697361293312, "text": "finally ran into that Macklemore dude this morning at my barbershop. \n\nHe had boots on that clarks makes for girls called the Pita Sedona.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 265840075, "name": "limousine lowell.", "screen_name": "inasleepingbag", "lang": "en", "location": "null", "create_at": date("2011-03-13"), "description": "null", "followers_count": 226, "friends_count": 47, "statues_count": 9454 }, "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-06T10:00:30.000Z"), "id": 684796697604714500, "text": "Radio premiere of \"Black Infinity (Upside Down)\" by @sulktheband coming up on @EVRadio on PEER⚡️PRESSURE https://t.co/hGA6sGuG1z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 334757472, 18815539 }}, "user": { "id": 324120931, "name": "Damian/Peer Pressure", "screen_name": "DamianGenuardi", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2011-06-25"), "description": "Host of Peer Pressure on @evradio, Vintage collector/dealer, Bassist for My Best Fiend, XPLSN, Panic, I.M.E", "followers_count": 689, "friends_count": 741, "statues_count": 1807 }, "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-06T10:00:30.000Z"), "id": 684796697747271681, "text": "Sen.@SenateMajLdr McConnell & @GovMattBevin : House flip only a matter of time. https://t.co/eBaYvsYx1X #kyga16 https://t.co/FO1sFjqhbL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "kyga16" }}, "user_mentions": {{ 1249982359, 3141213592 }}, "user": { "id": 25276932, "name": "Brad Bowman", "screen_name": "Bradleybowman", "lang": "en", "location": "Kentucky", "create_at": date("2009-03-19"), "description": "State gov/political journo @statejournal brad.bowman@state-journal.com 4 any tips or encrypted anonymously at fifthestate@tutanota.com. Tweets ≠ endorsements", "followers_count": 945, "friends_count": 1560, "statues_count": 3505 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Frankfort, KY", "id": "2a39cded7c6fc606", "name": "Frankfort", "place_type": "city", "bounding_box": rectangle("-84.930744,38.143682 -84.77634,38.231254") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21073, "countyName": "Franklin", "cityID": 2128900, "cityName": "Frankfort" } }
+{ "create_at": datetime("2016-01-06T10:00:30.000Z"), "id": 684796697868939264, "text": "Is there a trash emoji", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2272834087, "name": "Daddy Miller", "screen_name": "MillerJ46_", "lang": "en", "location": "Apopka, FL", "create_at": date("2014-01-02"), "description": "null", "followers_count": 679, "friends_count": 327, "statues_count": 7478 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Apopka, FL", "id": "18ff21ef9765522d", "name": "Apopka", "place_type": "city", "bounding_box": rectangle("-81.579909,28.664067 -81.479167,28.757424") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1201700, "cityName": "Apopka" } }
+{ "create_at": datetime("2016-01-06T10:00:30.000Z"), "id": 684796697877319682, "text": "Don't enter the new year with these long ass ghetto girl nails either .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1146658507, "name": "Kay Alexandria ✨", "screen_name": "callmekatiah", "lang": "en", "location": "SomewhereWaitingOnMyGloUp '", "create_at": date("2013-02-03"), "description": "TheUnsinkable1903. God Bless The Child That Can Hold HER Own. Fleek Them Eyebrows Honey. GodBlessinAllTheTrapNiggas. MelaninAppreciated", "followers_count": 1571, "friends_count": 2180, "statues_count": 64012 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manchester, VA", "id": "00677c8863cc1709", "name": "Manchester", "place_type": "city", "bounding_box": rectangle("-77.568289,37.464712 -77.515246,37.52151") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51041, "countyName": "Chesterfield", "cityID": 5148996, "cityName": "Manchester" } }
+{ "create_at": datetime("2016-01-06T10:00:30.000Z"), "id": 684796697998786561, "text": "Happy birthday @sennet23 ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 522690255 }}, "user": { "id": 223556429, "name": "Sonia Dimas †", "screen_name": "SassySonia_", "lang": "en", "location": "Wherever God takes me...", "create_at": date("2010-12-06"), "description": "✨Live in such a way that if anyone should speak badly of you, nobody would believe it. #UNT16 #Psalm46:5✨ #SaraStrong ZTA", "followers_count": 1037, "friends_count": 395, "statues_count": 63948 }, "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-06T10:00:30.000Z"), "id": 684796698049286145, "text": "“@thefactguide: Do not give people more than what they deserve...a simple plant dies if you over water it..” ����", "in_reply_to_status": 684653169352163331, "in_reply_to_user": 834988327, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 834988327 }}, "user": { "id": 105168765, "name": "JR Florence", "screen_name": "DaThotKilla", "lang": "en", "location": "null", "create_at": date("2010-01-15"), "description": "ONU'18 | football player ✊ | DB nation |Kik:Jamellflorence Snapchat:Thatsanono", "followers_count": 1349, "friends_count": 849, "statues_count": 8252 }, "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-06T10:00:30.000Z"), "id": 684796698296635392, "text": "See our latest #Indianapolis, IN #job and click to apply: Financial Services... - https://t.co/VkDx4iGgS6 #Finance https://t.co/R5hhCT9V3r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.1580423,39.7683765"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Indianapolis", "job", "Finance" }}, "user": { "id": 4099603043, "name": "WestPoint Financial", "screen_name": "WestpointF", "lang": "en", "location": "Indianapolis, IN", "create_at": date("2015-11-02"), "description": "Our mission at WestPoint Financial Group is simple. We are dedicated to educating our clients to help them fulfill their financial goals.", "followers_count": 34, "friends_count": 80, "statues_count": 10 }, "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-01-06T10:00:30.000Z"), "id": 684796698464358400, "text": "Called the bitch again and got no answer at this point I'm very pissy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 167187745, "name": "〽️RIP MOMMA〽️", "screen_name": "_ImTyght", "lang": "en", "location": "Arlington ✈️ Borsicana", "create_at": date("2010-07-15"), "description": "Doing It Big For My Moms ❤️ #NavarroCollege #GOATGang #SoXtra", "followers_count": 2461, "friends_count": 1561, "statues_count": 184786 }, "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-06T10:00:30.000Z"), "id": 684796698569404417, "text": "Sc me that pussy ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 544477497, "name": "ITS MY BIRTHDAY", "screen_name": "___PrettyGangg", "lang": "en", "location": "☀️SonnySide☀️", "create_at": date("2012-04-03"), "description": "FreeMyBrother", "followers_count": 7644, "friends_count": 3146, "statues_count": 143933 }, "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-06T10:00:30.000Z"), "id": 684796698749612034, "text": "Good luck to me waking up for school, sleeping schedule messed up big time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 317615779, "name": "Alx Little", "screen_name": "AlxLittle17", "lang": "en", "location": "T-Town/James 4:7", "create_at": date("2011-06-14"), "description": "Start strong, play strong, finish strong - sc: alxlittle17", "followers_count": 711, "friends_count": 675, "statues_count": 45784 }, "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-06T10:00:30.000Z"), "id": 684796698862944256, "text": "@TiffanyWatsonX by all means please cum!!! ����������������", "in_reply_to_status": 684795511421812736, "in_reply_to_user": 3418256273, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3418256273 }}, "user": { "id": 945651854, "name": "Kevin O", "screen_name": "KevinO_XXX", "lang": "en", "location": "Gettysburg, Pennsylvania ", "create_at": date("2012-11-13"), "description": "A lover of porn!!!!! My favorites @xoxoashleyadams @SamanthaRone @AriaAlexzilla @alixlynxXXX @TinaKayxxx @XNicoleAnistonX @HeatherVahn @AlexXxisAdams @xxxjms", "followers_count": 318, "friends_count": 1576, "statues_count": 11994 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gettysburg, PA", "id": "5b98b73bfd4304d9", "name": "Gettysburg", "place_type": "city", "bounding_box": rectangle("-77.246659,39.815866 -77.217064,39.843733") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42001, "countyName": "Adams", "cityID": 4228960, "cityName": "Gettysburg" } }
+{ "create_at": datetime("2016-01-06T10:00:30.000Z"), "id": 684796698946867200, "text": "https://t.co/DkIV9RaZW1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 67800625, "name": "Kimberly", "screen_name": "xxWildLife", "lang": "en", "location": "null", "create_at": date("2009-08-21"), "description": "Kimberly. 24. Columbus / Ohio.", "followers_count": 828, "friends_count": 200, "statues_count": 78761 }, "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-06T10:00:30.000Z"), "id": 684796699190124544, "text": "That's nigga act like, I ain't have that nigga back...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2450813404, "name": "MontanaDaWorld ❗️", "screen_name": "neesh_montana", "lang": "en", "location": "from Jersey to Charlotte ", "create_at": date("2014-03-31"), "description": "Motto: #LetsBeGreat", "followers_count": 53, "friends_count": 163, "statues_count": 778 }, "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-01-06T10:00:30.000Z"), "id": 684796699387154432, "text": "Interested in a #HR #job near #Livonia, MI? This could be a great fit: https://t.co/hnUzLZEMyR #Veterans #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.3527097,42.36837"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HR", "job", "Livonia", "Veterans", "Hiring" }}, "user": { "id": 23130397, "name": "TMJ-DTW HR Jobs", "screen_name": "tmj_dtw_hr", "lang": "en", "location": "Detroit, MI", "create_at": date("2009-03-06"), "description": "Follow this account for geo-targeted Human Resources job tweets in Detroit, MI. Need help? Tweet us at @CareerArc!", "followers_count": 388, "friends_count": 294, "statues_count": 20 }, "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-01-06T10:00:30.000Z"), "id": 684796699710218240, "text": "@SaltyWalty70 you're gay", "in_reply_to_status": 683481176158396416, "in_reply_to_user": 492566353, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 492566353 }}, "user": { "id": 859036754, "name": "❁demz❁", "screen_name": "demifg", "lang": "en", "location": "VA", "create_at": date("2012-10-02"), "description": "Romans 12:2 ~Affiliated with CBS~ Life Goes On(-:", "followers_count": 548, "friends_count": 289, "statues_count": 19402 }, "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-01-06T10:00:30.000Z"), "id": 684796699806568448, "text": "I know you thought we had something special, but you don't mean nothing 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": 973075980, "name": "s.lux", "screen_name": "Lvvx0", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-11-26"), "description": "we can't direct the wind", "followers_count": 238, "friends_count": 103, "statues_count": 1977 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:30.000Z"), "id": 684796700364517376, "text": "100% believe in karma", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 285389659, "name": "Sosa", "screen_name": "SamPrenatt", "lang": "en", "location": "inside ", "create_at": date("2011-04-20"), "description": "Kill it", "followers_count": 688, "friends_count": 419, "statues_count": 8017 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conneaut Lake, PA", "id": "013c1ecd65e6c0c3", "name": "Conneaut Lake", "place_type": "city", "bounding_box": rectangle("-80.31993,41.59426 -80.251004,41.62526") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42039, "countyName": "Crawford", "cityID": 4215744, "cityName": "Conneaut Lake" } }
+{ "create_at": datetime("2016-01-06T10:00:30.000Z"), "id": 684796700406484992, "text": "Marketing Your Blog https://t.co/sJrpeXCFZo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.1182951,40.9818873"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3058328650, "name": "Bright and Happy", "screen_name": "brighthappyplnr", "lang": "en", "location": "San Diego", "create_at": date("2015-02-23"), "description": "Marketing strategist & Virtual Assistant | I help small businesses create marketing strategy plans, automate systems and manage social media.", "followers_count": 119, "friends_count": 55, "statues_count": 2693 }, "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-01-06T10:00:30.000Z"), "id": 684796700540571650, "text": "There comes a point when two people aren't who they use 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": 442276548, "name": "Zae", "screen_name": "ZaeZae20", "lang": "en", "location": "null", "create_at": date("2011-12-20"), "description": "Sí se puede ♡ | Barista |", "followers_count": 2697, "friends_count": 1232, "statues_count": 59083 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Magna, UT", "id": "cdb50c699dd08c61", "name": "Magna", "place_type": "city", "bounding_box": rectangle("-112.140767,40.677067 -112.057613,40.718812") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4947290, "cityName": "Magna" } }
+{ "create_at": datetime("2016-01-06T10:00:31.000Z"), "id": 684796700888727552, "text": "���� SIT ON MY FACE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1597637138, "name": "rob santos", "screen_name": "recluserob", "lang": "en", "location": "Tempe, AZ", "create_at": date("2013-07-15"), "description": "Kobe20", "followers_count": 525, "friends_count": 416, "statues_count": 20368 }, "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-06T10:00:31.000Z"), "id": 684796700909789186, "text": "@J3ff_G a few songs here and there", "in_reply_to_status": 684796331655708672, "in_reply_to_user": 487282551, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 487282551 }}, "user": { "id": 211187520, "name": "Coming Soon", "screen_name": "JasmineRegal", "lang": "en", "location": "null", "create_at": date("2010-11-02"), "description": "fuck with me or fuck off..Egyptian and awesome", "followers_count": 395, "friends_count": 275, "statues_count": 13844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Rutherford, NJ", "id": "df14cd4d61b8e457", "name": "East Rutherford", "place_type": "city", "bounding_box": rectangle("-74.120861,40.788991 -74.061214,40.853386") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3419510, "cityName": "East Rutherford" } }
+{ "create_at": datetime("2016-01-06T10:00:31.000Z"), "id": 684796701568188416, "text": "Bandana round my head like I’m Makaveli", "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": "Maybe Lata Bitch", "screen_name": "Functixn", "lang": "en", "location": "Chicago DTX Cstat", "create_at": date("2012-01-02"), "description": "#GG http://Twitch.tv/richman16 GT XB1: Dynamics nE | Halo F/A hmu to run games | Chicago - DTX - College Station", "followers_count": 1298, "friends_count": 710, "statues_count": 80676 }, "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-06T10:00:31.000Z"), "id": 684796701668950017, "text": "I wonder is anyone new coming to South ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 244248044, "name": "Jan.Thirteen✨", "screen_name": "_SincerelyBreee", "lang": "en", "location": "Montgomery to Mobile", "create_at": date("2011-01-28"), "description": "babygirl❤️ U. of South Alabama", "followers_count": 1851, "friends_count": 908, "statues_count": 49379 }, "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-06T10:00:31.000Z"), "id": 684796701786386432, "text": "$SPX 1 hour to Fed minutes.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 210855604, "name": "Carolyn Marshall", "screen_name": "kiwitrader24", "lang": "en", "location": "Chicago Burbs", "create_at": date("2010-11-01"), "description": "Trading full-time since 2005. Know your levels = make $$. My trades are always front contract weekly options, unless otherwise noted. Tweets reflect that, GL&GT", "followers_count": 2497, "friends_count": 252, "statues_count": 33471 }, "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-06T10:00:31.000Z"), "id": 684796701803085824, "text": "@Aleeeeeezy You have one you want to sell?", "in_reply_to_status": 684795524000555008, "in_reply_to_user": 466801873, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 466801873 }}, "user": { "id": 228222856, "name": "Tom Sawyer", "screen_name": "Nonchxlxnt_", "lang": "en", "location": "-VA-✈-TX-", "create_at": date("2010-12-18"), "description": "Mikey B | My presence is a present kiss my ass", "followers_count": 854, "friends_count": 451, "statues_count": 26350 }, "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-06T10:00:31.000Z"), "id": 684796702021160961, "text": "@sydnitoilolo @TreeYoungLolo I'm pretty sure I totally spelt mamas name wrong but just pronounce it the way I spelt it and it sounds good����", "in_reply_to_status": 684796366984380417, "in_reply_to_user": 887772104, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 887772104, 2437166870 }}, "user": { "id": 3241074842, "name": "JHallaaaaaa", "screen_name": "jazzyhalla", "lang": "en", "location": "null", "create_at": date("2015-06-09"), "description": "null", "followers_count": 47, "friends_count": 37, "statues_count": 598 }, "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-06T10:00:31.000Z"), "id": 684796702482690048, "text": "Krista called me Pooty Tang������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2482077612, "name": "JSmoove", "screen_name": "JordansJordans3", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2014-05-07"), "description": "Workhorse. 2x All-PSAC. 2x All-American. agent67. Grindhouse CEO. KUFB 6⃣7⃣ IG: jordansjordans", "followers_count": 588, "friends_count": 745, "statues_count": 17225 }, "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-06T10:00:31.000Z"), "id": 684796702956457984, "text": "@Mr_Forrest_Pump @TonyDarkMusic be nice.", "in_reply_to_status": 684796185505210368, "in_reply_to_user": 169546931, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 169546931, 132655661 }}, "user": { "id": 37935188, "name": "El Negrito", "screen_name": "say_blk", "lang": "en", "location": "The Waxaholics. ", "create_at": date("2009-05-05"), "description": "“We can't solve problems by using the same kind of thinking we used when we created them.” *waxwednesday|alleykatthursdays|foxhollowsundays|", "followers_count": 2271, "friends_count": 1411, "statues_count": 110019 }, "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-06T10:00:31.000Z"), "id": 684796702969204737, "text": "she didn't fuck with me when i was broke,\nnow i put white girl in white girls nose.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2541067890, "name": "ㅤㅤㅤ", "screen_name": "creepinitreal1", "lang": "en", "location": " ", "create_at": date("2014-06-02"), "description": "⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀", "followers_count": 1026, "friends_count": 99, "statues_count": 10738 }, "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-06T10:00:31.000Z"), "id": 684796702994202624, "text": "@BrittanyJoBrown nice stand ;)", "in_reply_to_status": 684796019700285440, "in_reply_to_user": 69357068, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 69357068 }}, "user": { "id": 14143955, "name": "Rigel St. Pierre", "screen_name": "rigelstpierre", "lang": "en", "location": "Salt Lake City, UT", "create_at": date("2008-03-13"), "description": "Developer of Things @Nuvi. Crafter of Experiences. Listener to music you’ve never heard of. Previously @bandsintown", "followers_count": 914, "friends_count": 567, "statues_count": 20177 }, "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-06T10:00:31.000Z"), "id": 684796703094865920, "text": "Someone play hide & seek 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": 491800773, "name": "Deidra.", "screen_name": "missjones0021", "lang": "en", "location": "somewhere with spence. ", "create_at": date("2012-02-13"), "description": "@PAPPYGAWD55 is the LOML✊❣", "followers_count": 1409, "friends_count": 1055, "statues_count": 22117 }, "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-06T10:00:31.000Z"), "id": 684796703409455104, "text": "My number is the easiest to remember.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 814234574, "name": ".ﻮ", "screen_name": "genesei_", "lang": "en", "location": "661 dafuq b hop off my twitta ", "create_at": date("2012-09-09"), "description": "✨I gave her life; she gave me a reason to live.✨", "followers_count": 1182, "friends_count": 385, "statues_count": 18186 }, "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-06T10:00:31.000Z"), "id": 684796703455576064, "text": "Mahalo for the Hawaii History @DenbyFawcett ~ Malama Pono https://t.co/LtchedZRrO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 330723866 }}, "user": { "id": 390020760, "name": "Kamaile Tripp", "screen_name": "Kamailet801", "lang": "en", "location": "#SLC", "create_at": date("2011-10-13"), "description": "#Kanaka, Asst. Theater Manager @sundancefest #GrandTheatre #SLC, Founder @mailevine801, Hawaiian Teacher @phlearning, Community Outreach @lifestorylib &@utahimf", "followers_count": 1845, "friends_count": 1839, "statues_count": 13458 }, "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-06T10:00:31.000Z"), "id": 684796703585636353, "text": "my nephews getting a snapchat was the best thing that's ever happened ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1265345359, "name": "P8N", "screen_name": "PeytonReedus", "lang": "en", "location": "null", "create_at": date("2013-03-13"), "description": "bio", "followers_count": 580, "friends_count": 343, "statues_count": 3797 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kelso, WA", "id": "3314ad29993667c2", "name": "Kelso", "place_type": "city", "bounding_box": rectangle("-122.924112,46.091154 -122.869522,46.176106") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53015, "countyName": "Cowlitz", "cityID": 5335065, "cityName": "Kelso" } }
+{ "create_at": datetime("2016-01-06T10:00:31.000Z"), "id": 684796703862591488, "text": "I wanna have me bout ten kids, give em a mil each and tell em do what ya daddy did!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2289018352, "name": "Papi Mike", "screen_name": "__yadad", "lang": "en", "location": "Pittsburgh✈️Tampa", "create_at": date("2014-01-17"), "description": "lame asf", "followers_count": 960, "friends_count": 707, "statues_count": 30616 }, "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-01-06T10:00:31.000Z"), "id": 684796703921160192, "text": "Can't have an opinion about nothing nowadays because all people gone do is judge you over it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 378734845, "name": "Fredo.", "screen_name": "saucefredo_", "lang": "en", "location": "Houston/Fort Bend|San Marcos", "create_at": date("2011-09-23"), "description": "8️⃣3️⃣2️⃣ to my city. Texas State University. #eXclusivePromo", "followers_count": 1519, "friends_count": 1135, "statues_count": 35562 }, "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-01-06T10:00:31.000Z"), "id": 684796704185520129, "text": "I love you I love you ���� https://t.co/ImsjDjoILf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 818272555, "name": "Mikoo & Harveyy", "screen_name": "BowDown4DaQueen", "lang": "en", "location": "null", "create_at": date("2012-09-11"), "description": "IG: @Queen_Mikoo & @AyooooHarvey Top Gun Coed 4 Alumni & TGLC Alumni. Lion Heart IOC 5 15-16", "followers_count": 550, "friends_count": 1000, "statues_count": 12275 }, "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-06T10:00:31.000Z"), "id": 684796704282009604, "text": "@darrkaura choke me out chun li https://t.co/pCFSbDwPuQ", "in_reply_to_status": 684796564456345601, "in_reply_to_user": 615010042, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 615010042 }}, "user": { "id": 315910520, "name": "Raichi", "screen_name": "mattheew333", "lang": "en", "location": "null", "create_at": date("2011-06-12"), "description": "shoot your shot", "followers_count": 1052, "friends_count": 481, "statues_count": 94430 }, "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-06T10:00:32.000Z"), "id": 684796704940404736, "text": "The Dodgers are going to keep stockpiling pitchers till they have two rotations...the regular rotation and a spare rotation. Beachy now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1494195762, "name": "Tim Knoch⚾️", "screen_name": "Octim1", "lang": "en", "location": "anaheim, Ca.", "create_at": date("2013-06-08"), "description": "Loyal dodger fan living in Anaheim,surrounded by angel fans. Love dodgers, tweet 90 percent Dodgers. Not a broadcaster, once held Vinnys mike in the booth!", "followers_count": 1426, "friends_count": 1994, "statues_count": 46717 }, "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-06T10:00:32.000Z"), "id": 684796705217318913, "text": "Where can I acquire one of those dinosaur suits https://t.co/jKQizoA5UT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 319510142, "name": "samantha", "screen_name": "_canonized", "lang": "en", "location": "Staten Island, NY", "create_at": date("2011-06-18"), "description": "great things never came from comfort zones\\\\TS", "followers_count": 830, "friends_count": 284, "statues_count": 22094 }, "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-06T10:00:32.000Z"), "id": 684796705376579584, "text": "\" it's okay Megan, you're still cute \"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 62169707, "name": "Megan", "screen_name": "YoursTrulyJanee", "lang": "en", "location": "null", "create_at": date("2009-08-01"), "description": "ur fave heartbreaker", "followers_count": 568, "friends_count": 165, "statues_count": 25990 }, "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-06T10:00:32.000Z"), "id": 684796705653407744, "text": "I told yall it's too damn early for this but everyone is catching fades now shit I'm 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": 60265821, "name": "sadgirl", "screen_name": "astridtaughtyou", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-07-26"), "description": "an overtly sardonic plath enthusiast with a knack for seductive cruelty and a pout stained the violent color of blood.", "followers_count": 511, "friends_count": 687, "statues_count": 23446 }, "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-06T10:00:32.000Z"), "id": 684796705716482048, "text": "It's #winewednesday people. Break up the week with a little #happyhour at #aromaristorante… https://t.co/YXjg7JUHYO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.03296914,33.859433"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "winewednesday", "happyhour", "aromaristorante" }}, "user": { "id": 115795776, "name": "A'Roma Ristorante", "screen_name": "AromaRistorante", "lang": "en", "location": "La Palma, California", "create_at": date("2010-02-19"), "description": "null", "followers_count": 253, "friends_count": 55, "statues_count": 261 }, "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-01-06T10:00:32.000Z"), "id": 684796705825411073, "text": "The @warriors started off the @NBA season well!! (Like they did last season!) And I can IMAGINE how @Zendaya feels about it! #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_mentions": {{ 26270913, 19923144, 154280902 }}, "user": { "id": 2884646961, "name": "Robert S. Overfield", "screen_name": "RSO_3rd", "lang": "en", "location": "California, USA", "create_at": date("2014-11-19"), "description": "My name is Robert S. Overfield III. I have a younger brother and an older sister. I am a High School Graduate (Class of '07) I live at home with my parents.", "followers_count": 87, "friends_count": 342, "statues_count": 2314 }, "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-06T10:00:32.000Z"), "id": 684796705850535937, "text": "#NewYork, NY #Hospitality #Job: QC at Panera Bread https://t.co/EoSKje1K97 #parttime #Veterans #Jobs #Hiring https://t.co/0gfkivmDE8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9854732,40.74771"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NewYork", "Hospitality", "Job", "parttime", "Veterans", "Jobs", "Hiring" }}, "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": 166, "friends_count": 0, "statues_count": 12511 }, "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-06T10:00:32.000Z"), "id": 684796705980715009, "text": "I am not a very smart person", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 53423762, "name": "Natasha", "screen_name": "Kitsune1117", "lang": "en", "location": "Pennsylvania", "create_at": date("2009-07-03"), "description": "null", "followers_count": 85, "friends_count": 158, "statues_count": 2160 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Blawnox, PA", "id": "a2ef4b553f41d2ef", "name": "Blawnox", "place_type": "city", "bounding_box": rectangle("-79.865925,40.485079 -79.851753,40.498257") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4207000, "cityName": "Blawnox" } }
+{ "create_at": datetime("2016-01-06T10:00:32.000Z"), "id": 684796706035089408, "text": "I never fucked oomf thats fucked up if i did fuck she'd be 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": 1051314571, "name": "sussy⛽️", "screen_name": "sussyg_28", "lang": "en", "location": "null", "create_at": date("2012-12-31"), "description": "god is good ❤️ class of 2K16❤️", "followers_count": 2303, "friends_count": 1079, "statues_count": 32807 }, "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-01-06T10:00:32.000Z"), "id": 684796706047705088, "text": "We're #hiring! Read about our latest #job opening here: Student Nurse Apprentice (SNAP) - https://t.co/e02VG8tol0 #RN #Westfield, MA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.7592109,42.11807"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "RN", "Westfield" }}, "user": { "id": 2688167298, "name": "Noble Hospital Jobs", "screen_name": "JobsatNobleHosp", "lang": "en", "location": "Westfield, MA", "create_at": date("2014-07-28"), "description": "Noble Hospital is a regional community hospital, providing a broad range of expert services to Greater Westfield and surrounding areas.", "followers_count": 17, "friends_count": 0, "statues_count": 103 }, "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-01-06T10:00:32.000Z"), "id": 684796706584563712, "text": "@ashley_thora thank you!! Love you❤️❤️", "in_reply_to_status": 684795754817269760, "in_reply_to_user": 3261975036, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3261975036 }}, "user": { "id": 739042854, "name": "Ash", "screen_name": "Ashlynn_Rossi", "lang": "en", "location": "null", "create_at": date("2012-08-05"), "description": "•VHS Senior • Varsity Cheerleader•", "followers_count": 675, "friends_count": 619, "statues_count": 5606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Castaic, CA", "id": "01430bf766a1d670", "name": "Castaic", "place_type": "city", "bounding_box": rectangle("-118.651842,34.440304 -118.610385,34.516522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611796, "cityName": "Castaic" } }
+{ "create_at": datetime("2016-01-06T10:00:32.000Z"), "id": 684796706660179968, "text": "Place you order https://t.co/2X7f7VPbQi @ BLM Boutique https://t.co/t98UxwyUTO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.374512,33.576401"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2193221826, "name": "BLM Boutique", "screen_name": "B031575k", "lang": "en", "location": "6525 Tara blvd Jonesboro GA", "create_at": date("2013-11-13"), "description": "null", "followers_count": 50, "friends_count": 253, "statues_count": 343 }, "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": 13063, "countyName": "Clayton" } }
+{ "create_at": datetime("2016-01-06T10:00:32.000Z"), "id": 684796706668466176, "text": "impressive is the understatement of the year lol https://t.co/eKue3HHzYy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26010290, "name": "JPEG GREG", "screen_name": "GregNoire", "lang": "en", "location": "where the lighting is lowkey", "create_at": date("2009-03-23"), "description": "classic. info@gregnoire.com", "followers_count": 1873, "friends_count": 716, "statues_count": 71839 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-06T10:00:32.000Z"), "id": 684796706672787456, "text": "“@AriesOuotes: #Aries Positive Traits: optimistic, energetic.” Real shit", "in_reply_to_status": 684794813628190720, "in_reply_to_user": 316404475, "favorite_count": 0, "coordinate": point("-82.93281979,42.36421679"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Aries" }}, "user_mentions": {{ 316404475 }}, "user": { "id": 367536865, "name": "Hazy", "screen_name": "Young_HazyHaze", "lang": "en", "location": "E Jefferson", "create_at": date("2011-09-03"), "description": "♈ Aries @LAClippers", "followers_count": 520, "friends_count": 357, "statues_count": 8358 }, "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-06T10:00:32.000Z"), "id": 684796706945396736, "text": "Looking do a new faucet (at @Lowes Home Improvement in Staten Island, NY w/ @goblin1173) https://t.co/P7JeWuBXxQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.2231,40.5483"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19336500, 2547313289 }}, "user": { "id": 27845031, "name": "Angela Haire", "screen_name": "AHaire", "lang": "en", "location": "Palm Coast, FL", "create_at": date("2009-03-31"), "description": "Living Life too fast and loving everyday!", "followers_count": 109, "friends_count": 22, "statues_count": 1772 }, "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-06T10:00:32.000Z"), "id": 684796707003961345, "text": "Going home to dry out, until tomorrow. #ElNino", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ElNino" }}, "user": { "id": 24616657, "name": "Averi Harper", "screen_name": "AveriHarper", "lang": "en", "location": "Oakland, CA", "create_at": date("2009-03-15"), "description": "Reporter @kron4news. @uncmjschool & @columbiajourn alumna. New Yorker in the Wild West.", "followers_count": 1255, "friends_count": 1043, "statues_count": 16010 }, "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-06T10:00:32.000Z"), "id": 684796707272527873, "text": "How do I become successful doing something I love and make a ton of money by tomorrow? #resolutions", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "resolutions" }}, "user": { "id": 123130867, "name": "Alyssa Dupre", "screen_name": "adupre22", "lang": "en", "location": "null", "create_at": date("2010-03-14"), "description": "Information and such", "followers_count": 57, "friends_count": 97, "statues_count": 4428 }, "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-01-06T10:00:32.000Z"), "id": 684796707478044673, "text": "@Chaunceiaaa and you're a hater ����", "in_reply_to_status": 684795521161019393, "in_reply_to_user": 1602736909, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1602736909 }}, "user": { "id": 298286061, "name": "Little Burrito", "screen_name": "nthngbutcheeks_", "lang": "en", "location": "null", "create_at": date("2011-05-13"), "description": "on February 23,2012 a Rose went to heaven. Chicago✈NOLA. #XULA17", "followers_count": 543, "friends_count": 439, "statues_count": 51407 }, "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-06T10:00:32.000Z"), "id": 684796707566153729, "text": "I feel like I at least need one more hour 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": 224354592, "name": "Edgar Allan Hoe", "screen_name": "XIXMCMLXXXV", "lang": "en", "location": "Angel Grove", "create_at": date("2010-12-08"), "description": "Welcome...AND RAQUEL TAKE THAT FUCKING JACKET OFF!! #StopGunViolence", "followers_count": 2191, "friends_count": 629, "statues_count": 64751 }, "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-06T10:00:32.000Z"), "id": 684796707578720257, "text": "@Reince @WSJ how manyof your relatives use Obama care? Evry1 knows u don't have a club", "in_reply_to_status": 684795569714249728, "in_reply_to_user": 20733972, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20733972, 3108351 }}, "user": { "id": 140578500, "name": "gary lee clark 1", "screen_name": "glc1122", "lang": "en", "location": "clermont, fl", "create_at": date("2010-05-05"), "description": "Retired Financial Advisor - Author of The Simplification of Becoming a Millionaire\nAuthor \nActor", "followers_count": 817, "friends_count": 2187, "statues_count": 1997 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rio Pinar, FL", "id": "015d74b454950934", "name": "Rio Pinar", "place_type": "city", "bounding_box": rectangle("-81.276412,28.510068 -81.25239,28.539293") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1260562, "cityName": "Rio Pinar" } }
+{ "create_at": datetime("2016-01-06T10:00:32.000Z"), "id": 684796708266459137, "text": "Worry is the evidence of misplaced faith.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 110601128, "name": "Daniel Chavez", "screen_name": "DanielChavez07", "lang": "en", "location": "Oklahoma City, Oklahoma", "create_at": date("2010-02-01"), "description": "May you always seek adventure, smell like coffee and love like Christ.", "followers_count": 514, "friends_count": 372, "statues_count": 3153 }, "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-01-06T10:00:32.000Z"), "id": 684796708266577921, "text": "Why is weed a drug you have to get tested for to get a good job ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 65458018, "name": "Beautiful Brains", "screen_name": "_iPinkiPromise", "lang": "en", "location": "DETROIT MI ~~ ", "create_at": date("2009-08-13"), "description": "Elegant & Ambitious ASU Alumni #MagnaCumLuade #ASUHornets #teamSCORPIO ;) November_07 #teamGorgeous JESUS is the WAY & the LIGHT!", "followers_count": 542, "friends_count": 487, "statues_count": 17025 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland Park, MI", "id": "cee231d015e7efe2", "name": "Highland Park", "place_type": "city", "bounding_box": rectangle("-83.121913,42.387951 -83.073517,42.418198") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2638180, "cityName": "Highland Park" } }
+{ "create_at": datetime("2016-01-06T10:00:32.000Z"), "id": 684796708555853824, "text": "dropping out https://t.co/hWZ4tgkG8P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 497512937, "name": "MOANica.", "screen_name": "xvoxv_", "lang": "en", "location": "null", "create_at": date("2012-02-19"), "description": "hannah & pizza, pizza & hannah.", "followers_count": 1768, "friends_count": 542, "statues_count": 245623 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-06T10:00:32.000Z"), "id": 684796708568449027, "text": "Deep Cleaning Party! See you all Friday! #basecoatnailsalon #denvernails #nontoxicnails… https://t.co/oJzBtehMyH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.003798,39.762902"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "basecoatnailsalon", "denvernails", "nontoxicnails" }}, "user": { "id": 1631953920, "name": "BASE COAT ™", "screen_name": "basecoatnails", "lang": "en", "location": "Denver, CO", "create_at": date("2013-07-29"), "description": "PURE. CURATED. BEAUTY. A non-toxic nail salon featuring a highly curated selection of natural + organic beauty products in Denver, CO. ✖️✖️", "followers_count": 327, "friends_count": 297, "statues_count": 620 }, "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-06T10:00:32.000Z"), "id": 684796708979539968, "text": "Advantage Solutions #Sales #Job: Event Specialist Part Time Sales at Safeway (#Washington, DC) https://t.co/8maGUoaDTv #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.0363658,38.8951118"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "Job", "Washington", "Jobs", "Hiring" }}, "user": { "id": 20828449, "name": "TMJ - WAS Sales Jobs", "screen_name": "tmj_dc_sales", "lang": "en", "location": "Washington, DC", "create_at": date("2009-02-13"), "description": "Follow this account for geo-targeted Sales job tweets in DC Metro. Need help? Tweet us at @CareerArc!", "followers_count": 465, "friends_count": 315, "statues_count": 361 }, "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-06T10:00:33.000Z"), "id": 684796709285806080, "text": "Magic Kingdom: para as crianças, um delírio. Para os adultos, zzzzzzzz https://t.co/xgCl0q9ZSn @ Disney's… https://t.co/9xyh9CxY1O", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.58059508,28.41515336"), "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 25703891, "name": "Gilberto Scofield", "screen_name": "Gilscofield", "lang": "pt", "location": "Rio de Janeiro, Brasil", "create_at": date("2009-03-21"), "description": "Brazilian content producer, storyteller, digital marketer, married, father. All tweets and retweets are my own, and don't necessarily reflect an endorsement", "followers_count": 4493, "friends_count": 3537, "statues_count": 8040 }, "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-01-06T10:00:33.000Z"), "id": 684796709310873600, "text": "This is a lovehate thing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2832404154, "name": "Kayla Reynaud", "screen_name": "ReynaudKayla", "lang": "en", "location": "ETX || CSTAT ", "create_at": date("2014-09-25"), "description": "null", "followers_count": 150, "friends_count": 141, "statues_count": 503 }, "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-01-06T10:00:33.000Z"), "id": 684796709331812353, "text": "Logistics Associate - Office Depot: (#RapidCity, SD) https://t.co/NKFEoJI7pN #Veterans #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-103.2409678,44.0837027"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RapidCity", "Veterans", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 41686590, "name": "TMJ-SD-USA Jobs", "screen_name": "tmj_sd_usa_jobs", "lang": "en", "location": "South Dakota", "create_at": date("2009-05-21"), "description": "Follow this account for geo-targeted Other job tweets in South Dakota Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 576, "friends_count": 527, "statues_count": 65 }, "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-06T10:00:33.000Z"), "id": 684796709340221441, "text": "Um...lets hope not https://t.co/bPlHjaNUZb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 473556358, "name": "Amy McCain", "screen_name": "aemccain33", "lang": "en", "location": "Dallas, Texas", "create_at": date("2012-01-24"), "description": "Executive Producer - Video, Travel Industry", "followers_count": 116, "friends_count": 564, "statues_count": 370 }, "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-06T10:00:33.000Z"), "id": 684796709365506048, "text": "St Paul, MN: Golly gee wilikers, it's 29ºF and mostly cloudy.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.1,44.95"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3195747619, "name": "Weather By Brian", "screen_name": "WeatherByBrian", "lang": "en", "location": "null", "create_at": date("2015-05-14"), "description": "Tweets the weather about wherever Brian last tweets.\n\nRun by @bman4789", "followers_count": 12, "friends_count": 1, "statues_count": 1614 }, "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.89076 -93.003514,44.992279") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27123, "countyName": "Ramsey", "cityID": 2758000, "cityName": "St. Paul" } }
+{ "create_at": datetime("2016-01-06T10:00:33.000Z"), "id": 684796709369704449, "text": "I'm at Raider Café @ Skyline High School in Dallas, TX https://t.co/zCA2lJkRmv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.69066332,32.78188988"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 58947484, "name": "Brian Hennig", "screen_name": "NrthTxsGuy", "lang": "en", "location": "Mesquite", "create_at": date("2009-07-21"), "description": "Teacher, sci-fi geek, debater, doctoral student. So...when am I supposed to start feeling old?", "followers_count": 145, "friends_count": 432, "statues_count": 1298 }, "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-06T10:00:33.000Z"), "id": 684796709558329344, "text": "Cuh bet not throw a bunch of weak ass songs we already heard on there https://t.co/V2C14i3tse", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1146518995, "name": "wash jr.", "screen_name": "H2rold", "lang": "en", "location": "N$DP", "create_at": date("2013-02-03"), "description": "#aNYthiNG", "followers_count": 1151, "friends_count": 586, "statues_count": 15845 }, "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-06T10:00:33.000Z"), "id": 684796709856231429, "text": "Sharing a bowl of cereal with my son is the best way to bond #ReesePuffs ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ReesePuffs" }}, "user": { "id": 344074728, "name": "Christi Houx", "screen_name": "ChristiHoux", "lang": "en", "location": "Fort Wayne, IN", "create_at": date("2011-07-28"), "description": "I am an Account Manager for Aarons. I love my family and enjoying the small things in life.", "followers_count": 34, "friends_count": 110, "statues_count": 468 }, "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-06T10:00:33.000Z"), "id": 684796710003052546, "text": "What is @getstolen ? Shit.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3255100813 }}, "user": { "id": 48790585, "name": "ShutUpOkan", "screen_name": "OkanIsAllNow", "lang": "en", "location": "null", "create_at": date("2009-06-19"), "description": "Probably pretending to know what i'm doing at any given time| Marvel Fanatic| Working on DC| Huskies are life| Garbage is forever garbage", "followers_count": 9896, "friends_count": 429, "statues_count": 16790 }, "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-01-06T10:00:33.000Z"), "id": 684796711089377280, "text": "Whhhaaaaatt?!? https://t.co/M7QXK0t7vX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15419764, "name": "Joe Craven", "screen_name": "imacraven", "lang": "en", "location": "AN", "create_at": date("2008-07-13"), "description": "yeah... life is pretty damn good", "followers_count": 159, "friends_count": 314, "statues_count": 3394 }, "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-06T10:00:33.000Z"), "id": 684796711164887041, "text": "Lightly hoppy & refreshing. - Drinking a Grisette by @ManorHillBrew at @scottk1969 — https://t.co/vcUwjKIHtH #photo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.4132,38.3672"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "photo" }}, "user_mentions": {{ 2322594446, 349676946 }}, "user": { "id": 349676946, "name": "Scott Kettler", "screen_name": "scottk1969", "lang": "en", "location": "Lusby, Maryland", "create_at": date("2011-08-06"), "description": "I'm a Nuclear Reactor Operator. I love Beer, Hokie Football, and the KC Royals & Chiefs.", "followers_count": 111, "friends_count": 413, "statues_count": 2189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chesapeake Ranch Estates, MD", "id": "4dd2cff80a3e3636", "name": "Chesapeake Ranch Estates", "place_type": "city", "bounding_box": rectangle("-76.451763,38.319092 -76.380792,38.387749") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24009, "countyName": "Calvert", "cityID": 2416050, "cityName": "Chesapeake Ranch Estates" } }
+{ "create_at": datetime("2016-01-06T10:00:33.000Z"), "id": 684796711240257537, "text": "Was that this season? AJ looks like a child �� #Bengals https://t.co/FLhvtwUGPW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Bengals" }}, "user": { "id": 967123674, "name": "Christopher Ortiz", "screen_name": "OrtizzleSD", "lang": "en", "location": "Point Loma, CA", "create_at": date("2012-11-23"), "description": "Love my lady, family, warm friends, cold drinks, the beach, playing soccer, riding bikes, crosswords and all things sports. ⚡️GO CHARGERS⚡️Viva Mexico", "followers_count": 93, "friends_count": 145, "statues_count": 2971 }, "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-06T10:00:33.000Z"), "id": 684796711932420097, "text": "I literally can't be put in a bad mood today ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 33719221, "name": "Big Red (Ariel)", "screen_name": "ChelaDelBae", "lang": "en", "location": "your boyfriend's house", "create_at": date("2009-04-20"), "description": "Pizza, iced coffee, & brunch obsessed. Social media guru. Time magazine's person of the year 2006. I was born 21 years ago, it was awful. Call me captain.", "followers_count": 931, "friends_count": 1096, "statues_count": 50459 }, "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-06T10:00:33.000Z"), "id": 684796711991001088, "text": "More like Oculus RIP.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 147090745, "name": "Tim E. Kish!", "screen_name": "kimtish", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-05-22"), "description": "YouTube Opinion Haver | Standup | big dork\n\nsnap: timekish1", "followers_count": 1346, "friends_count": 479, "statues_count": 10391 }, "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-06T10:00:33.000Z"), "id": 684796712137789440, "text": "Last couple days been 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": 3000504763, "name": "Cubita", "screen_name": "UbahinaUber", "lang": "en", "location": "713/225", "create_at": date("2015-01-28"), "description": "posted like a Mf. / Southern Marsh professor", "followers_count": 354, "friends_count": 735, "statues_count": 8067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-06T10:00:33.000Z"), "id": 684796712267825152, "text": "@TROBwhorley10 ��", "in_reply_to_status": 684619639779889152, "in_reply_to_user": 857248122, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 857248122 }}, "user": { "id": 2581304357, "name": "Francesca", "screen_name": "jakewayf", "lang": "en", "location": "Michigan State University", "create_at": date("2014-06-03"), "description": "to each their own", "followers_count": 338, "friends_count": 163, "statues_count": 5459 }, "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-06T10:00:33.000Z"), "id": 684796712305750016, "text": "If I was tweeting about you then I would have @ you but I didn't so don't assume things ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3951451873, "name": "Kylee Janae", "screen_name": "kylee_cx", "lang": "en", "location": "null", "create_at": date("2015-10-19"), "description": "Zac❤️", "followers_count": 78, "friends_count": 135, "statues_count": 894 }, "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-06T10:00:33.000Z"), "id": 684796712674811904, "text": "Yo this is happening. I'm already too hype now for it not to happen.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 236066633, "name": "LadyDavine", "screen_name": "LadyDavine_DBN", "lang": "en", "location": "null", "create_at": date("2011-01-09"), "description": "Who's that peaking in my window? Nobody cuz I live in a penthouse. Davine By Nature #HeGotNext", "followers_count": 62, "friends_count": 181, "statues_count": 3237 }, "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-06T10:00:33.000Z"), "id": 684796712779563008, "text": "oms ���� https://t.co/6yj0SYH5O1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4445894234, "name": "Montana of 300' Jr", "screen_name": "delmillio", "lang": "en", "location": "pg", "create_at": date("2015-12-03"), "description": "null", "followers_count": 39, "friends_count": 62, "statues_count": 30 }, "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-06T10:00:33.000Z"), "id": 684796712943157248, "text": "I was waiting for the aftershock TBH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1124016127, "name": "Pretty Flacko", "screen_name": "King_Dejavu8", "lang": "en", "location": "562✈️916", "create_at": date("2013-01-26"), "description": "Jeremiah 29:11 // ACTOR // SC: freshkiiid //SACRAMENTO STATE FOOTBALL ||VM ALUMNI|| The 9ine", "followers_count": 932, "friends_count": 789, "statues_count": 12030 }, "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-06T10:00:33.000Z"), "id": 684796713018736640, "text": "Las fotos de chiabes que mandaron a sacar de la Asamblea están llorando amoníaco.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 40385239, "name": "José Rafael Silva", "screen_name": "jSILVAe", "lang": "en", "location": "United States", "create_at": date("2009-05-15"), "description": "Soy de San Juan de los Morros, Guárico. Vivo en Chicago, Illinois.", "followers_count": 746, "friends_count": 545, "statues_count": 35761 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Chicago, IL", "id": "772f1cfac1076a67", "name": "North Chicago", "place_type": "city", "bounding_box": rectangle("-87.903832,42.2877 -87.825943,42.341453") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1753559, "cityName": "North Chicago" } }
+{ "create_at": datetime("2016-01-06T10:00:33.000Z"), "id": 684796713031184384, "text": "@me0wcarol Yea lol ��", "in_reply_to_status": 684793786468192256, "in_reply_to_user": 104660824, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 104660824 }}, "user": { "id": 371893240, "name": "A.J. Dominguez", "screen_name": "AdanJose_", "lang": "en", "location": "Weslaco, Texas", "create_at": date("2011-09-11"), "description": "Weslaco East High School | Class of 2016 #TAMU20", "followers_count": 1350, "friends_count": 840, "statues_count": 7120 }, "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-06T10:00:34.000Z"), "id": 684796713522081792, "text": "Can she please �� https://t.co/DQEQLNce3Y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2184781852, "name": "Pretty Kee", "screen_name": "Keesoprettyyy", "lang": "en", "location": "null", "create_at": date("2013-11-17"), "description": "Parts of my are in Heaven 10.13.15 Remy you changed my life forever G4ever", "followers_count": 351, "friends_count": 340, "statues_count": 15032 }, "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.022437") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2407125, "cityName": "Bethesda" } }
+{ "create_at": datetime("2016-01-06T10:00:34.000Z"), "id": 684796713694031873, "text": "WIN! Join Carolyn on the 40 Days to Personal Revolution journey! Our Marketing Director is… https://t.co/PuVoOI5duH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.6589056,38.25223503"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19392538, "name": "Rainbow Blossom", "screen_name": "rainbowblossom", "lang": "en", "location": "Louisville, Kentucky", "create_at": date("2009-01-23"), "description": "Keeping the 'Ville healthy since 1977, we are Louisville's original health food store!", "followers_count": 2596, "friends_count": 1672, "statues_count": 2617 }, "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-01-06T10:00:34.000Z"), "id": 684796713719091200, "text": "I'm not just beautiful I'm a work of art", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 513920338, "name": "hilary bank$", "screen_name": "_myrakaye", "lang": "en", "location": "okc", "create_at": date("2012-03-03"), "description": "Put yourself first.", "followers_count": 12823, "friends_count": 1256, "statues_count": 18324 }, "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-06T10:00:34.000Z"), "id": 684796713740140545, "text": "1/ \"major startup opportunities: now that > 2B ppl have internet... create systems to efficiently share & store their collective knowledge\"", "in_reply_to_status": 684796355236200449, "in_reply_to_user": 3013702747, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3013702747, "name": "Anthony Bardaro", "screen_name": "AnthPB", "lang": "en", "location": "Boston, MA", "create_at": date("2015-02-08"), "description": "Cuz I never wanna be the thing that you hear many say //\nLike I could've been, I should've been, but never did it anyway", "followers_count": 66, "friends_count": 78, "statues_count": 2470 }, "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-06T10:00:34.000Z"), "id": 684796713849110528, "text": "Estar solita sin nada q hacer y sin compu :(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 153234130, "name": "Manene", "screen_name": "madecortesg", "lang": "es", "location": "Utah, USA", "create_at": date("2010-06-07"), "description": "Un Día a la Vez♡.... Summer 1993. Insta: madecortesg", "followers_count": 684, "friends_count": 514, "statues_count": 67521 }, "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-06T10:00:34.000Z"), "id": 684796714033659904, "text": "Nah Sis we good https://t.co/Wc410JvUdn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 100590309, "name": "Trell", "screen_name": "_trellmixx", "lang": "en", "location": "Saint Louis , MO ", "create_at": date("2009-12-30"), "description": "Broken Crayons Still Color", "followers_count": 1684, "friends_count": 503, "statues_count": 104768 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ferguson, MO", "id": "0a62ce0f6aa37536", "name": "Ferguson", "place_type": "city", "bounding_box": rectangle("-90.319108,38.725028 -90.234517,38.785786") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2923986, "cityName": "Ferguson" } }
+{ "create_at": datetime("2016-01-06T10:00:34.000Z"), "id": 684796714557898752, "text": "@djcomeback Hot R&B Alert!! Download @OfficialCJBanks mixtape \"Est95\" via #Datpiff --->> https://t.co/nxR2oLDzSk #TheBanksBrand #music", "in_reply_to_status": -1, "in_reply_to_user": 38100736, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Datpiff", "TheBanksBrand", "music" }}, "user_mentions": {{ 38100736, 3956938515 }}, "user": { "id": 174894240, "name": "C. L. Banks Sr.", "screen_name": "IamCoreyDrumz", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-08-04"), "description": "Father/Manager of r&b artist and writer CJ Banks (@officialcjbanks). Legendary Hollis Queens hip-hop artist CeStyle aka #IKEBoy Corey Drumz. #TheBanksBrand #TTO", "followers_count": 2416, "friends_count": 1735, "statues_count": 69123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:34.000Z"), "id": 684796714834776065, "text": "@astrotreks how was it? What are you going to see today #AiaaSciTech #gochargers", "in_reply_to_status": 684782714306744321, "in_reply_to_user": 28432593, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AiaaSciTech", "gochargers" }}, "user_mentions": {{ 28432593 }}, "user": { "id": 2511038065, "name": "Nettie Roozeboom", "screen_name": "NettieRooz", "lang": "en", "location": "Mountain View, CA", "create_at": date("2014-05-20"), "description": "Aerospace Engineer at NASA Ames Research Center - #WomanInStem\nCurriculum Developer - @ThePocketLab", "followers_count": 209, "friends_count": 286, "statues_count": 474 }, "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-06T10:00:34.000Z"), "id": 684796715115778048, "text": "@_IAintShitSon lmao !", "in_reply_to_status": 684766964061999104, "in_reply_to_user": 257276567, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 257276567 }}, "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": 22683, "friends_count": 5275, "statues_count": 387152 }, "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-06T10:00:34.000Z"), "id": 684796715363340292, "text": "@bungdan you don't seem to understand what the words smug, cheap, political, and mockery mean. Nor do you have any defense to your statement", "in_reply_to_status": 684796432138776576, "in_reply_to_user": 424699769, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 80669530 }}, "user": { "id": 424699769, "name": "Michael A. Coughlin", "screen_name": "ABereanOne", "lang": "en", "location": "Pickerington, OH", "create_at": date("2011-11-29"), "description": "Degenerate sinner, regenerate saint by the grace of God through our Lord Jesus Christ.", "followers_count": 535, "friends_count": 166, "statues_count": 12183 }, "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-06T10:00:34.000Z"), "id": 684796715526914048, "text": "Second #MyOwnBusinessChallenge class of the week and we’re getting ready for students to #pitch us their #businesses https://t.co/6Se4WxViA9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.79328101,40.7328839"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MyOwnBusinessChallenge", "pitch", "businesses" }}, "user": { "id": 245642727, "name": "Youthful Savings", "screen_name": "YouthfulSavings", "lang": "en", "location": "New York, New York", "create_at": date("2011-01-31"), "description": "#EdTech company dedicated to empowering the next generation with financial education and entrepreneurship training through curriculum, technology and community", "followers_count": 1867, "friends_count": 245, "statues_count": 1557 }, "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-06T10:00:34.000Z"), "id": 684796715614928896, "text": "Good afternoon Columbus! Today's sunset will be at 5:22 pm. Here is the top news at this moment. https://t.co/yRBmvJYtMT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.99527267,39.96378925"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 63964871, "name": "Columbus CP", "screen_name": "ColumbusCP", "lang": "en", "location": "Columbus, OH", "create_at": date("2009-08-08"), "description": "Columbus City And Press", "followers_count": 1458, "friends_count": 9, "statues_count": 111644 }, "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-06T10:00:34.000Z"), "id": 684796715619110912, "text": "@I_hearttwatter @_LyssaNelson remember when we had the GM count down till her Bday lmao", "in_reply_to_status": 684796559708430336, "in_reply_to_user": 418328134, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 418328134, 723258572 }}, "user": { "id": 520475566, "name": "Donny Thompson", "screen_name": "Thompson_Era", "lang": "en", "location": "2✌5✋1☝ ALABAMA ", "create_at": date("2012-03-10"), "description": "GET'EM Thompson |CHICAGO|#SouthernDominance #FightLife [̲̅$̲̅(̲̅1̲̅00)̲̅$̲̅]MoneyTrain[̲̅$̲̅(̲̅1̲̅00)̲̅$̲̅]", "followers_count": 5938, "friends_count": 978, "statues_count": 56503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gulf Shores, AL", "id": "dd4fb0ab4e6cd837", "name": "Gulf Shores", "place_type": "city", "bounding_box": rectangle("-87.753372,30.239427 -87.659759,30.334228") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1003, "countyName": "Baldwin", "cityID": 132272, "cityName": "Gulf Shores" } }
+{ "create_at": datetime("2016-01-06T10:00:34.000Z"), "id": 684796715778633729, "text": "Girls Take Their Bestfriends To The Extreme To Where They Can't Fight Their Own Battles Because They Need They Bestfriend ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1064566350, "name": "✨Yamina✨", "screen_name": "Yamina___", "lang": "en", "location": "null", "create_at": date("2013-01-05"), "description": "CT Sophomore Instagram @yamina____", "followers_count": 737, "friends_count": 527, "statues_count": 2763 }, "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-06T10:00:34.000Z"), "id": 684796715828920320, "text": "@hiphophotshot nah change ya name kid u lame", "in_reply_to_status": 684796203431817216, "in_reply_to_user": 4726757543, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4726757543 }}, "user": { "id": 2354914670, "name": "DZNTZ", "screen_name": "GanjaManMark", "lang": "en", "location": "null", "create_at": date("2014-02-21"), "description": "#ETG #VerbalAssault #HHH #RawEmpire #AunthenticMobb gang gagang gang gang", "followers_count": 389, "friends_count": 147, "statues_count": 12269 }, "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-06T10:00:34.000Z"), "id": 684796716449574912, "text": "Fuck what people say. If that person makes you happy be with them. Because at the end of the day your friends aren't gonna give you head. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3251064488, "name": "aims", "screen_name": "aimjonessss", "lang": "en", "location": "southern oregon", "create_at": date("2015-06-20"), "description": "savage attitude with a heart of gold", "followers_count": 301, "friends_count": 250, "statues_count": 2078 }, "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-06T10:00:34.000Z"), "id": 684796716802007042, "text": "@wwwbigbaldhead Happy birthday. Your my favorite on the walking dead ���� hope you have a good day!", "in_reply_to_status": -1, "in_reply_to_user": 25460615, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25460615 }}, "user": { "id": 2562590851, "name": "Samantha", "screen_name": "sammiers17", "lang": "en", "location": "null", "create_at": date("2014-06-11"), "description": "I love Bethany Mota and YouTubers :) beth followed 12~19~14 \n fashion is my life style 5sauce \nYOUNOW -Vince,Ted", "followers_count": 374, "friends_count": 677, "statues_count": 5593 }, "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-01-06T10:00:34.000Z"), "id": 684796716806213632, "text": "@cwalt_9 it was just decided we're going to Paoli Saturday", "in_reply_to_status": 684795774874578946, "in_reply_to_user": 2448043946, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2269495417 }}, "user": { "id": 2448043946, "name": "Kaytie Miller", "screen_name": "SullivanKaytie", "lang": "en", "location": "Mamaw-12/19/14 ❤️", "create_at": date("2014-04-16"), "description": "Be fearless in the pursuit of what sets your soul on fire. @cmmiller81 & our sweet Parker have my heart! 2 Corinthians 5:7", "followers_count": 348, "friends_count": 151, "statues_count": 1152 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elizabethtown, KY", "id": "998c5b6035f9b5a4", "name": "Elizabethtown", "place_type": "city", "bounding_box": rectangle("-85.952048,37.626084 -85.789831,37.771861") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21093, "countyName": "Hardin", "cityID": 2124274, "cityName": "Elizabethtown" } }
+{ "create_at": datetime("2016-01-06T10:00:35.000Z"), "id": 684796717934362625, "text": "Wow I just passed 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": 2800923349, "name": "kaylee", "screen_name": "kaylee_laverty", "lang": "en", "location": "RIP Cam, Texas", "create_at": date("2014-09-09"), "description": "I'll probably work at sonic for the rest of my life", "followers_count": 530, "friends_count": 415, "statues_count": 7491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aledo, TX", "id": "010891b30e635705", "name": "Aledo", "place_type": "city", "bounding_box": rectangle("-97.666435,32.670028 -97.592444,32.703829") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48367, "countyName": "Parker", "cityID": 4801744, "cityName": "Aledo" } }
+{ "create_at": datetime("2016-01-06T10:00:35.000Z"), "id": 684796717963726849, "text": "Every morning I'll spend about 10 min going through the pictures and videos on my phone\nAnd today it made me miss my sisters. ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 143327155, "name": "↠bianca♡", "screen_name": "_miss__bee", "lang": "en", "location": "City of Trees", "create_at": date("2010-05-12"), "description": "➵ ➵ Keep smiling, the world is watching. ♡ ♫ ☾ IG&Snap: _miss__bee", "followers_count": 337, "friends_count": 241, "statues_count": 15394 }, "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-06T10:00:35.000Z"), "id": 684796718135808001, "text": "@RyanWillbert @NorthDakotaHemp @samanthhapaigee great house warming gift", "in_reply_to_status": 670134154651181056, "in_reply_to_user": 1539418496, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1539418496, 2281715252, 1237002348 }}, "user": { "id": 2276638962, "name": "Free Trend Alerts", "screen_name": "Blackboxtrends", "lang": "en", "location": "null", "create_at": date("2014-01-04"), "description": "TUNA Followers, Algorithm Based Trend Alerts, 0 Fundamentals, Black Box System Development, MENTAL HEALTH WEED ADVOCATE! IRISH 1! HULKAMANIAC SINCE I WAS 14", "followers_count": 9765, "friends_count": 7087, "statues_count": 19531 }, "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-06T10:00:35.000Z"), "id": 684796718357950464, "text": "Damn Janet ... Hope all is well .", "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 ", "screen_name": "tsolete", "lang": "en", "location": "null", "create_at": date("2009-03-24"), "description": "#HangOverBrunch figuring shit out.", "followers_count": 1354, "friends_count": 675, "statues_count": 125864 }, "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-06T10:00:35.000Z"), "id": 684796718454575106, "text": "my calculus teacher has just informed us she has a meme generator", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2424895091, "name": "jom", "screen_name": "joncopes", "lang": "en", "location": "Raleigh, NC", "create_at": date("2014-03-20"), "description": "stay at home dad", "followers_count": 158, "friends_count": 120, "statues_count": 7962 }, "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-06T10:00:35.000Z"), "id": 684796718731235328, "text": "@FknAngie damn g indeed. Good to see you back on the tweety though.", "in_reply_to_status": 684796375888838656, "in_reply_to_user": 108503408, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 108503408 }}, "user": { "id": 489844299, "name": "Geoff", "screen_name": "TheGiantsFan707", "lang": "en", "location": "Fairfield, CA", "create_at": date("2012-02-11"), "description": "San Francisco Giants. Food. Trying to stay faithful to the 49ers. View my attempts at photography!! http://flickr.com/photos/geoffeyeyellow", "followers_count": 266, "friends_count": 257, "statues_count": 51845 }, "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-01-06T10:00:35.000Z"), "id": 684796718752206848, "text": "@panchorgl si lazaro presenta pruebas, haria de todo para lo protegieran por testigo", "in_reply_to_status": 684796621691981825, "in_reply_to_user": 7771192, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 7771192 }}, "user": { "id": 60418470, "name": "Pau", "screen_name": "paulyn1979_", "lang": "en", "location": "Buenos Aires", "create_at": date("2009-07-26"), "description": "Cultivadora del blanco verde, socia del team invierno.", "followers_count": 1416, "friends_count": 402, "statues_count": 116998 }, "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-01-06T10:00:35.000Z"), "id": 684796718941089792, "text": "Yes. https://t.co/LkB2YxTyQ8 https://t.co/7pLtI2X4tO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 29995316, "name": "Donald Drew", "screen_name": "TheDonDrew", "lang": "en", "location": "Tri State Area (NY, NJ, CT)", "create_at": date("2009-04-09"), "description": "Host of #TheSitdown, an internet talk show named as 1 of @ComplexMag's Top 25 Most Exciting Things in Sneakers Right Now. Email thedondrew@me.com.", "followers_count": 8302, "friends_count": 125, "statues_count": 51608 }, "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-06T10:00:35.000Z"), "id": 684796718970306560, "text": "Discounts, DJs, Strollers, and Hurricanes Pride at the Ipreo Ice Rink: #RaleighOnIce https://t.co/gi3wimudx4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.6406859,35.77414893"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RaleighOnIce" }}, "user": { "id": 21774716, "name": "Marti Hampton", "screen_name": "Martihampton", "lang": "en", "location": "Raleigh, NC", "create_at": date("2009-02-24"), "description": "With over 37 years of experience in the Triangle real estate market, you can trust the Expert... #1 RE/MAX Office in The Carolinas... http://t.co/PENpIQvXLj", "followers_count": 671, "friends_count": 241, "statues_count": 4986 }, "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-06T10:00:35.000Z"), "id": 684796719280861184, "text": "#WCW goes to @katyperry in that Plump mascara commercial. �� mmmmmm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WCW" }}, "user_mentions": {{ 21447363 }}, "user": { "id": 65819113, "name": "A̤̮N̤̮D̤̮R̤̮E̤̮A̤̮", "screen_name": "Aaandreeeea", "lang": "en", "location": "Grand Rapids, MI", "create_at": date("2009-08-14"), "description": "null", "followers_count": 251, "friends_count": 368, "statues_count": 9819 }, "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-06T10:00:35.000Z"), "id": 684796719440113665, "text": "@belledejour_uk @jessalyn do most not deal with young people? And define young. :P I'm in my mid 30s.", "in_reply_to_status": 684796559255457792, "in_reply_to_user": 13857342, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 46059944, 8223662 }}, "user": { "id": 13857342, "name": "Randi Lee Harper", "screen_name": "randileeharper", "lang": "en", "location": "Portland, OR", "create_at": date("2008-02-22"), "description": "Author, @ggautoblocker. Founder, Online Abuse Prevention Initiative. https://t.co/okBqI4N0Pg https://t.co/zadlXVRLW5 randi@onlineabuseprevention.org", "followers_count": 18439, "friends_count": 703, "statues_count": 89007 }, "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-06T10:00:35.000Z"), "id": 684796719595274240, "text": "@ManOnTheMoon56 I hate him so much", "in_reply_to_status": 684659740098428928, "in_reply_to_user": 424717794, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 424717794 }}, "user": { "id": 615853814, "name": "ARELI ❄️", "screen_name": "leli_97", "lang": "en", "location": "60120•60607", "create_at": date("2012-06-22"), "description": "• University of Illinois at Chicago • ♋️", "followers_count": 352, "friends_count": 589, "statues_count": 8549 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clinton, OK", "id": "87852f01543c6d86", "name": "Clinton", "place_type": "city", "bounding_box": rectangle("-99.008105,35.467049 -98.956102,35.52699") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40039, "countyName": "Custer", "cityID": 4015400, "cityName": "Clinton" } }
+{ "create_at": datetime("2016-01-06T10:00:35.000Z"), "id": 684796719872249856, "text": "Pillar of Fire is hiring! Custodian #jobs in EAST MILLSTONE Apply today https://t.co/0Fzmwinw0G", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.525118,40.49887"), "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": 169, "friends_count": 42, "statues_count": 72230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Middlebush, NJ", "id": "007bb8ec7f04b29e", "name": "Middlebush", "place_type": "city", "bounding_box": rectangle("-74.560655,40.484075 -74.51619,40.513671") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34035, "countyName": "Somerset", "cityID": 3445870, "cityName": "Middlebush" } }
+{ "create_at": datetime("2016-01-06T10:00:35.000Z"), "id": 684796720098750464, "text": "#OleUglyAssNigga https://t.co/DfHRUNu4HV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "OleUglyAssNigga" }}, "user": { "id": 579464322, "name": "#1k", "screen_name": "TaySmith_7", "lang": "en", "location": "South Carolina, USA", "create_at": date("2012-05-13"), "description": "11/21❤ | SC:Basedgodtay", "followers_count": 660, "friends_count": 524, "statues_count": 18987 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rock Hill, SC", "id": "00696a56e1f9f443", "name": "Rock Hill", "place_type": "city", "bounding_box": rectangle("-81.086454,34.869151 -80.943098,34.961986") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45091, "countyName": "York", "cityID": 4561405, "cityName": "Rock Hill" } }
+{ "create_at": datetime("2016-01-06T10:00:35.000Z"), "id": 684796720169943042, "text": "I wish I had a twin, so we could pull funny pranks", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1263134275, "name": "Hayley Perry", "screen_name": "HayleyPerryyy", "lang": "en", "location": "Anchorage, AK", "create_at": date("2013-03-12"), "description": "The truth is... I'm Hannah Montana", "followers_count": 349, "friends_count": 506, "statues_count": 7223 }, "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-06T10:00:35.000Z"), "id": 684796720744656896, "text": "Looking at my timehop....like dammmmnnnn I used to be crazy lmfao. U don't realize how much u mature until you see it for yourself.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 416672534, "name": "Russian Princess♊➰", "screen_name": "KrystaMilo", "lang": "en", "location": "null", "create_at": date("2011-11-19"), "description": "Pray as if it's all up to God, work as if it's all up to you.", "followers_count": 555, "friends_count": 553, "statues_count": 12984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morris, IL", "id": "3424f1874011f58b", "name": "Morris", "place_type": "city", "bounding_box": rectangle("-88.473214,41.350194 -88.383605,41.408684") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17063, "countyName": "Grundy", "cityID": 1750491, "cityName": "Morris" } }
+{ "create_at": datetime("2016-01-06T10:00:35.000Z"), "id": 684796721101012992, "text": "These hoes give head & bk texting they nifgka. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3866395814, "name": "323", "screen_name": "crenshawkidd", "lang": "en", "location": "South Central, CA", "create_at": date("2015-10-11"), "description": "null", "followers_count": 1572, "friends_count": 198, "statues_count": 1541 }, "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-06T10:00:35.000Z"), "id": 684796721457664000, "text": "@SubiriC - very kind...�� back 2 U like a boomerang. ��", "in_reply_to_status": 684783311642832896, "in_reply_to_user": 2873636344, "favorite_count": 0, "coordinate": point("-118.470914,34.2131217"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2873636344 }}, "user": { "id": 2929078309, "name": "Lisa Johnson", "screen_name": "LisaTruthJohns", "lang": "en", "location": "null", "create_at": date("2014-12-13"), "description": "Motivational Truth Speaker & Certified Hypnotherapist - Inspiring others to live their best life through the renewing of the mind.", "followers_count": 2617, "friends_count": 2275, "statues_count": 2557 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:36.000Z"), "id": 684796721814179841, "text": "@BaileePeek of course ���� 912282**** ��", "in_reply_to_status": 684796524241547264, "in_reply_to_user": 4197062253, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4197062253 }}, "user": { "id": 1122500612, "name": "Tionne ✨", "screen_name": "ReNesiaB", "lang": "en", "location": "CCHS", "create_at": date("2013-01-26"), "description": "❤ R.I.P: Kyn & Karter , Ma Fudge ❤ Snapchat: renesia_tionne ❤ Erriana Celaina ❤", "followers_count": 1222, "friends_count": 1060, "statues_count": 24683 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Homerville, GA", "id": "36554a901ec8739c", "name": "Homerville", "place_type": "city", "bounding_box": rectangle("-82.779847,31.021698 -82.735062,31.060961") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13065, "countyName": "Clinch", "cityID": 1339748, "cityName": "Homerville" } }
+{ "create_at": datetime("2016-01-06T10:00:36.000Z"), "id": 684796721986023428, "text": "When none of my friends wanna go to Moses 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": 346928237, "name": "selena ▽", "screen_name": "SelenaLanderos", "lang": "en", "location": "null", "create_at": date("2011-08-01"), "description": "18 | Warden ↠ WSU Pullman | Romans 8:39 ❁ † ▽ insta: selenalanderos", "followers_count": 585, "friends_count": 414, "statues_count": 18475 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warden, WA", "id": "002bc58663e35858", "name": "Warden", "place_type": "city", "bounding_box": rectangle("-119.055378,46.955228 -119.026877,46.977058") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53025, "countyName": "Grant", "cityID": 5376160, "cityName": "Warden" } }
+{ "create_at": datetime("2016-01-06T10:00:36.000Z"), "id": 684796722250305537, "text": "@itzel6ix @Yaboii_ST3V3 @_youungking if only I had a car", "in_reply_to_status": 684543014564278273, "in_reply_to_user": 633354086, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 633354086, 1147165386, 3158926132 }}, "user": { "id": 2293503906, "name": ".The Great.", "screen_name": "_Deesuz", "lang": "en", "location": "null", "create_at": date("2014-01-15"), "description": "711 ¶∆©k ~I play golf and soccer~", "followers_count": 92, "friends_count": 120, "statues_count": 1780 }, "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-06T10:00:36.000Z"), "id": 684796722321555456, "text": "@ukeetweeted even better����", "in_reply_to_status": 684796513805942785, "in_reply_to_user": 156285649, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 156285649 }}, "user": { "id": 305373315, "name": "☹☹☹", "screen_name": "KingGeorgee", "lang": "en", "location": "Dreamville, Tx", "create_at": date("2011-05-25"), "description": "null", "followers_count": 367, "friends_count": 331, "statues_count": 26553 }, "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-06T10:00:36.000Z"), "id": 684796722334281732, "text": "SKILL. https://t.co/9J0JD9i2im", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "sv", "is_retweet": false, "user": { "id": 3914586802, "name": "Rae", "screen_name": "rachelecki", "lang": "en", "location": "null", "create_at": date("2015-10-09"), "description": "xxi. happy ツ. healthy ☤. driven ♡.", "followers_count": 103, "friends_count": 140, "statues_count": 613 }, "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-01-06T10:00:36.000Z"), "id": 684796722447577089, "text": "NATIONAL DUST C is hiring! DRIVER / ROUTE #jobs in MIDDLESEX Apply today https://t.co/gWiQNNoyE2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.498354,40.574029"), "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": 169, "friends_count": 42, "statues_count": 72231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Middlesex, NJ", "id": "9e9d157857a4205c", "name": "Middlesex", "place_type": "city", "bounding_box": rectangle("-74.527044,40.555399 -74.469693,40.59266") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34035, "countyName": "Somerset", "cityID": 3445900, "cityName": "Middlesex" } }
+{ "create_at": datetime("2016-01-06T10:00:36.000Z"), "id": 684796722535624706, "text": "@mickHEELryan @bigpeteycool enjoy...and enjoy the banality of their phone calls", "in_reply_to_status": 684796542914527233, "in_reply_to_user": 1069186628, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1069186628, 2286279549 }}, "user": { "id": 3076245165, "name": "DavKind", "screen_name": "DavKind76", "lang": "en", "location": "I follow back wrestling fans ", "create_at": date("2015-03-06"), "description": "#WWE Wrestling fan~LIVE tweeting #RAW, PPVs and sometimes #WWENxT", "followers_count": 2805, "friends_count": 2275, "statues_count": 13643 }, "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-06T10:00:36.000Z"), "id": 684796722699202560, "text": "28th Ward Ald. Jason Ervin: \"This is not an accident. This is a pattern. Amazing in a civilized society that we continue to deal w this.\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.733238,41.8819382"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 45555548, "name": "Alex Nitkin", "screen_name": "AlexNitkin", "lang": "en", "location": "Chicago", "create_at": date("2009-06-08"), "description": "@DNAinfoChi reporter, recent @MedillSchool grad. Formerly of @CatalystChicago and @ChicagoBureau. Amateur knitter, CT Eagle Scout, lover of all things edible.", "followers_count": 612, "friends_count": 448, "statues_count": 1468 }, "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-06T10:00:36.000Z"), "id": 684796722728550400, "text": "@crazy_sanz @Friendly_bini @reddy_nakshatra @debjanipk @mamathan @kuch_pani yahoo me in meeting just checked gud job �� #WeStandByParth", "in_reply_to_status": 684778910429659137, "in_reply_to_user": 3251627012, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WeStandByParth" }}, "user_mentions": {{ 3251627012, 1175962135, 3603772337, 1699870662, 56625161, 3173016544 }}, "user": { "id": 2974108132, "name": "Aditi Bagchi", "screen_name": "bagchi12", "lang": "en", "location": "United States", "create_at": date("2015-01-11"), "description": "Pediatric Oncologist .........!!! Hum Dariya hain! Humein hamara hunar maaloom hai, Jis taraf hum nikal parenge, Raasta apne aap banta jaayega", "followers_count": 239, "friends_count": 103, "statues_count": 12945 }, "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-06T10:00:36.000Z"), "id": 684796722732756992, "text": "@moneypitmusic u know it!", "in_reply_to_status": 684795915723517952, "in_reply_to_user": 3248100956, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3248100956 }}, "user": { "id": 463904681, "name": "AL$AP ROCKY", "screen_name": "weirder_al", "lang": "en", "location": "NJ/DE", "create_at": date("2012-01-14"), "description": "Alex @inorganicrecs Life will always be weird", "followers_count": 388, "friends_count": 679, "statues_count": 23722 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Loch Arbour, NJ", "id": "9f56f088298b8c0c", "name": "Loch Arbour", "place_type": "city", "bounding_box": rectangle("-74.009384,40.230231 -73.995005,40.234022") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3441010, "cityName": "Loch Arbour" } }
+{ "create_at": datetime("2016-01-06T10:00:36.000Z"), "id": 684796722812436480, "text": "Fuck around on me & Il fuck you both up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 389435754, "name": "Kristaps", "screen_name": "Quentinanthony7", "lang": "en", "location": "D.M.[V]", "create_at": date("2011-10-12"), "description": "I might smile and say wassup but I don't fuck wit you. #IRap #YSL #BsUpHoesDown #SavageSZN", "followers_count": 1015, "friends_count": 288, "statues_count": 9146 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winchester, VA", "id": "b4add85f41a3e371", "name": "Winchester", "place_type": "city", "bounding_box": rectangle("-78.201069,39.13086 -78.137654,39.204922") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51840, "countyName": "Winchester", "cityID": 5186720, "cityName": "Winchester" } }
+{ "create_at": datetime("2016-01-06T10:00:36.000Z"), "id": 684796723420618752, "text": "Don't get on our bad side �� @ Bayside - Port of Miami https://t.co/C2iHn4T1fi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.1861047,25.77787366"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 128940232, "name": "A Real Mermaid", "screen_name": "CaliforniaBuzz_", "lang": "en", "location": "The Sea, Atlantis", "create_at": date("2010-04-02"), "description": "FMT Daytona Fall '13 | Zoe Girl | ReKonstruKtion | II-XXIV-MCMXCV | SC & IG: California_Buzz | #10Reasons | #DoIt4Dame | I'm a Mermaid I'm meant to swim freely", "followers_count": 1765, "friends_count": 1421, "statues_count": 61027 }, "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-06T10:00:36.000Z"), "id": 684796723902857216, "text": "When they give you a 69 like just give me the extra point for bringing a pencil https://t.co/NDBpt8MjpM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2778431890, "name": "erin", "screen_name": "erinnicolegh", "lang": "en", "location": "Weslaco, TX", "create_at": date("2014-09-20"), "description": "null", "followers_count": 1008, "friends_count": 279, "statues_count": 27345 }, "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-06T10:00:36.000Z"), "id": 684796724108509184, "text": "Even though I am sore today, last night was a balst! Can't wait to do it again tomorrow!!… https://t.co/bLLOagzDty", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.6645248,43.57802742"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 108797496, "name": "Riann Reurink", "screen_name": "RiannReurink", "lang": "en", "location": "Rapid City", "create_at": date("2010-01-26"), "description": "Senior at Augustana College, majoring in Business Communications", "followers_count": 230, "friends_count": 308, "statues_count": 2977 }, "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-06T10:00:36.000Z"), "id": 684796724167241728, "text": "Canada jawn yeaa i think that bitch from the 6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 523213194, "name": "Jovan", "screen_name": "Jovan_Almighty", "lang": "en", "location": "null", "create_at": date("2012-03-13"), "description": "Snapchat - Jovan2397 : HVCC #RIPYusef #SKUDA", "followers_count": 868, "friends_count": 613, "statues_count": 29189 }, "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-06T10:00:36.000Z"), "id": 684796724511174656, "text": "@ohoysakk ���������� I miss you too!!! Wednesday!!", "in_reply_to_status": 684796477483397120, "in_reply_to_user": 1894592029, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1894592029 }}, "user": { "id": 37770482, "name": "Riley Sharpe", "screen_name": "rileysharpe", "lang": "en", "location": "Hudson/Berea", "create_at": date("2009-05-04"), "description": "Delta Zeta. BWU Cheerleading. Odyssey Writer/Social Media. What Would Blair Waldorf Do?", "followers_count": 511, "friends_count": 201, "statues_count": 20571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hudson, OH", "id": "3ee2ce2b87ba9c14", "name": "Hudson", "place_type": "city", "bounding_box": rectangle("-81.494466,41.208066 -81.380598,41.277286") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3936651, "cityName": "Hudson" } }
+{ "create_at": datetime("2016-01-06T10:00:36.000Z"), "id": 684796724594905088, "text": "If The Broom Fits https://t.co/LfO0o7m8kp https://t.co/mg1yCVgGH3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.767709,45.429898"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16028156, "name": "Debbie", "screen_name": "Sbed", "lang": "en", "location": "Oregon", "create_at": date("2008-08-28"), "description": "Debbie Bailey is an American Witch, of 20+ years, living with her husband and fur babies in the Pacific Northwest.", "followers_count": 185, "friends_count": 181, "statues_count": 3062 }, "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-06T10:00:36.000Z"), "id": 684796724729090049, "text": ";) howd u know https://t.co/MHdN3L5zxG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3096655856, "name": "papí angel", "screen_name": "happypappii", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-03-18"), "description": "dhs | vie sentimentale", "followers_count": 198, "friends_count": 166, "statues_count": 5503 }, "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-06T10:00:36.000Z"), "id": 684796724813008896, "text": "my mom is mad at me cause I left school and went home to fix my makeup", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3122899460, "name": "brat", "screen_name": "glvsss", "lang": "en", "location": "null", "create_at": date("2015-03-31"), "description": "mak", "followers_count": 326, "friends_count": 206, "statues_count": 5909 }, "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-06T10:00:36.000Z"), "id": 684796724905381889, "text": "Daughters of Mi is hiring! Nurse Educator #jobs in CLIFTON Apply today https://t.co/gpycFtc8oB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.142536,40.878262"), "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": 169, "friends_count": 42, "statues_count": 72232 }, "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-01-06T10:00:36.000Z"), "id": 684796725119324161, "text": "@jeremylovebacon Because you believe geopolitic has anything to do with morality, it doesn't.", "in_reply_to_status": 684710212289232896, "in_reply_to_user": 39983059, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 39983059 }}, "user": { "id": 25738750, "name": "Brandy Boss", "screen_name": "BranThaBuilder", "lang": "en", "location": "The Hill ", "create_at": date("2009-03-21"), "description": "I'm the liberal media. // Phi Gamma Delta // The Sake Set", "followers_count": 176, "friends_count": 184, "statues_count": 7724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte Hall, MD", "id": "2f973010ca1bf5ce", "name": "Charlotte Hall", "place_type": "city", "bounding_box": rectangle("-76.790756,38.46618 -76.77058,38.50232") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24037, "countyName": "St. Mary's", "cityID": 2415475, "cityName": "Charlotte Hall" } }
+{ "create_at": datetime("2016-01-06T10:00:36.000Z"), "id": 684796725417095170, "text": "\"im not lion\" @xcvrlevx \n\nLying* educate yourself.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4417850789 }}, "user": { "id": 1566947022, "name": "Black Friend Joe", "screen_name": "JoeName_", "lang": "en", "location": "null", "create_at": date("2013-07-03"), "description": "null", "followers_count": 130, "friends_count": 198, "statues_count": 438 }, "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-01-06T10:00:36.000Z"), "id": 684796725568081920, "text": "You Hea Me https://t.co/o60ThC3Nis", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 379886524, "name": "❤️Ay'Deshaun❤", "screen_name": "SuckMee___", "lang": "en", "location": "❤", "create_at": date("2011-09-25"), "description": "Takes No One BullShit |Cancer ♋♋️| [D]MV Fuck Everything.~Employed", "followers_count": 2157, "friends_count": 2519, "statues_count": 29304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seat Pleasant, MD", "id": "afe336fdd7d1b837", "name": "Seat Pleasant", "place_type": "city", "bounding_box": rectangle("-76.912348,38.886827 -76.886724,38.914939") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2470850, "cityName": "Seat Pleasant" } }
+{ "create_at": datetime("2016-01-06T10:00:36.000Z"), "id": 684796725781872640, "text": "@SpeakerRyan Why do you hate America?", "in_reply_to_status": 684791983093710849, "in_reply_to_user": 18916432, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18916432 }}, "user": { "id": 34937126, "name": "John Hardy", "screen_name": "JohnHardy303", "lang": "en", "location": "Temescal, CA 94609", "create_at": date("2009-04-24"), "description": "Liberal. Democrat. Tall. Celt. Dad. Wonk. Partner. On-My-Way-To-Oaktown! Harp. 47-yr. New England native/resident. Passion. East Bay. Change. Show Up.", "followers_count": 214, "friends_count": 1058, "statues_count": 6875 }, "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-06T10:00:37.000Z"), "id": 684796726130163712, "text": "Mississippi damned is a good Netflix movie.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 552434499, "name": "ᎯЅℋ", "screen_name": "InkGoldSkin", "lang": "en", "location": "Huntsville, AL", "create_at": date("2012-04-12"), "description": "my apologies toward anyone who's ever missed out on my love you lost a good thing. ✨", "followers_count": 1857, "friends_count": 1460, "statues_count": 19417 }, "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-06T10:00:37.000Z"), "id": 684796726398566401, "text": "@jrocvsop Now, all we need is Mattingly to shave his sideburns", "in_reply_to_status": 684796455576559617, "in_reply_to_user": 267443112, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 267443112 }}, "user": { "id": 60489290, "name": "Joe Dwyer", "screen_name": "jdwyer02", "lang": "en", "location": "null", "create_at": date("2009-07-26"), "description": "Everyone wants a slice of the beefcake", "followers_count": 62, "friends_count": 176, "statues_count": 2125 }, "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-06T10:00:37.000Z"), "id": 684796726931288064, "text": "lowkey depressed I gotta go back to school , I had to much fun 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": 41916656, "name": "That Boy Cortez✊", "screen_name": "Cortez_AirBorne", "lang": "en", "location": "null", "create_at": date("2009-05-22"), "description": "email: cortavius14@gmail.com … GVO UAPB\n\n101.3 The Beat Intern", "followers_count": 2811, "friends_count": 1750, "statues_count": 50792 }, "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-06T10:00:37.000Z"), "id": 684796727124099073, "text": "Join the UMC team! See our latest #Nursing #job opening here: https://t.co/psqTodhVra #cardiaccare #nurse #Lubbock, TX #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-101.8919288,33.5881366"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "cardiaccare", "nurse", "Lubbock", "Hiring" }}, "user": { "id": 2940007148, "name": "UMC Nursing Jobs", "screen_name": "UMCNursingJobs", "lang": "en", "location": "null", "create_at": date("2014-12-22"), "description": "UMC Health System is the leader in comprehensive healthcare delivery in West Texas and Eastern New Mexico. Follow us to learn about our Nursing Jobs!", "followers_count": 92, "friends_count": 278, "statues_count": 78 }, "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-06T10:00:37.000Z"), "id": 684796727572889600, "text": "Interested in a #Marketing #job near #Charlotte, NC? This could be a great fit: https://t.co/LhzmDdzFja #QA #Agile #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.8431267,35.2270869"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Marketing", "job", "Charlotte", "QA", "Agile", "Hiring" }}, "user": { "id": 2935068602, "name": "NASCAR Jobs", "screen_name": "NASCARJobs", "lang": "en", "location": "null", "create_at": date("2014-12-18"), "description": "The National Association for Stock Car Auto Racing, Inc. (NASCAR) is the sanctioning body for one of North America's premier sports.", "followers_count": 1110, "friends_count": 14, "statues_count": 160 }, "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-06T10:00:37.000Z"), "id": 684796727602319360, "text": "Henry Repeating is hiring! Shipping Manage #jobs in BAYONNE Apply today https://t.co/zAnUxkcmnN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.106366,40.67078"), "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": 169, "friends_count": 42, "statues_count": 72233 }, "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-06T10:00:37.000Z"), "id": 684796727660953600, "text": "@PanaZuluaga9 (-':", "in_reply_to_status": 684795834999783424, "in_reply_to_user": 2388397201, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2388397201 }}, "user": { "id": 2267516845, "name": "Kase", "screen_name": "_KaseyMichelle_", "lang": "en", "location": "TX", "create_at": date("2013-12-29"), "description": "See Panas profile for more details @PanaZuluaga9", "followers_count": 458, "friends_count": 186, "statues_count": 11305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hutto, TX", "id": "005fd8d279073e1f", "name": "Hutto", "place_type": "city", "bounding_box": rectangle("-97.575589,30.500251 -97.527116,30.569311") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48491, "countyName": "Williamson", "cityID": 4835624, "cityName": "Hutto" } }
+{ "create_at": datetime("2016-01-06T10:00:37.000Z"), "id": 684796727660957697, "text": "God I see what you doing you not sneaky 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": 231997207, "name": "eatme", "screen_name": "cravebuttvh", "lang": "en", "location": "balifornia", "create_at": date("2010-12-29"), "description": "forgive me", "followers_count": 2600, "friends_count": 1577, "statues_count": 126388 }, "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-06T10:00:37.000Z"), "id": 684796728038539264, "text": "I just want @shaynafix to know that @HeyyKatie3 and I are fighting about which one of us should date her in our GC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15249766, 294885708 }}, "user": { "id": 740999281, "name": "Leila Von Blon", "screen_name": "leilavonb", "lang": "en", "location": "Atlanta, GA", "create_at": date("2012-08-06"), "description": "wanna get lei'd?", "followers_count": 517, "friends_count": 354, "statues_count": 45555 }, "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-06T10:00:37.000Z"), "id": 684796728621453312, "text": "she's a damn babe idgaf ���� https://t.co/lfLnkozmnu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3316630982, "name": "r a n g g g", "screen_name": "Hadden812_", "lang": "en", "location": "friendzone", "create_at": date("2015-08-16"), "description": "making it on my own. catch up.", "followers_count": 345, "friends_count": 336, "statues_count": 3364 }, "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-06T10:00:37.000Z"), "id": 684796728764153856, "text": "Check out the #danielquagliozzi YouTube channel for no bullshit cat advice. You just got… https://t.co/PH2fIIF5nv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.46393394,37.76457876"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "danielquagliozzi" }}, "user": { "id": 711460351, "name": "Go, Cat, Go!", "screen_name": "gocatgosf", "lang": "en", "location": "San Francisco", "create_at": date("2012-07-22"), "description": "Teaching humans how to be more like cats.", "followers_count": 658, "friends_count": 1107, "statues_count": 2720 }, "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-06T10:00:37.000Z"), "id": 684796728889847808, "text": "@One2Go_Results do you have a link for the Masters 55-59 race for live viewing. It's not in your drop-down. Thx!", "in_reply_to_status": -1, "in_reply_to_user": 1383836198, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1383836198 }}, "user": { "id": 41962560, "name": "Kent McNeill", "screen_name": "KJMCNEILL", "lang": "en", "location": "Bellevue, Nebraska", "create_at": date("2009-05-22"), "description": "Once was pro MTB racer enjoying life as a dad, hack entrepreneur, and grower of our sport. Trying to slow down enough to not miss the #DivineBeauty", "followers_count": 576, "friends_count": 788, "statues_count": 2038 }, "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-06T10:00:37.000Z"), "id": 684796729217134593, "text": "They're only targeting him because he's uneducated He's defenseless coming from a backwater town. EDUCATION IS SO IMPORTANT #MakingAMurderer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MakingAMurderer" }}, "user": { "id": 248486620, "name": "Andy", "screen_name": "AndrewIsKitten", "lang": "en", "location": "Corvallis, OR//Northfield, MN", "create_at": date("2011-02-06"), "description": "⠀⠀⠀⠀⠀⠀Do not speak unless spoken to⠀⠀⠀⠀⠀ ⠀⠀⠀⠀⠀⠀⠀⠀You're all underhumans ⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀St. Olaf College", "followers_count": 419, "friends_count": 230, "statues_count": 16665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Northfield, MN", "id": "f1a27bbf286571f8", "name": "Northfield", "place_type": "city", "bounding_box": rectangle("-93.201231,44.422625 -93.127284,44.47687") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27131, "countyName": "Rice", "cityID": 2746924, "cityName": "Northfield" } }
+{ "create_at": datetime("2016-01-06T10:00:37.000Z"), "id": 684796729355456512, "text": "@Uberbored_80 e meno male! @FabioWhat", "in_reply_to_status": 684796590566027264, "in_reply_to_user": 345020176, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user_mentions": {{ 345020176, 284455533 }}, "user": { "id": 395268713, "name": "pinupz", "screen_name": "pinupz", "lang": "it", "location": "null", "create_at": date("2011-10-21"), "description": "scrivo cazzate, non dico bugie.", "followers_count": 1089, "friends_count": 1354, "statues_count": 10959 }, "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-06T10:00:37.000Z"), "id": 684796729422524416, "text": "Lit https://t.co/ivzdPAHLf3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1687788324, "name": "➕Šhüń➕", "screen_name": "blessed_boyz", "lang": "en", "location": "@ the night show ", "create_at": date("2013-08-21"), "description": ".. +ŁØĘ+ if you leave shun's world its a long walk home", "followers_count": 574, "friends_count": 661, "statues_count": 24735 }, "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-06T10:00:37.000Z"), "id": 684796729506574338, "text": "@NASA certainly looks like it... Eta carnae", "in_reply_to_status": 684795960942301185, "in_reply_to_user": 11348282, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 11348282 }}, "user": { "id": 1323359760, "name": "D Mo", "screen_name": "mojamesd64", "lang": "en", "location": "null", "create_at": date("2013-04-02"), "description": "World traveler, get like me...", "followers_count": 19, "friends_count": 38, "statues_count": 830 }, "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-01-06T10:00:37.000Z"), "id": 684796729741295616, "text": "Another year, another CES, and a mass of humans ready for Central Hall to open. Here for #PaxEx tech applications. https://t.co/1YcsJxg6mO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PaxEx" }}, "user": { "id": 17542672, "name": "Runway Girl Network", "screen_name": "RunwayGirl", "lang": "en", "location": "null", "create_at": date("2008-11-21"), "description": "Where air transport intelligence meets the passenger experience. Editor Mary Kirby. #PaxEx #Safety #AvGeek #LeanIntoAviation #HeForShe #IoT", "followers_count": 47730, "friends_count": 22841, "statues_count": 53881 }, "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-06T10:00:37.000Z"), "id": 684796729775013889, "text": "Los amigos y el dinero es otra cosa si se juntan pueden ser peligrosa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 247056646, "name": "César Rincón", "screen_name": "CRincon97", "lang": "en", "location": "Miami, FL. ", "create_at": date("2011-02-03"), "description": "18. Instagram: crincon15. Venezolano antes que todo.", "followers_count": 379, "friends_count": 111, "statues_count": 30995 }, "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-06T10:00:38.000Z"), "id": 684796730311745540, "text": "@Tunechieee follow me on snap bro Jay-easy", "in_reply_to_status": 684796576678608897, "in_reply_to_user": 161597743, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 161597743 }}, "user": { "id": 475498561, "name": "Kobe number 8", "screen_name": "Jay_Easy_", "lang": "en", "location": "null", "create_at": date("2012-01-26"), "description": "No talking, we bringing drama fuck you and yo muthafucking mama", "followers_count": 1107, "friends_count": 784, "statues_count": 28591 }, "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-06T10:00:38.000Z"), "id": 684796730387345408, "text": "@rickandbubba Want to play @martybrownusa Make You Feel My Love on ur radio,email ilikeitthatwaymusic@yahoo.com https://t.co/4Kokpw0CHq", "in_reply_to_status": -1, "in_reply_to_user": 19352525, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19352525, 1552469791 }}, "user": { "id": 1283634092, "name": "Belinda Loyd", "screen_name": "BelleLoyd", "lang": "en", "location": "Home Town of Marty Brown", "create_at": date("2013-03-20"), "description": "FROM MACEO, KY. THREE CHILDREN, .RETIRED, PROMOTES COUNTRY ARTIST MARTY BROWN", "followers_count": 780, "friends_count": 1413, "statues_count": 7864 }, "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-01-06T10:00:38.000Z"), "id": 684796730546589697, "text": "New move , boxing for money", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 381358403, "name": "Yung $tunna Bruh", "screen_name": "ShmopLifeDom", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-09-27"), "description": "Leader of the Fee World #RIpBabyRJ", "followers_count": 357, "friends_count": 287, "statues_count": 24930 }, "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-01-06T10:00:38.000Z"), "id": 684796730659880960, "text": "@MeninistTweet @AustinTheBailey", "in_reply_to_status": 684796359870918659, "in_reply_to_user": 2870078111, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2870078111, 2279881087 }}, "user": { "id": 1862952246, "name": "So Salty", "screen_name": "saltywaters_", "lang": "en", "location": "Shacked, TX", "create_at": date("2013-09-14"), "description": "(FISY) #boys #spurs #chipdoctor #nicksdaddy", "followers_count": 356, "friends_count": 349, "statues_count": 9814 }, "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-01-06T10:00:38.000Z"), "id": 684796730752143360, "text": "Finally, a nice morning where its not -15 degrees", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 32974865, "name": "Rachel Thomas", "screen_name": "RachelllMarieee", "lang": "en", "location": "null", "create_at": date("2009-04-18"), "description": "on my own trip", "followers_count": 171, "friends_count": 109, "statues_count": 4236 }, "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-06T10:00:38.000Z"), "id": 684796730840330240, "text": "Just posted a photo @ Maldonado Home https://t.co/brRIqyI0x4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-120.51047714,46.57437843"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3119230753, "name": "Priscilla Maldonado", "screen_name": "sweetcurves309", "lang": "en", "location": "null", "create_at": date("2015-03-30"), "description": "null", "followers_count": 14, "friends_count": 72, "statues_count": 333 }, "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-06T10:00:38.000Z"), "id": 684796730957819904, "text": "Así o más claro ?? #MaloNoEs #Venezuela #Asamblea #Cambio #HenryRamosAllup @ Cinco Ranch, Texas https://t.co/iqXID4WyyZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.7583,29.7415"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "MaloNoEs", "Venezuela", "Asamblea", "Cambio", "HenryRamosAllup" }}, "user": { "id": 109074629, "name": "Luis Oscar Rodriguez", "screen_name": "LuisOscarR", "lang": "en", "location": "Houston, TX", "create_at": date("2010-01-27"), "description": "Venezolano / Periodista / On Air Talent / Morning Show Co Host Mega En Tus Mañanas / CBS Radio Houston / IG:@LuisOscarR @mega101houston #MaloNoEs", "followers_count": 521, "friends_count": 122, "statues_count": 1434 }, "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-06T10:00:38.000Z"), "id": 684796731184164865, "text": "Interested in a #IT #job near #Nashville, TN? This could be a great fit: https://t.co/hLLsdVLtZz #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.7805961,36.1724885"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "IT", "job", "Nashville", "Hiring", "CareerArc" }}, "user": { "id": 82676628, "name": "TMJ-BNA IT Adm. Jobs", "screen_name": "tmj_bna_adm", "lang": "en", "location": "Nashville, TN", "create_at": date("2009-10-15"), "description": "Follow this account for geo-targeted IT-Support/Administration job tweets in Nashville, TN from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 299, "friends_count": 279, "statues_count": 34 }, "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.99451 -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-01-06T10:00:38.000Z"), "id": 684796731423338496, "text": "free wifi in class ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 772648729, "name": "lola", "screen_name": "Liliamx18", "lang": "en", "location": "null", "create_at": date("2012-08-21"), "description": "Alexander's girl.", "followers_count": 387, "friends_count": 198, "statues_count": 9543 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hialeah Gardens, FL", "id": "6fb5fca39609ee86", "name": "Hialeah Gardens", "place_type": "city", "bounding_box": rectangle("-80.372397,25.853728 -80.322682,25.898707") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1230025, "cityName": "Hialeah Gardens" } }
+{ "create_at": datetime("2016-01-06T10:00:38.000Z"), "id": 684796731477757953, "text": "Want to work in #Hershey, PA? View our latest opening: https://t.co/4hltm2RbUJ #Transportation #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.6502468,40.2859239"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hershey", "Transportation", "Job", "Jobs", "Hiring" }}, "user": { "id": 148979169, "name": "TMJ-PA Transport.", "screen_name": "tmj_PA_transp", "lang": "en", "location": "Pennsylvania", "create_at": date("2010-05-27"), "description": "Follow this account for geo-targeted Transportation job tweets in Pennsylvania Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 348, "friends_count": 292, "statues_count": 465 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hershey, PA", "id": "8647282dba1cee1e", "name": "Hershey", "place_type": "city", "bounding_box": rectangle("-76.689419,40.246151 -76.601027,40.302317") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42043, "countyName": "Dauphin", "cityID": 4234144, "cityName": "Hershey" } }
+{ "create_at": datetime("2016-01-06T10:00:38.000Z"), "id": 684796731490435072, "text": "Never too early! https://t.co/zrk4Aceyg2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 11776352, "name": "CCwriter", "screen_name": "ccwriter", "lang": "en", "location": "Washington, D.C. & Northern VA", "create_at": date("2008-01-02"), "description": "Writer. Fan of @SamHarrisOrg #FreeSpeech #Nats #JacksonPollock. #VaWine #Space Michigan native. Alum of Central Michigan University + Georgetown U.", "followers_count": 1227, "friends_count": 1507, "statues_count": 19848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reston, VA", "id": "ec212eb1116b92e2", "name": "Reston", "place_type": "city", "bounding_box": rectangle("-77.393246,38.90861 -77.304868,39.002993") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5166672, "cityName": "Reston" } }
+{ "create_at": datetime("2016-01-06T10:00:38.000Z"), "id": 684796731620507648, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 35932145, "name": "amirah b. ✨", "screen_name": "fvckmira", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-04-27"), "description": "stay humble | georgia state university ✨ ⠀⠀⠀⠀⠀⠀ snap:fvckmira #youdontknowme", "followers_count": 10973, "friends_count": 4777, "statues_count": 98582 }, "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-06T10:00:38.000Z"), "id": 684796732123803648, "text": "it be some crazy shit going on on facebook .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.421944,37.544714"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 513437743, "name": "#bossmaynedonns.", "screen_name": "shortBeauty___", "lang": "en", "location": "Richmond, VA", "create_at": date("2012-03-03"), "description": "null", "followers_count": 1671, "friends_count": 1077, "statues_count": 123543 }, "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-06T10:00:38.000Z"), "id": 684796732161445888, "text": "idc idc ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2748456570, "name": "Daddie'MACC", "screen_name": "DaddieNiqNiq_", "lang": "en", "location": "null", "create_at": date("2014-08-20"), "description": "✊", "followers_count": 1117, "friends_count": 1822, "statues_count": 3651 }, "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-06T10:00:38.000Z"), "id": 684796732190818304, "text": "@MrPryority its some truth to it doe lmao", "in_reply_to_status": 684796640348233728, "in_reply_to_user": 30589197, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 30589197 }}, "user": { "id": 269200310, "name": "JayZOverrated", "screen_name": "JayZOverrated", "lang": "en", "location": "null", "create_at": date("2011-03-20"), "description": "null", "followers_count": 7304, "friends_count": 1381, "statues_count": 290689 }, "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-01-06T10:00:38.000Z"), "id": 684796732278882304, "text": "So much style I can't handle it https://t.co/wZMVs79OXC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2384492198, "name": "Chat Shit Get Banged", "screen_name": "TheSignifigenc", "lang": "en", "location": "null", "create_at": date("2014-03-11"), "description": "I'm like the Sun. Everyone needs a little me in there life but if they get to much I give them skin cancer", "followers_count": 219, "friends_count": 208, "statues_count": 9207 }, "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-06T10:00:38.000Z"), "id": 684796732329230336, "text": "Field Repair-Montrose Centre - Jared: (#Akron, OH) https://t.co/nSkUjgPhUz #SkilledTrade #Jeweler #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.5190053,41.0814447"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Akron", "SkilledTrade", "Jeweler", "Job", "Jobs", "Hiring" }}, "user": { "id": 3248848340, "name": "Jobs at Jared", "screen_name": "JobsAtJared", "lang": "en", "location": "null", "create_at": date("2015-06-18"), "description": "Your Future Starts Here.", "followers_count": 14, "friends_count": 1, "statues_count": 504 }, "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-01-06T10:00:38.000Z"), "id": 684796732534878211, "text": "This some petty shit I would do https://t.co/3DmCgAhaqT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 172928888, "name": "DURTY", "screen_name": "DeeLaSmoothe", "lang": "en", "location": "with y(our) nigga ", "create_at": date("2010-07-30"), "description": "coolest black Jew smoking the devil's lettuce", "followers_count": 671, "friends_count": 585, "statues_count": 18949 }, "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-01-06T10:00:38.000Z"), "id": 684796732752957440, "text": "@dncsnr 2019 ��", "in_reply_to_status": 684796584773550080, "in_reply_to_user": 539267512, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 539267512 }}, "user": { "id": 418602866, "name": "Daryl Granario", "screen_name": "chiri386", "lang": "en", "location": "Auburn, AL", "create_at": date("2011-11-22"), "description": "Auburn U Math GTA and UP Diliman I-Math Instructor. Just another Deku Scrub", "followers_count": 399, "friends_count": 398, "statues_count": 12228 }, "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-01-06T10:00:38.000Z"), "id": 684796732929015808, "text": "@SoulxErosion @jdank_ but you get to see me and it would all be worth it ����", "in_reply_to_status": 684796624082604032, "in_reply_to_user": 515127905, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 515127905, 337967410 }}, "user": { "id": 301181447, "name": "elizabeth", "screen_name": "chvserx", "lang": "en", "location": "Dubuque", "create_at": date("2011-05-18"), "description": "18. I love the wonder years ok", "followers_count": 432, "friends_count": 325, "statues_count": 31386 }, "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-06T10:00:38.000Z"), "id": 684796733201616896, "text": "not two minutes into her shift this chick is grabbing the store Advil and being loud about it smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 197116059, "name": "queen of the damned", "screen_name": "p0ni3s", "lang": "en", "location": "@pwni3s", "create_at": date("2010-09-30"), "description": "it's me big t", "followers_count": 62, "friends_count": 124, "statues_count": 3669 }, "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-06T10:00:38.000Z"), "id": 684796733428137984, "text": "@RyanRiesling Calle tacos!", "in_reply_to_status": 684793543492268032, "in_reply_to_user": 4094307383, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 4094307383 }}, "user": { "id": 612489667, "name": "Chelsea Marlee.", "screen_name": "pigletwiggles", "lang": "en", "location": "Hollywood, CA", "create_at": date("2012-06-19"), "description": "| rock&roll | musicians institute | engaged to @burials_ | @dixiediesfeld is my queen |", "followers_count": 569, "friends_count": 161, "statues_count": 9884 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -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-06T10:00:38.000Z"), "id": 684796733683978240, "text": "Dellavedova!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 2938866330, "name": "€", "screen_name": "M2KORS", "lang": "en", "location": "Vancouver, WA", "create_at": date("2014-12-21"), "description": "HBHS '18| PNW✈️. Loading.... 6'2", "followers_count": 400, "friends_count": 305, "statues_count": 4948 }, "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-06T10:00:38.000Z"), "id": 684796733784780801, "text": "@24Karat_OfOG when in March here & might as well just get the slugs here lol.", "in_reply_to_status": 684795698223681536, "in_reply_to_user": 69471395, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 69471395 }}, "user": { "id": 179802518, "name": "Chester", "screen_name": "kaseynicolle_", "lang": "en", "location": "Cockville, USA", "create_at": date("2010-08-17"), "description": "good vibes. good times. good dimes.", "followers_count": 633, "friends_count": 161, "statues_count": 23726 }, "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-01-06T10:00:38.000Z"), "id": 684796734078369792, "text": ".@CUNA mo. #CU estimates: Home equity loans power November CU loan growth https://t.co/WabqDLMmKV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CU" }}, "user_mentions": {{ 82677928 }}, "user": { "id": 16516851, "name": "News Now LiveWire", "screen_name": "NewsNowLiveWire", "lang": "en", "location": "Washington, D.C.", "create_at": date("2008-09-29"), "description": "Providing credit unions up-to-the-second credit union news from anywhere a cell phone will roam.", "followers_count": 5575, "friends_count": 303, "statues_count": 14125 }, "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-06T10:00:38.000Z"), "id": 684796734107602944, "text": "I should let that bitch find her own way home", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3339583472, "name": "Go Texans!", "screen_name": "ShooterMcGavan", "lang": "en", "location": "The Hills", "create_at": date("2015-08-25"), "description": "I'm cool like the back of a pillow", "followers_count": 247, "friends_count": 891, "statues_count": 2006 }, "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-06T10:00:38.000Z"), "id": 684796734187388928, "text": "Get em' bae. �� https://t.co/TMTDStrApJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2707536652, "name": "finesse queen.", "screen_name": "nayaBAEcelona", "lang": "en", "location": "null", "create_at": date("2014-07-13"), "description": "The blasian princessa. ✨ Pro. MUA. ✨ | BOS2LA. | nayabarcelona92@gmail.com | Peace, glo, and prosper.", "followers_count": 253, "friends_count": 273, "statues_count": 10591 }, "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-06T10:00:38.000Z"), "id": 684796734195806209, "text": "Yall aint tell me it was a Bloomingdales outlet in here! They're having a great sale", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 37201702, "name": "Miss Cecil B", "screen_name": "MonaLimah", "lang": "en", "location": "Temple University ", "create_at": date("2009-05-02"), "description": "Confidence is the companion of success. No choice but to feel myself.", "followers_count": 936, "friends_count": 508, "statues_count": 41370 }, "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-06T10:00:39.000Z"), "id": 684796734292267012, "text": "@I_AM_V3NGFUL its not like it would always show you the whole post\nIf it kept on showing at ~200 char, with a \"...\" to expand, it would work", "in_reply_to_status": 684796324068376578, "in_reply_to_user": 282221403, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1242178471 }}, "user": { "id": 282221403, "name": "BChaps", "screen_name": "BChapy", "lang": "en", "location": "Cincinnati, OH", "create_at": date("2011-04-14"), "description": "Aerospace Engineer from Virginia Tech.\nVTM15 Aerodynamics leader.\nLove Video Games and Hockey.", "followers_count": 302, "friends_count": 239, "statues_count": 7185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glendale, OH", "id": "6600d817d5d6d521", "name": "Glendale", "place_type": "city", "bounding_box": rectangle("-84.47811,39.260555 -84.440447,39.279694") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3930380, "cityName": "Glendale" } }
+{ "create_at": datetime("2016-01-06T10:00:39.000Z"), "id": 684796734304698370, "text": "Can't wait to travel the world with the love of my life someday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1445060809, "name": "Alyssa Baratta", "screen_name": "alyssabaratta", "lang": "en", "location": "null", "create_at": date("2013-05-20"), "description": "Hi I'm Alyssa Baratta✨", "followers_count": 690, "friends_count": 612, "statues_count": 21836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elmwood Park, IL", "id": "20c885d40cb26f0f", "name": "Elmwood Park", "place_type": "city", "bounding_box": rectangle("-87.826203,41.908494 -87.805665,41.937955") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1723724, "cityName": "Elmwood Park" } }
+{ "create_at": datetime("2016-01-06T10:00:39.000Z"), "id": 684796734547963904, "text": "Don't feel like moving around but I need to go do what I got 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": 2288586398, "name": "Slim.T ✨", "screen_name": "lilmissMia_", "lang": "en", "location": "Beaumont, TX", "create_at": date("2014-01-12"), "description": "It's somn bout the rings she got in ha nipples #409 #RIPpapa #RIPgran #RIPmatt", "followers_count": 1829, "friends_count": 939, "statues_count": 66854 }, "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-06T10:00:39.000Z"), "id": 684796734585704448, "text": "@Jane_Jets_Son they both do, I've seen Kylie rapping his songs too", "in_reply_to_status": 684796619070541824, "in_reply_to_user": 154208384, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 154208384 }}, "user": { "id": 124559605, "name": "TJK", "screen_name": "YourAverageJet", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2010-03-19"), "description": "Oh, you really need us to show you how we're harder than finding a fuckin' Snicker bar in Whole Foods?", "followers_count": 1121, "friends_count": 799, "statues_count": 96744 }, "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-06T10:00:39.000Z"), "id": 684796734845878272, "text": "Happy birthday @Wil_funderbu!! 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": {{ 2558168703 }}, "user": { "id": 588431448, "name": "Sara Sutton", "screen_name": "SaraJSutton", "lang": "en", "location": "null", "create_at": date("2012-05-23"), "description": "null", "followers_count": 1164, "friends_count": 857, "statues_count": 2157 }, "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-01-06T10:00:39.000Z"), "id": 684796735290388480, "text": "CDL-A Company Truck Drivers- Truckload - Roadrunner Transportation: (#Savannah, GA) https://t.co/VPmAfL5WzA #Transportation #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.0998342,32.0835407"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Savannah", "Transportation", "Job", "Jobs" }}, "user": { "id": 149734682, "name": "TMJ-GAS Transport.", "screen_name": "tmj_GAS_transp", "lang": "en", "location": "Savannah, GA", "create_at": date("2010-05-29"), "description": "Follow this account for geo-targeted Transportation job tweets in Savannah, GA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 281, "friends_count": 273, "statues_count": 33 }, "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-06T10:00:39.000Z"), "id": 684796735621709824, "text": "@Kibbie2FastTho I miss it ��������", "in_reply_to_status": 663859996656963584, "in_reply_to_user": 325152923, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 325152923 }}, "user": { "id": 325152923, "name": "Daidrick2⃣1⃣", "screen_name": "Kibbie2FastTho", "lang": "en", "location": "HTX✈️Cal", "create_at": date("2011-06-27"), "description": "The Struggle Made Me | SouthBank, Tx | Shasta College | DB Nation | Coosie | Cowboy Nation | 5'11| 195 | 40: 4.35 | sophomore", "followers_count": 1652, "friends_count": 1036, "statues_count": 65158 }, "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-01-06T10:00:39.000Z"), "id": 684796735806283776, "text": "This #Hospitality #job might be a great fit for you: Line Cook I - https://t.co/ZvKrj1pW2o #VegasLife #LasVegas, NV https://t.co/ENbc5ITKpn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.1745805,36.1062236"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "VegasLife", "LasVegas" }}, "user": { "id": 3089847157, "name": "Mandarin Las Vegas", "screen_name": "MO_LVJobs", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-03-16"), "description": "Nevada's only Triple Forbes Five-Star luxury destination located on the world-famous #LasVegas Strip. Apply here to open #jobs.", "followers_count": 86, "friends_count": 59, "statues_count": 46 }, "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-06T10:00:39.000Z"), "id": 684796735877726209, "text": "@AyeeRio https://t.co/0qxjRA4GfI", "in_reply_to_status": -1, "in_reply_to_user": 702205188, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 702205188 }}, "user": { "id": 549609757, "name": "Spencer", "screen_name": "bvtterz", "lang": "en", "location": "904 910", "create_at": date("2012-04-09"), "description": "kayla!!", "followers_count": 634, "friends_count": 448, "statues_count": 11102 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hope Mills, NC", "id": "014cce082fca34f9", "name": "Hope Mills", "place_type": "city", "bounding_box": rectangle("-79.033596,34.894886 -78.834267,35.009696") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37051, "countyName": "Cumberland", "cityID": 3732640, "cityName": "Hope Mills" } }
+{ "create_at": datetime("2016-01-06T10:00:39.000Z"), "id": 684796735957397504, "text": "Shot by me! I dabble my gift is catching the moment! However all photography is shoot by… https://t.co/GX8j3ZTuuq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.6409773,43.215069"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 32573963, "name": "Rahcos_Modern_Life", "screen_name": "Rahco", "lang": "en", "location": "Los Angeles/ New York/ ROC", "create_at": date("2009-04-17"), "description": "Fashion Expert/Host #Thephmagency #oddmomout #fashionstylist #fashionisto360 Rahco@thephmagency.com IG:@rahcos_modern_life ♠️♠️♠️♠️♠️♠️♠️♠️♠️♠️♠️♠️♠️♠️♠️", "followers_count": 820, "friends_count": 1931, "statues_count": 4882 }, "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": 3663000, "cityName": "Rochester" } }
+{ "create_at": datetime("2016-01-06T10:00:39.000Z"), "id": 684796736057954307, "text": "@NRawww ����", "in_reply_to_status": -1, "in_reply_to_user": 369332908, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 369332908 }}, "user": { "id": 354936969, "name": "ᏟhᎾᏟᎾᏞᎪᏆᎬ ᏢᎪᏢᎥ", "screen_name": "TwanHefner_", "lang": "en", "location": "Isla Vista, CA", "create_at": date("2011-08-14"), "description": "I may be single but at least I'm not getting cheated on! Inglewood ~ Corona ~ Santa Barbara", "followers_count": 1420, "friends_count": 803, "statues_count": 132317 }, "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-06T10:00:39.000Z"), "id": 684796736292847616, "text": "current weather in Tupelo: fair, 41°F\n49% humidity, wind 6mph, visibility 10mi, pressure 30.37in falling", "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": 119, "friends_count": 2, "statues_count": 20459 }, "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-06T10:00:39.000Z"), "id": 684796736565604352, "text": "@calebmgillen I need to know who voted for him to be in prison..", "in_reply_to_status": 684786271160504322, "in_reply_to_user": 2740531375, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2740531375 }}, "user": { "id": 161010132, "name": "Haley-lujah", "screen_name": "ChildishHaley", "lang": "en", "location": "Carrie Bradshaw's NY Apt ", "create_at": date("2010-06-29"), "description": "Part-time Rapper, Full-time cat lover", "followers_count": 354, "friends_count": 342, "statues_count": 12944 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Perrysburg, OH", "id": "c9ac14a04cfc8e73", "name": "Perrysburg", "place_type": "city", "bounding_box": rectangle("-83.703195,41.503612 -83.507239,41.601441") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39173, "countyName": "Wood", "cityID": 3962148, "cityName": "Perrysburg" } }
+{ "create_at": datetime("2016-01-06T10:00:39.000Z"), "id": 684796736619937792, "text": "Can you recommend anyone for this #job? Part-time American Sign Language Instructor - https://t.co/0ng3Z7XUQa #Vancouver, WA #Education", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.6509315,45.6360374"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Vancouver", "Education" }}, "user": { "id": 21838455, "name": "TMJ-PDX Educ. Jobs", "screen_name": "tmj_pdx_edu", "lang": "en", "location": "Portland, OR", "create_at": date("2009-02-24"), "description": "Follow this account for geo-targeted Education/Teaching job tweets in Portland, OR. Need help? Tweet us at @CareerArc!", "followers_count": 355, "friends_count": 235, "statues_count": 31 }, "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-06T10:00:39.000Z"), "id": 684796736678694912, "text": "That boy @tre_neil bout to drop some fire ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 592369641 }}, "user": { "id": 629540313, "name": "Frank Darien Colin", "screen_name": "FrankyColin5", "lang": "en", "location": "null", "create_at": date("2012-07-07"), "description": "OU 19' ΔΤΔ", "followers_count": 543, "friends_count": 334, "statues_count": 4145 }, "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-06T10:00:39.000Z"), "id": 684796736850640896, "text": "@steveesuniga @ChiIIVlbes das a bold move", "in_reply_to_status": 684795299257126912, "in_reply_to_user": 1013680866, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1013680866, 2193607094 }}, "user": { "id": 2579915107, "name": "kassandra", "screen_name": "kassiebee123", "lang": "en", "location": "null", "create_at": date("2014-06-20"), "description": "Give it to God. Dont criticize someone on their grind, we all start somewhere. Trust the process.", "followers_count": 1002, "friends_count": 344, "statues_count": 21004 }, "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-06T10:00:39.000Z"), "id": 684796737383460864, "text": "My dad just explained to me how to prepare for an airport.... I'm sorry do you know me?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 559229346, "name": "René Zerfas", "screen_name": "Zerfboardt", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2012-04-20"), "description": "Don't practice until you get it right; Practice until you cant get it wrong.", "followers_count": 246, "friends_count": 325, "statues_count": 2396 }, "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-06T10:00:39.000Z"), "id": 684796737672888320, "text": "i got LAID off. lol not fired", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2873738616, "name": "Ashley Dean", "screen_name": "ashhdeann99", "lang": "en", "location": "null", "create_at": date("2014-10-23"), "description": "junior @ bchs | cheer |", "followers_count": 157, "friends_count": 193, "statues_count": 821 }, "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-06T10:00:39.000Z"), "id": 684796737702133760, "text": "Look at beccas moms review on river valley:) https://t.co/PiNjcnFW3K", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 615534092, "name": "Maggie", "screen_name": "maggiee_joness", "lang": "en", "location": "null", "create_at": date("2012-06-22"), "description": "null", "followers_count": 638, "friends_count": 599, "statues_count": 6382 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Park Heights, MN", "id": "6854a1b57a5d84ee", "name": "Oak Park Heights", "place_type": "city", "bounding_box": rectangle("-92.856805,45.021416 -92.770003,45.043148") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27163, "countyName": "Washington", "cityID": 2747914, "cityName": "Oak Park Heights" } }
+{ "create_at": datetime("2016-01-06T10:00:39.000Z"), "id": 684796737857417216, "text": "@MattyIce819 Rod liked them so you'll def love them https://t.co/EokhteuMui", "in_reply_to_status": -1, "in_reply_to_user": 40114214, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 40114214 }}, "user": { "id": 39415403, "name": "mike guittar", "screen_name": "GdoubleT", "lang": "en", "location": "Rochester, NY", "create_at": date("2009-05-11"), "description": "Owner of Buffalo Bills crater on Mars - hockey player #LGM snapchat - @gdoublet - never #sawft", "followers_count": 1703, "friends_count": 1449, "statues_count": 54659 }, "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-01-06T10:00:39.000Z"), "id": 684796738117353472, "text": "It may be January and raining all day, but inside my office it is bright and cheerful from these… https://t.co/WNGGFoCX4K", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.524,37.9833"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1602092690, "name": "Kerrisa Trader", "screen_name": "kerrisa_trader", "lang": "en", "location": "California, USA", "create_at": date("2013-07-17"), "description": "I might not change the world, but I can change your world.", "followers_count": 31, "friends_count": 60, "statues_count": 548 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Rafael, CA", "id": "322ed6e7cc8ff243", "name": "San Rafael", "place_type": "city", "bounding_box": rectangle("-122.592422,37.942583 -122.449033,38.030467") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6041, "countyName": "Marin", "cityID": 668364, "cityName": "San Rafael" } }
+{ "create_at": datetime("2016-01-06T10:00:39.000Z"), "id": 684796738209624064, "text": "Want to work at Kelly Services? We're #hiring in #Tempe, AZ! Click for details: https://t.co/7FsmKaIOHT #Job #KellyJobs #KellyServices #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.9403254,33.4366655"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Tempe", "Job", "KellyJobs", "KellyServices", "Jobs" }}, "user": { "id": 71943150, "name": "TMJ-PHX CstSrv Jobs", "screen_name": "tmj_phx_cstsrv", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2009-09-05"), "description": "Follow this account for geo-targeted Customer Service job tweets in Phoenix, AZ. Need help? Tweet us at @CareerArc!", "followers_count": 360, "friends_count": 281, "statues_count": 310 }, "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-06T10:00:39.000Z"), "id": 684796738297663488, "text": "Smh @lenzzzzo when you played with the Celtics this what you do �� https://t.co/5YXSKrz8LF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 766994929 }}, "user": { "id": 634992955, "name": "BSMASH", "screen_name": "bsmash24", "lang": "en", "location": "Elena Lea Rayos ❤️", "create_at": date("2012-07-13"), "description": "#24 #WTB. #BSMASH IG and SC:smash24001. West.African.Guy", "followers_count": 433, "friends_count": 455, "statues_count": 7625 }, "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-06T10:00:40.000Z"), "id": 684796738465443840, "text": "Hot damn nah it ain't like that������ https://t.co/zILtwpTOn4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 148240818, "name": "Dabzingis", "screen_name": "K00LAiDSLiM", "lang": "en", "location": "The Crib", "create_at": date("2010-05-25"), "description": "I'm just chillin. And I hoop. And I like Nike hoop kicks. sc:j_loading0", "followers_count": 886, "friends_count": 1387, "statues_count": 50018 }, "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-06T10:00:40.000Z"), "id": 684796738750787584, "text": "This semester goal: Stay on campus and stop going back home in between classes 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": 159969105, "name": "Shan Shipman", "screen_name": "_Shann2u", "lang": "en", "location": "null", "create_at": date("2010-06-26"), "description": "Tuskegee University Senior ❤️KRD", "followers_count": 1383, "friends_count": 1772, "statues_count": 28848 }, "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-06T10:00:40.000Z"), "id": 684796738792640512, "text": "@Faith_Guajardo in the cafeteria �� https://t.co/VTVRHBrPYP", "in_reply_to_status": -1, "in_reply_to_user": 605085919, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 605085919 }}, "user": { "id": 2910528662, "name": "D", "screen_name": "MckinneyDelaney", "lang": "en", "location": "null", "create_at": date("2014-11-25"), "description": "dphs", "followers_count": 523, "friends_count": 473, "statues_count": 3971 }, "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-06T10:00:40.000Z"), "id": 684796738813562880, "text": "Advantage Solutions #Job: Specialist Order to Cash (#GrandRapids, MI) https://t.co/dYEvCqTNMJ #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.6680863,42.9633599"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Job", "GrandRapids", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 41654882, "name": "Michigan Jobs", "screen_name": "tmj_mi_usa_jobs", "lang": "en", "location": "Michigan", "create_at": date("2009-05-21"), "description": "Follow this account for geo-targeted Other job tweets in Michigan Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 842, "friends_count": 564, "statues_count": 206 }, "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-06T10:00:40.000Z"), "id": 684796739069423616, "text": "1. #Twitter10k\n2. #DrummondPuddleWatch\n3. Oculus Rift\n4. #Powerball\n5. #NHLAllStar\n\n2016/1/6 11:51 CST #trndnl https://t.co/IvleifByXY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.029,38.8991"), "retweet_count": 0, "lang": "de", "is_retweet": false, "hashtags": {{ "Twitter10k", "DrummondPuddleWatch", "Powerball", "NHLAllStar", "trndnl" }}, "user": { "id": 1180025371, "name": "Trendinalia USA", "screen_name": "trendinaliaUS", "lang": "es", "location": "USA", "create_at": date("2013-02-14"), "description": "Information about Twitter's #TrendingTopics in the United States of America #trndnl", "followers_count": 9510, "friends_count": 56, "statues_count": 734301 }, "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-06T10:00:40.000Z"), "id": 684796740118052864, "text": "I am Kelsey https://t.co/c6KbQKZJh6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 528023882, "name": "-vera.", "screen_name": "WanderingVera", "lang": "en", "location": "Glendale, AZ", "create_at": date("2012-03-17"), "description": "Passionate for change. Enthusiastic about life.\n\nArizona State ☀", "followers_count": 304, "friends_count": 229, "statues_count": 24177 }, "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-06T10:00:40.000Z"), "id": 684796740290146304, "text": "Boy was ass vs Roc https://t.co/Rf5cle2HNo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 828240511, "name": "D.Mart", "screen_name": "6sixsick", "lang": "en", "location": "Houston, TX", "create_at": date("2012-09-16"), "description": "Bench players talking like starters I hate it.", "followers_count": 1038, "friends_count": 369, "statues_count": 32399 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,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-06T10:00:40.000Z"), "id": 684796740663250944, "text": "6. Alex Gordon\n7. Rosa Parks\n8. #AfterTheNovemberElection\n9. Reyes\n10. Prince George\n\n2016/1/6 11:51 CST #trndnl https://t.co/IvleifByXY", "in_reply_to_status": 684796739069423616, "in_reply_to_user": 1180025371, "favorite_count": 0, "coordinate": point("-77.029,38.8991"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AfterTheNovemberElection", "trndnl" }}, "user": { "id": 1180025371, "name": "Trendinalia USA", "screen_name": "trendinaliaUS", "lang": "es", "location": "USA", "create_at": date("2013-02-14"), "description": "Information about Twitter's #TrendingTopics in the United States of America #trndnl", "followers_count": 9510, "friends_count": 56, "statues_count": 734302 }, "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-06T10:00:40.000Z"), "id": 684796740675997696, "text": "I rarely ever have dreams... Well scientist say ur brain dreams all night but u don't remember them..... but anyways i rarely dream", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 776977777, "name": "Khaos", "screen_name": "thatkidkhaos", "lang": "en", "location": "null", "create_at": date("2012-08-23"), "description": "#PhiladelphiaEagles #DetroitPistons#monroeredjackets #05/07/1995", "followers_count": 362, "friends_count": 387, "statues_count": 22282 }, "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-01-06T10:00:40.000Z"), "id": 684796740759760897, "text": "Can you recommend anyone for this #Retail #job? https://t.co/H3kxDYdz5h #Bristol, CT #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.9459111,41.6722922"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "Bristol", "Hiring", "CareerArc" }}, "user": { "id": 28707699, "name": "TMJ-CTG Retail Jobs", "screen_name": "tmj_ctg_retail", "lang": "en", "location": "Greater New Britain, CT", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in New Britain, CT from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 365, "friends_count": 309, "statues_count": 81 }, "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": 9003, "countyName": "Hartford", "cityID": 908420, "cityName": "Bristol" } }
+{ "create_at": datetime("2016-01-06T10:00:40.000Z"), "id": 684796740768296960, "text": "I made a John Mayer, Jack Johnson, and Jason Mraz Pandora station. It's so great", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1130013018, "name": "tuesday", "screen_name": "iamryangrzelak", "lang": "en", "location": "null", "create_at": date("2013-01-28"), "description": "Gonq rat, snap: ryflyguy011\ninsta: rjgrz", "followers_count": 389, "friends_count": 98, "statues_count": 2689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carpentersville, IL", "id": "4f26ae03fcf96fea", "name": "Carpentersville", "place_type": "city", "bounding_box": rectangle("-88.355015,42.102886 -88.238181,42.154302") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17089, "countyName": "Kane", "cityID": 1711358, "cityName": "Carpentersville" } }
+{ "create_at": datetime("2016-01-06T10:00:40.000Z"), "id": 684796740805984256, "text": "@Dayloveme_ @ChmCaroline sfalafljaaglfaljfaBWTE", "in_reply_to_status": 684549562476982272, "in_reply_to_user": 94181780, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 94181780, 1155343410 }}, "user": { "id": 447372099, "name": "r dot ღ", "screen_name": "_Reynaaax3", "lang": "en", "location": "Cobleskill, NY", "create_at": date("2011-12-26"), "description": "Honduran ✨ | @kendricklamar ღ | est. 96.| Cobleskill 19' ΣΩΕ.", "followers_count": 909, "friends_count": 667, "statues_count": 25640 }, "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-01-06T10:00:40.000Z"), "id": 684796740973793280, "text": "@Daneuntamed hmmm, I was gonna say I'm I #n uptown and mines fine", "in_reply_to_status": 684794500171038720, "in_reply_to_user": 2110771, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "n" }}, "user_mentions": {{ 2110771 }}, "user": { "id": 17742232, "name": "dennis", "screen_name": "dennisdire", "lang": "en", "location": "Chicago, IL", "create_at": date("2008-11-29"), "description": "[1989] I tweet for love, justice & satire.", "followers_count": 1367, "friends_count": 254, "statues_count": 36600 }, "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-06T10:00:40.000Z"), "id": 684796741040738305, "text": "Back on the campus grind", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 235782943, "name": "Dominique(Leo Great)", "screen_name": "DomNevaLow", "lang": "en", "location": "Atlanta is the Land of the Fly", "create_at": date("2011-01-08"), "description": "Just livin life really. #MechEngineer #TU16", "followers_count": 652, "friends_count": 516, "statues_count": 33840 }, "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-06T10:00:40.000Z"), "id": 684796741128855552, "text": "I am so beyond ready 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": 488948448, "name": "2-01✨", "screen_name": "CampaignPapi_", "lang": "en", "location": "With my Niece ", "create_at": date("2012-02-10"), "description": ",My Holiday Bitch 06-23♋️, Sc RastaaRhaeaaa,A'kerah holiday 5-29 , August Alsina's WIFE ❤️ Tattoos are LIFE HUN INKMEPLEASE❤️", "followers_count": 751, "friends_count": 899, "statues_count": 16163 }, "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-06T10:00:40.000Z"), "id": 684796741158227969, "text": "this kid needs some collledge https://t.co/0pkjMpY6iP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2783124253, "name": "haley amacker", "screen_name": "haleyamackerr", "lang": "en", "location": "null", "create_at": date("2014-08-31"), "description": "love wins ⚢ lost & insecure", "followers_count": 572, "friends_count": 318, "statues_count": 20830 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, MS", "id": "6e6ed692ac0b92c9", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-89.203925,30.331174 -89.124696,30.383762") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28047, "countyName": "Harrison", "cityID": 2841680, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-06T10:00:40.000Z"), "id": 684796741330284544, "text": "@nypl_labs @nypl Do you have to be located in Atlanta to participate in the #RemixResidency? ����", "in_reply_to_status": -1, "in_reply_to_user": 275031945, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RemixResidency" }}, "user_mentions": {{ 275031945, 17544803 }}, "user": { "id": 121233972, "name": "kc commander", "screen_name": "KCcommander", "lang": "en", "location": "Atlanta, Georgia", "create_at": date("2010-03-08"), "description": "|| I'll be the human inside your social media // #music #art #digital #public // Snap: kc.commander // conducting social media at @atlantasymphony ||", "followers_count": 763, "friends_count": 2043, "statues_count": 1883 }, "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-06T10:00:40.000Z"), "id": 684796741514825728, "text": "#SamsungCES SUHD Quantum Dot HDR - This is TV @samsungmobileca @ Las Vegas Convention Center https://t.co/rttBaO8862", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.15171932,36.13111754"), "retweet_count": 0, "lang": "pt", "is_retweet": false, "hashtags": {{ "SamsungCES" }}, "user_mentions": {{ 15159725 }}, "user": { "id": 187672160, "name": "Mark Childs", "screen_name": "MarkInspired", "lang": "en", "location": "Toronto", "create_at": date("2010-09-06"), "description": "Inspired by creativity, innovation & technology. Passionate about reaching our full potential, paying it forward & LGBT inclusion.", "followers_count": 754, "friends_count": 197, "statues_count": 1301 }, "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-06T10:00:40.000Z"), "id": 684796741581840384, "text": "������������������ https://t.co/yyaRzibKqF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 48914726, "name": "Fred the III", "screen_name": "freddie129", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2009-06-19"), "description": "O'connor 16', OC Volleyball #11,18 years old", "followers_count": 579, "friends_count": 596, "statues_count": 4357 }, "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-06T10:00:40.000Z"), "id": 684796741628116992, "text": "�� https://t.co/ImYfjY3zTF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2900630140, "name": "Rachel Carr", "screen_name": "rachelmcarr16", "lang": "en", "location": "null", "create_at": date("2014-12-01"), "description": "I love My Wonderful Savior - 18 years old", "followers_count": 141, "friends_count": 196, "statues_count": 1198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grafton, OH", "id": "01c775e6160c19be", "name": "Grafton", "place_type": "city", "bounding_box": rectangle("-82.070576,41.263269 -82.001861,41.303413") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39093, "countyName": "Lorain", "cityID": 3931150, "cityName": "Grafton" } }
+{ "create_at": datetime("2016-01-06T10:00:40.000Z"), "id": 684796741674205185, "text": "Just wanna go back to the Philippines and do touristy things����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 816589704, "name": "louise.", "screen_name": "iam_loiis", "lang": "en", "location": "MD", "create_at": date("2012-09-10"), "description": "null", "followers_count": 135, "friends_count": 123, "statues_count": 6615 }, "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-06T10:00:40.000Z"), "id": 684796741690863618, "text": "@ALLANKNGDM happy bday man, keep workin������", "in_reply_to_status": -1, "in_reply_to_user": 40402340, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 40402340 }}, "user": { "id": 2603137577, "name": "marriòn, k", "screen_name": "DarthMiro_", "lang": "en", "location": "ur imagination ", "create_at": date("2014-06-11"), "description": "13th wonder, anu", "followers_count": 973, "friends_count": 737, "statues_count": 50077 }, "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-06T10:00:40.000Z"), "id": 684796741774897153, "text": "You don't wanna speak to me you dont gotta speak to me but just know I won't be pressed to speak to you either", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43272777, "name": "TMH", "screen_name": "DaLegendaryTati", "lang": "en", "location": "Pride Rock ✌", "create_at": date("2009-05-28"), "description": "Dreams do come true, if only we wish hard enough. You can have anything in life if you will sacrifice everything else for it.", "followers_count": 565, "friends_count": 530, "statues_count": 35026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cranbury, NJ", "id": "010cea6f5a910a2b", "name": "Cranbury", "place_type": "city", "bounding_box": rectangle("-74.54946,40.288358 -74.476519,40.338744") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3415520, "cityName": "Cranbury" } }
+{ "create_at": datetime("2016-01-06T10:00:40.000Z"), "id": 684796741850251265, "text": "@Mile_by_Mile @YouTube love this! Check out our #SisterhoodOfTheTravelingPants quiz! https://t.co/aGK0Syn7hh", "in_reply_to_status": 683693176176001026, "in_reply_to_user": 543355769, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SisterhoodOfTheTravelingPants" }}, "user_mentions": {{ 543355769, 10228272 }}, "user": { "id": 377844134, "name": "women.com", "screen_name": "womendotcom", "lang": "en", "location": "Los Angeles, California", "create_at": date("2011-09-21"), "description": "The ultimate destination for women", "followers_count": 3192, "friends_count": 1228, "statues_count": 7343 }, "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-06T10:00:40.000Z"), "id": 684796741946716160, "text": "Big thanks @AAA_Washington for coming to the rescue! It's no wonder you're who everyone calls when they're in a jam! 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": {{ 16072309 }}, "user": { "id": 258511256, "name": "Cantlon", "screen_name": "cantRYAN", "lang": "en", "location": "null", "create_at": date("2011-02-27"), "description": "Hello, my name's Cantlon Ryan. People call me Cantlon Ryan", "followers_count": 102, "friends_count": 188, "statues_count": 210 }, "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-06T10:00:40.000Z"), "id": 684796742114668546, "text": "@singhaccount #leasvegas #paris hotel #eifultower #lasvegasstrip… https://t.co/FLueBpNvDR", "in_reply_to_status": -1, "in_reply_to_user": 2348790098, "favorite_count": 0, "coordinate": point("-115.17147064,36.11235778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "leasvegas", "paris", "eifultower", "lasvegasstrip" }}, "user_mentions": {{ 2348790098 }}, "user": { "id": 2348790098, "name": "@singhaccount", "screen_name": "singhaccount", "lang": "en-gb", "location": "London, England", "create_at": date("2014-02-17"), "description": "Ricky Singhaccount Kalsi®", "followers_count": 64, "friends_count": 307, "statues_count": 4384 }, "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-06T10:00:40.000Z"), "id": 684796742265630720, "text": "Click link in bio \"KEEPNIT WEIRD TEE\" #emerikentshirtcompany #snaptee #urbanclothing #tshirts… https://t.co/Y68wuvCKHe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.6352558,33.16713943"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "emerikentshirtcompany", "snaptee", "urbanclothing", "tshirts" }}, "user": { "id": 2187977982, "name": "wallstreethippiesetc", "screen_name": "ALLAMERICANKIDZ", "lang": "en", "location": "Dallas, Tx", "create_at": date("2013-11-11"), "description": "Go to the link for coolest tees http://wallstreethippies.net", "followers_count": 870, "friends_count": 1925, "statues_count": 6477 }, "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-06T10:00:40.000Z"), "id": 684796742278066176, "text": "Can you recommend anyone for this #Hospitality #job? https://t.co/U6M7WCjcgL #SONIC #Pineville, LA #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.397717,31.333366"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "SONIC", "Pineville", "Hiring", "CareerArc" }}, "user": { "id": 88181812, "name": "TMJ-LA HRTA Jobs", "screen_name": "tmj_la_hrta", "lang": "en", "location": "Louisiana", "create_at": date("2009-11-07"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Louisiana Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 374, "friends_count": 282, "statues_count": 1497 }, "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" } }
diff --git a/asterixdb/asterix-app/data/twitter/real.adm b/asterixdb/asterix-app/data/twitter/real.adm
new file mode 100644
index 0000000..054300a
--- /dev/null
+++ b/asterixdb/asterix-app/data/twitter/real.adm
@@ -0,0 +1,5000 @@
+{ "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("2015-11-23T16:14:07.000Z"), "id": 668945654840848384, "text": "@DandCShow Rex reminds me of my drunk uncle.", "in_reply_to_status": 668945177176879104, "in_reply_to_user": 1056182258, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1056182258 }}, "user": { "id": 15865412, "name": "asterism1", "screen_name": "asterism1", "lang": "en", "location": "New Hampshire", "create_at": date("2008-08-15"), "description": "null", "followers_count": 20, "friends_count": 68, "statues_count": 596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hudson, NH", "id": "01a216c0183feb44", "name": "Hudson", "place_type": "city", "bounding_box": rectangle("-71.463685,42.69879 -71.360807,42.824745") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33011, "countyName": "Hillsborough", "cityID": 3337860, "cityName": "Hudson" } }
+{ "create_at": datetime("2015-11-23T16:14:07.000Z"), "id": 668945654845218817, "text": "I'm at Westminster Apts Dog Park in Marietta, GA https://t.co/l9bQqHVTOr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.48069351,33.90668212"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18137474, "name": "TheComputerMan", "screen_name": "TheComputerMan", "lang": "en", "location": "33.690458,-84.404892", "create_at": date("2008-12-15"), "description": "I am here to SAVE you!", "followers_count": 289, "friends_count": 710, "statues_count": 8031 }, "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("2015-11-23T16:14:07.000Z"), "id": 668945654975188992, "text": "Querido Pasado gracias por las lecciones \nQuerido Futuro, Alla Voy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 3135742528, "name": "Cesar Galeano", "screen_name": "cesargaleano129", "lang": "es", "location": "null", "create_at": date("2015-04-04"), "description": "null", "followers_count": 212, "friends_count": 660, "statues_count": 2387 }, "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("2015-11-23T16:14:07.000Z"), "id": 668945655025549312, "text": "dimelo papi @christophervele stopher", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "sl", "is_retweet": false, "user_mentions": {{ 157940838 }}, "user": { "id": 1460778306, "name": "Ava ♡♡", "screen_name": "avacaiola", "lang": "en", "location": "boston", "create_at": date("2013-05-26"), "description": "just a girl who's a little too obsessed with music || English & Español", "followers_count": 463, "friends_count": 618, "statues_count": 7118 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Framingham, MA", "id": "dbd1411661903bb0", "name": "Framingham", "place_type": "city", "bounding_box": rectangle("-71.496927,42.257263 -71.374193,42.352931") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2524960, "cityName": "Framingham" } }
+{ "create_at": datetime("2015-11-23T16:14:07.000Z"), "id": 668945655163805696, "text": "Throwback https://t.co/WtmvJWY3b4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 912932748, "name": "pudge", "screen_name": "FuckingStowe", "lang": "en", "location": "where my heart is ", "create_at": date("2012-10-29"), "description": "Dopo tutto quello che abbiamo passato Continuo a credere nella magia ? Ebbene sì , lo", "followers_count": 440, "friends_count": 412, "statues_count": 17489 }, "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("2015-11-23T16:14:07.000Z"), "id": 668945655360913408, "text": "What 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": 142790723, "name": "Chillin ™", "screen_name": "zakk__morris", "lang": "en", "location": "IM AROUND ", "create_at": date("2010-05-11"), "description": "Educator Flawless Marketing Group , LLC ( Co- Owner ) @_flawlessMG ♦️ http://acmethursdays.splashthat.com || http://snlplushrva.splashthat.com #FLAWLESSDC", "followers_count": 2627, "friends_count": 1667, "statues_count": 78086 }, "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("2015-11-23T16:14:07.000Z"), "id": 668945655361118208, "text": "Like baby", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 537936376, "name": "BEIGE LOAF", "screen_name": "_Heavens_Gates", "lang": "en", "location": "10UHSEA", "create_at": date("2012-03-27"), "description": "IG @Biege_Loaph I Follow Back Most the Time.", "followers_count": 2695, "friends_count": 2588, "statues_count": 16041 }, "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("2015-11-23T16:14:07.000Z"), "id": 668945655512104960, "text": "Butter & brown sugar", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 370874637, "name": "Kirsten Wilcox", "screen_name": "KirstenWilcox", "lang": "en", "location": "null", "create_at": date("2011-09-09"), "description": "you don't give up just because things get hard", "followers_count": 395, "friends_count": 485, "statues_count": 6993 }, "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("2015-11-23T16:14:07.000Z"), "id": 668945655621165057, "text": "@larrynemecek About to watch ST: First Contact in a theater for #TrekClass. My 1st time! ❤️ to hear all that great background info Dr. Trek!", "in_reply_to_status": -1, "in_reply_to_user": 66884558, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TrekClass" }}, "user_mentions": {{ 66884558 }}, "user": { "id": 1629168836, "name": "Dawn✨", "screen_name": "Acesrule", "lang": "en", "location": "null", "create_at": date("2013-07-28"), "description": "Mom. Teacher. Animal lover. Trekkie. Fangirl.", "followers_count": 143, "friends_count": 136, "statues_count": 2086 }, "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("2015-11-23T16:14:07.000Z"), "id": 668945655667277824, "text": "im sucha cry baby lol.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2347654228, "name": "dej.", "screen_name": "raaevenn", "lang": "en", "location": "null", "create_at": date("2014-02-16"), "description": "sc: mrssrosee", "followers_count": 1183, "friends_count": 1101, "statues_count": 16035 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Centreville, VA", "id": "ffcc53c4a4e7a620", "name": "Centreville", "place_type": "city", "bounding_box": rectangle("-77.479597,38.802143 -77.397429,38.880183") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5114440, "cityName": "Centreville" } }
+{ "create_at": datetime("2015-11-23T16:14:07.000Z"), "id": 668945655931527168, "text": "Omg i love being ignored������ ignore me some more >>>", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2363101796, "name": "layl", "screen_name": "lmccants2746", "lang": "en", "location": "null", "create_at": date("2014-02-26"), "description": "null", "followers_count": 50, "friends_count": 68, "statues_count": 264 }, "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("2015-11-23T16:14:07.000Z"), "id": 668945656115949568, "text": "Hello xAdele touches 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": 235804233, "name": "1202♐️", "screen_name": "_annaivyaf", "lang": "en", "location": "Houston, TX ✨ ", "create_at": date("2011-01-08"), "description": "ass been fat fuck a cabbage ❤️", "followers_count": 984, "friends_count": 610, "statues_count": 59399 }, "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("2015-11-23T16:14:07.000Z"), "id": 668945656350814209, "text": "you know where to 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": 1632300606, "name": "✨", "screen_name": "mandiee_33", "lang": "en", "location": "null", "create_at": date("2013-07-30"), "description": "Blissful", "followers_count": 261, "friends_count": 208, "statues_count": 4504 }, "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("2015-11-23T16:14:07.000Z"), "id": 668945656875233280, "text": "Homecoming anthem right there lol https://t.co/2w2FT9waZ4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 63482234, "name": "Kimberly Hallums", "screen_name": "KimmieIs2Real", "lang": "en", "location": "DC/Baltimore/The Carolinas", "create_at": date("2009-08-06"), "description": "~ Your Favorite MarComm Pro ~ OES-PHA ~ @UofMaryland Ph.D. Student ~ @UofSC and @TrinityDC Grad ~ #Gamecocks #Cowboys #Panthers #Mavs", "followers_count": 1165, "friends_count": 1399, "statues_count": 79067 }, "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.965622,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("2015-11-23T16:14:07.000Z"), "id": 668945657256873984, "text": "I'm at @7Eleven in Carrollton, TX https://t.co/jOF8CeI0iW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.8549352,32.97119962"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 80102659 }}, "user": { "id": 16007068, "name": "Dusty H. ", "screen_name": "dustyhouk", "lang": "en", "location": "Dallas,Texas", "create_at": date("2008-08-26"), "description": "Christian,Son,Brother,Brother In Law,Uncle. Love my Texas Rangers and things with wings!", "followers_count": 281, "friends_count": 86, "statues_count": 19859 }, "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("2015-11-23T16:14:07.000Z"), "id": 668945657609248768, "text": "Oh my goodness ���� https://t.co/MjYUFSJU3l", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2911505056, "name": "Kotaa♚", "screen_name": "___kotaa", "lang": "en", "location": "Maryland ", "create_at": date("2014-12-08"), "description": "licensed hair stylist | 5.10.15 ❤️", "followers_count": 729, "friends_count": 533, "statues_count": 8612 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riva, MD", "id": "7f05cec39c134e6b", "name": "Riva", "place_type": "city", "bounding_box": rectangle("-76.635706,38.922462 -76.571669,38.981466") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24003, "countyName": "Anne Arundel", "cityID": 2466400, "cityName": "Riva" } }
+{ "create_at": datetime("2015-11-23T16:14:07.000Z"), "id": 668945657789620224, "text": "@chriskelly1031 https://t.co/IVlAl2D7WS", "in_reply_to_status": 668932638909071360, "in_reply_to_user": 3890983642, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3890983642 }}, "user": { "id": 344134994, "name": "Cody Munnich", "screen_name": "CodyMunnich", "lang": "en", "location": "631", "create_at": date("2011-07-28"), "description": "speak a little truth and everybody loses their mind", "followers_count": 240, "friends_count": 736, "statues_count": 3316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Terryville, NY", "id": "21a6a0bb33c8181a", "name": "Terryville", "place_type": "city", "bounding_box": rectangle("-73.072409,40.888371 -73.028007,40.931631") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3673352, "cityName": "Terryville" } }
+{ "create_at": datetime("2015-11-23T16:14:07.000Z"), "id": 668945657823109121, "text": "@TasRogers https://t.co/Shih3RBInj", "in_reply_to_status": -1, "in_reply_to_user": 2546429751, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2546429751 }}, "user": { "id": 179676201, "name": "Brian Ufford", "screen_name": "brianufford", "lang": "en", "location": "San Antonio, TX", "create_at": date("2010-08-17"), "description": "Software Developer | Texas Tech Grad | Amateur Footballer/Guitarist | US National Team Supporter", "followers_count": 171, "friends_count": 142, "statues_count": 6495 }, "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:07.000Z"), "id": 668945657831387137, "text": "Happy Birthday #Samibeigi������@SamiBeigi https://t.co/cdfTgCfZP6 https://t.co/NIk0FeRvbv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Samibeigi" }}, "user_mentions": {{ 34889199 }}, "user": { "id": 461688027, "name": "World Music Awards", "screen_name": "WORLDMUSICAWARD", "lang": "en", "location": "null", "create_at": date("2012-01-11"), "description": "The World Music Awards is an international awards ceremony honoring the best-selling most popular recording artists from around the world.", "followers_count": 142022, "friends_count": 631, "statues_count": 3572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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:07.000Z"), "id": 668945657894297600, "text": "#PhoenixLostFound LOST MALTESE TINY WHITE DOG (Bell Rd & 47th Ave): Our tiny white Maltese very old female went… https://t.co/EZEQFfNjPr", "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": 14062 }, "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:07.000Z"), "id": 668945658053791744, "text": "Matthew got me fucked up lmao.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 78706722, "name": "❣Erieeee❣", "screen_name": "boss_erie", "lang": "en", "location": "Heaven With My Daddy", "create_at": date("2009-09-30"), "description": "Cheeks' PIC❤️RIP My Daddy❤️ #LadaveWorld4L #SIUe19 I do hair sometimes ❤️MR❤️", "followers_count": 3049, "friends_count": 2948, "statues_count": 58469 }, "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:07.000Z"), "id": 668945658221498368, "text": "when a cute boy walks by https://t.co/falr1hIrJu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2891328732, "name": "ashley", "screen_name": "ashley_kuehne", "lang": "en", "location": "null", "create_at": date("2014-11-05"), "description": "ravioli ravioli give me the formuoli", "followers_count": 108, "friends_count": 109, "statues_count": 464 }, "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("2015-11-23T16:14:08.000Z"), "id": 668945658296987648, "text": "a burger 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": 2310905352, "name": "туlιn⚡️", "screen_name": "TYLINN69", "lang": "en", "location": "Dallas,Tx", "create_at": date("2014-01-25"), "description": "ig: tylinn", "followers_count": 186, "friends_count": 124, "statues_count": 11199 }, "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:08.000Z"), "id": 668945658443771905, "text": "Well i hope the boys had a fun time vacationing in Maui #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": 99803049, "name": "Lloyd Christmas", "screen_name": "MenaghinoTheDON", "lang": "en", "location": "DENVER - CHICAGO - INDIANA", "create_at": date("2009-12-27"), "description": "null", "followers_count": 379, "friends_count": 493, "statues_count": 7883 }, "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:08.000Z"), "id": 668945658603315200, "text": "wish you could mute people on instagram", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1411532912, "name": "hales", "screen_name": "haleyybrownn_", "lang": "en", "location": "null", "create_at": date("2013-05-07"), "description": "life is good", "followers_count": 244, "friends_count": 310, "statues_count": 2778 }, "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("2015-11-23T16:14:08.000Z"), "id": 668945658607501312, "text": "@Sierra_lott10 same������", "in_reply_to_status": 668939166697304064, "in_reply_to_user": 733334472, "favorite_count": 0, "coordinate": point("-83.99779751,43.44165106"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 733334472 }}, "user": { "id": 563356896, "name": "Cassidy Stoneburner", "screen_name": "cassaroni97", "lang": "en", "location": "null", "create_at": date("2012-04-25"), "description": "Blondes do it better", "followers_count": 341, "friends_count": 201, "statues_count": 8567 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Saginaw Township North, MI", "id": "24aa79dc9093f841", "name": "Saginaw Township North", "place_type": "city", "bounding_box": rectangle("-84.061931,43.436617 -83.949565,43.48891") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26145, "countyName": "Saginaw" } }
+{ "create_at": datetime("2015-11-23T16:14:08.000Z"), "id": 668945658632642569, "text": "ole boa was like he gone pick you up and bring you out here cause I stay 5 mins from mansion . https://t.co/fPFWE8o4DG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2645935370, "name": "✨", "screen_name": "__pocahontaaas", "lang": "en", "location": "at bae house⭐", "create_at": date("2014-07-14"), "description": "living on a natural high..", "followers_count": 1393, "friends_count": 1140, "statues_count": 37142 }, "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("2015-11-23T16:14:08.000Z"), "id": 668945658859167746, "text": "this ain't even a game. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 279245553, "name": "kelseawilson ☀︎", "screen_name": "kelseawilson", "lang": "en", "location": "423 • 865", "create_at": date("2011-04-08"), "description": "the university of tennessee. ♡ 07.05.15", "followers_count": 1612, "friends_count": 539, "statues_count": 64726 }, "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("2015-11-23T16:14:08.000Z"), "id": 668945659261681664, "text": "Run: they want the wrong thing and encourage others to want it too. https://t.co/8YcdmM0yye", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.26215587,36.30063874"), "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": 1290, "friends_count": 1983, "statues_count": 72141 }, "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:08.000Z"), "id": 668945659341443072, "text": "I'm honestly happy, I'm getting drums for Christmas ●ω●\n#MTVStars 5 Seconds Of Summer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MTVStars" }}, "user": { "id": 3075498153, "name": "♡Lex♥", "screen_name": "thepunkchick223", "lang": "en", "location": "Corry, PA", "create_at": date("2015-03-06"), "description": "15 year old girl with a dream. Falling hard for someone ♡♥♡", "followers_count": 238, "friends_count": 315, "statues_count": 2462 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corry, PA", "id": "1fa74a7edcd5497d", "name": "Corry", "place_type": "city", "bounding_box": rectangle("-79.663454,41.907578 -79.610799,41.94426") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42049, "countyName": "Erie", "cityID": 4216296, "cityName": "Corry" } }
+{ "create_at": datetime("2015-11-23T16:14:08.000Z"), "id": 668945659341512704, "text": "every inch of me is bruised.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1693068542, "name": "mushroom princess", "screen_name": "bignoooodle", "lang": "en", "location": "death & taxes", "create_at": date("2013-08-22"), "description": "ten cent pistol", "followers_count": 52, "friends_count": 201, "statues_count": 1661 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bartlett, IL", "id": "7d7d57e9679ec16b", "name": "Bartlett", "place_type": "city", "bounding_box": rectangle("-88.263058,41.935691 -88.148859,42.016598") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1704013, "cityName": "Bartlett" } }
+{ "create_at": datetime("2015-11-23T16:14:08.000Z"), "id": 668945659358154752, "text": "@AddInfoOrg LOL", "in_reply_to_status": 668942114034548736, "in_reply_to_user": 188014427, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 188014427 }}, "user": { "id": 3138691440, "name": "Nora Brockhaus", "screen_name": "noraandkids", "lang": "en", "location": "West Valley City, UT", "create_at": date("2015-04-04"), "description": "Retired nurse, Mother, Grandmother and lover of nature and animals. Also Kevin Bacon Fan and fan of The Following", "followers_count": 66, "friends_count": 154, "statues_count": 5638 }, "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("2015-11-23T16:14:08.000Z"), "id": 668945659500888064, "text": "����������bye https://t.co/gInUFJNc5j", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 222875686, "name": "HuntyLocahontas⌛️", "screen_name": "_buckFITCHES", "lang": "en", "location": "married to Mary ", "create_at": date("2010-12-04"), "description": "❤️✨ #TheBlackGirlArchives ...pro fucking hoe #LoveMySistas ❤️. #LoveMyBlackness #MelaninManiac #ALLblacklivesMATTER #ALLHOESMATTER", "followers_count": 2314, "friends_count": 906, "statues_count": 149879 }, "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:08.000Z"), "id": 668945659530248192, "text": "https://t.co/VuOsXGdQwl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2818939472, "name": "☾", "screen_name": "titlefightmeee", "lang": "en", "location": "laters baby ", "create_at": date("2014-09-18"), "description": "LI,NY ♡18♡ R.I.P Daniel Grym ♡ @fuckmonica_", "followers_count": 987, "friends_count": 1044, "statues_count": 16288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Massapequa, NY", "id": "71cc5b0321940f1a", "name": "Massapequa", "place_type": "city", "bounding_box": rectangle("-73.48709,40.647199 -73.454806,40.692953") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3645986, "cityName": "Massapequa" } }
+{ "create_at": datetime("2015-11-23T16:14:08.000Z"), "id": 668945659681083392, "text": "Low interest+high rent+refugees = red hot https://t.co/DgKwzTs8z9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 945214747, "name": ".BUILD", "screen_name": "dotbuild_gtld", "lang": "en", "location": "The Internet", "create_at": date("2012-11-12"), "description": "The New Internet is here! New names are now available for anyone who builds | From personal projects to skyscrapers to spaceships.", "followers_count": 1466, "friends_count": 1720, "statues_count": 1669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yountville, CA", "id": "b0ccc334a6d569d3", "name": "Yountville", "place_type": "city", "bounding_box": rectangle("-122.384084,38.380206 -122.349882,38.413965") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6055, "countyName": "Napa", "cityID": 686930, "cityName": "Yountville" } }
+{ "create_at": datetime("2015-11-23T16:14:08.000Z"), "id": 668945659958030337, "text": "Greatest Of All Time @ Frewsburg Lawn Mower Sales & Service https://t.co/uEZdb0wsw0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.15729,42.061248"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1045621003, "name": "Glunk", "screen_name": "Glenn_jamin88", "lang": "en", "location": "out here", "create_at": date("2012-12-29"), "description": "A Man Without A Plan", "followers_count": 325, "friends_count": 236, "statues_count": 3868 }, "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": 36013, "countyName": "Chautauqua", "cityID": 3627672, "cityName": "Frewsburg" } }
+{ "create_at": datetime("2015-11-23T16:14:08.000Z"), "id": 668945660088033282, "text": "Movies to see the new Hunger Games ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 349371820, "name": "Morgan Durkacz", "screen_name": "missmorgannn_xo", "lang": "en", "location": "null", "create_at": date("2011-08-05"), "description": "Pittsburgh. WVU. Delta Gamma ⚓️ 13.1", "followers_count": 991, "friends_count": 893, "statues_count": 16629 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baidland, PA", "id": "0011b92e4b5def36", "name": "Baidland", "place_type": "city", "bounding_box": rectangle("-79.975466,40.165529 -79.928171,40.199219") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42125, "countyName": "Washington", "cityID": 4203768, "cityName": "Baidland" } }
+{ "create_at": datetime("2015-11-23T16:14:08.000Z"), "id": 668945660662550528, "text": "��������swear https://t.co/xU6R4BFDbB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 239623873, "name": "〽️❗️ssy", "screen_name": "NiinaHermosa", "lang": "en", "location": "22. Grinding Like Im Broke ✊", "create_at": date("2011-01-17"), "description": "Free Jason ❤️", "followers_count": 1812, "friends_count": 1010, "statues_count": 111484 }, "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("2015-11-23T16:14:08.000Z"), "id": 668945660713017344, "text": "E A sports 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": 3181555893, "name": "s bands", "screen_name": "_isstef", "lang": "en", "location": "@ ur place ", "create_at": date("2015-04-18"), "description": "716 type |IG:mostdefsteff", "followers_count": 293, "friends_count": 176, "statues_count": 6977 }, "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("2015-11-23T16:14:08.000Z"), "id": 668945661157576704, "text": "Guy cardigans & band tees", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 340547700, "name": "Taylor Roberts", "screen_name": "tnrobs", "lang": "en", "location": "VCU || RVA ", "create_at": date("2011-07-22"), "description": "let all that you do be done in love ☼ ☯ ☾rad dude", "followers_count": 931, "friends_count": 464, "statues_count": 22836 }, "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("2015-11-23T16:14:08.000Z"), "id": 668945661258244096, "text": "https://t.co/9KmxO9VtWC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 29263496, "name": "J.A. Rosati", "screen_name": "northsidenotch", "lang": "en", "location": "Louisville, KY", "create_at": date("2009-04-06"), "description": "Retired Pittsburgh Pirates blogger. Non-retired giver of hot takes.\n\nUniversity of Louisville alum.", "followers_count": 5126, "friends_count": 2270, "statues_count": 40156 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Douglass Hills, KY", "id": "fe45fe7b7e38aacc", "name": "Douglass Hills", "place_type": "city", "bounding_box": rectangle("-85.562145,38.223754 -85.523877,38.253048") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2122204, "cityName": "Douglass Hills" } }
+{ "create_at": datetime("2015-11-23T16:14:08.000Z"), "id": 668945661295919105, "text": "One of my #astrovandalistas messages to the #future @therubincenter @utepnews #laser https://t.co/tpfBa4ZdfL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "astrovandalistas", "future", "laser" }}, "user_mentions": {{ 172767190, 36201298 }}, "user": { "id": 1891915454, "name": "Nicolas D Silva", "screen_name": "nicodsilva", "lang": "en", "location": "El Paso, TX", "create_at": date("2013-09-21"), "description": "Art+Science+etc --Passionate twenty-something-- Interested in many questions through the perspective and process of many disciplines.", "followers_count": 56, "friends_count": 274, "statues_count": 209 }, "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("2015-11-23T16:14:08.000Z"), "id": 668945661505720320, "text": "exactly y I only eat rice and broccoli from there�� https://t.co/PSNUWYZINQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1014011215, "name": "kyrie.☯⠀⠀⠀⠀⠀", "screen_name": "KButta_", "lang": "en", "location": "NYC", "create_at": date("2012-12-15"), "description": "lemme find out I got u Amy whined out.❣✨", "followers_count": 9982, "friends_count": 5083, "statues_count": 24713 }, "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("2015-11-23T16:14:08.000Z"), "id": 668945661673545728, "text": "Huge", "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": 349 }, "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:08.000Z"), "id": 668945661698678785, "text": "@ShiquanJohnson you aren't slick ����", "in_reply_to_status": 668944853041094657, "in_reply_to_user": 495199832, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 495199832 }}, "user": { "id": 466194751, "name": "Galina", "screen_name": "lucygooseyyyy", "lang": "en", "location": "null", "create_at": date("2012-01-16"), "description": "null", "followers_count": 1013, "friends_count": 533, "statues_count": 17793 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conyers, GA", "id": "32b057a29a79ad43", "name": "Conyers", "place_type": "city", "bounding_box": rectangle("-84.045724,33.624919 -83.927994,33.702968") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13247, "countyName": "Rockdale", "cityID": 1319336, "cityName": "Conyers" } }
+{ "create_at": datetime("2015-11-23T16:14:08.000Z"), "id": 668945662084444160, "text": "Even with all the drive you still need gas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3312712178, "name": "Brogen Johnson", "screen_name": "Brawgen", "lang": "en", "location": "dis side", "create_at": date("2015-08-11"), "description": "18 years old / the future / Ben Carson 2016 ever been so broke you had sleep for dinner?", "followers_count": 339, "friends_count": 443, "statues_count": 1056 }, "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("2015-11-23T16:14:09.000Z"), "id": 668945662495621120, "text": "Every time the Blues play Buffalo I just remember these ugly, ugly alternate jerseys Steve Ott had to model/wear. https://t.co/QPRkOZQ7rN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 719332226, "name": "Cristiano Simonetta", "screen_name": "CMS_74_", "lang": "en", "location": "#LongLiveTheNote ", "create_at": date("2012-07-26"), "description": "University of Illinois '18. Illini Hockey Broadcaster. I cover the Blues & the Blackhawks for Last Word on Sports.", "followers_count": 2027, "friends_count": 1775, "statues_count": 4896 }, "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:09.000Z"), "id": 668945662554316800, "text": "I'm at Wyrsch Homestead in Kansas City, MO https://t.co/qM1l8KPOsO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.58491571,38.94918879"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 23271156, "name": "Tom Wyrsch", "screen_name": "kcolddog", "lang": "en", "location": "Kansas City, Missouri", "create_at": date("2009-03-07"), "description": "Chairman-Jackson County Democratic Committee. Proud alumnus of the University of Central Missouri.", "followers_count": 376, "friends_count": 498, "statues_count": 12602 }, "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("2015-11-23T16:14:09.000Z"), "id": 668945662592081920, "text": "Paper Towns is 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": 1646295390, "name": "m4d. ☔️", "screen_name": "cravemiranda_", "lang": "en", "location": "null", "create_at": date("2013-08-04"), "description": ":)", "followers_count": 1125, "friends_count": 466, "statues_count": 54800 }, "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("2015-11-23T16:14:09.000Z"), "id": 668945662612926464, "text": "there's a dead body right where i live... @KTVU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15652540 }}, "user": { "id": 57274688, "name": "ROB", "screen_name": "_thatniggarob", "lang": "en", "location": "$F || '93", "create_at": date("2009-07-16"), "description": "null", "followers_count": 408, "friends_count": 273, "statues_count": 18525 }, "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("2015-11-23T16:14:09.000Z"), "id": 668945662617088000, "text": "@_KSophia26 uhummmmmmmmm ��", "in_reply_to_status": 668945601149534208, "in_reply_to_user": 411882990, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user_mentions": {{ 411882990 }}, "user": { "id": 952767901, "name": "MannyTheMammoth", "screen_name": "Curiouschris_12", "lang": "en", "location": "null", "create_at": date("2012-11-16"), "description": "key to life is working harder than your situation", "followers_count": 661, "friends_count": 425, "statues_count": 28167 }, "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("2015-11-23T16:14:09.000Z"), "id": 668945662755540994, "text": "@jad2933 @Hogan2424 I'll fight to the death though", "in_reply_to_status": 668945530915950592, "in_reply_to_user": 506770864, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 506770864, 81005877 }}, "user": { "id": 58813193, "name": "Nick Hogan", "screen_name": "nhoags6", "lang": "en", "location": "null", "create_at": date("2009-07-21"), "description": "Goals without actions are just dreams #YSB #TOPSZN #RedSoxNation #BearDown sc- nickyhoags", "followers_count": 800, "friends_count": 714, "statues_count": 29019 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gibbstown, NJ", "id": "eed0668fae2e491e", "name": "Gibbstown", "place_type": "city", "bounding_box": rectangle("-75.299808,39.807947 -75.236961,39.848024") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34015, "countyName": "Gloucester", "cityID": 3426100, "cityName": "Gibbstown" } }
+{ "create_at": datetime("2015-11-23T16:14:09.000Z"), "id": 668945662935891968, "text": "#PhoenixLostFound LOST HONDA KEY (Roosevelt/75th Ave): 1 Honda key, on a small ring that has a Corona sandal. I… https://t.co/fP3mGOviu9", "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": 14063 }, "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:09.000Z"), "id": 668945662948544512, "text": "if you always bitching about something or talking about how someone else looks or how they live their life somethings wrong wit ya boo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3376791413, "name": "☹", "screen_name": "simraaaann_", "lang": "en", "location": "$tockton, CA", "create_at": date("2015-07-14"), "description": "xo forever", "followers_count": 281, "friends_count": 99, "statues_count": 11977 }, "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("2015-11-23T16:14:09.000Z"), "id": 668945662952652800, "text": "BREAKING: Photographic evidence that ISIS has recruited Americans and are training them in their devious ways. https://t.co/Hhhp2Boide", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1733753772, "name": "Creamsickill", "screen_name": "Creamsickill", "lang": "en", "location": "anunusualplace", "create_at": date("2013-09-05"), "description": "The right wing Looney tune your professor warned you about", "followers_count": 1840, "friends_count": 2096, "statues_count": 18504 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:09.000Z"), "id": 668945663002980352, "text": "@MoLinzee0 thank you. Love you", "in_reply_to_status": 668945564701077505, "in_reply_to_user": 3122487770, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3122487770 }}, "user": { "id": 2697477243, "name": "❃ alexa crawford ❃", "screen_name": "alexaaa_marie15", "lang": "en", "location": "Sam Houston State University", "create_at": date("2014-07-10"), "description": "huntsvegas, tx & I'm in love with my cat", "followers_count": 263, "friends_count": 398, "statues_count": 3103 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:09.000Z"), "id": 668945663007334401, "text": "You're so damn fine though", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 998653206, "name": "Jason Veliz", "screen_name": "jsveliz", "lang": "en", "location": "null", "create_at": date("2012-12-08"), "description": "Snapchat: jsveliz", "followers_count": 194, "friends_count": 164, "statues_count": 646 }, "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("2015-11-23T16:14:09.000Z"), "id": 668945663414153216, "text": "Josh gave me his car with no gas, I am currently stuck in the middle of the road...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 93746022, "name": "Sassy Shorts", "screen_name": "Katie_Bam", "lang": "en", "location": "null", "create_at": date("2009-11-30"), "description": "null", "followers_count": 101, "friends_count": 182, "statues_count": 799 }, "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("2015-11-23T16:14:09.000Z"), "id": 668945663426699268, "text": "Crap. I should have ordered more soup dumplings.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 31180638, "name": "Lily Corvo", "screen_name": "lilygracecorvo", "lang": "en", "location": "New York, NY", "create_at": date("2009-04-14"), "description": "Lily was a princess, she was fair-skinned and precious as a child. She did what she had to do, she had that certain flash every time she smiled (@HuffPostLive)", "followers_count": 689, "friends_count": 2023, "statues_count": 6310 }, "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("2015-11-23T16:14:09.000Z"), "id": 668945663510446080, "text": "@asliwajiha you mean whorelicks?? ������", "in_reply_to_status": 668878557918154753, "in_reply_to_user": 614549360, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 614549360 }}, "user": { "id": 236638920, "name": "Zaal Shar", "screen_name": "sharmazaal", "lang": "en", "location": "Boston, MA", "create_at": date("2011-01-10"), "description": "“It takes an athlete to dance, but an artist to be a dancer.”", "followers_count": 82, "friends_count": 73, "statues_count": 2518 }, "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("2015-11-23T16:14:09.000Z"), "id": 668945663720366080, "text": "@JayyM20 @sanchieson same I'm on episode 10 :)))", "in_reply_to_status": 668945519255777280, "in_reply_to_user": 179208939, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 179208939, 446500088 }}, "user": { "id": 3083530704, "name": "kai", "screen_name": "kaileenyy_", "lang": "en", "location": "TX", "create_at": date("2015-03-14"), "description": "Fxck high school I live dangerously; still tryna graduate tho ☠☠ #MOB #co16 #ILoveMyMom", "followers_count": 1185, "friends_count": 775, "statues_count": 12767 }, "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("2015-11-23T16:14:09.000Z"), "id": 668945663724507136, "text": "@LVPANaNo hard to be everywhere at once.", "in_reply_to_status": 668945375781380097, "in_reply_to_user": 74528662, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 74528662 }}, "user": { "id": 1900545205, "name": "Turkey Cupcake", "screen_name": "PenMinion", "lang": "en", "location": "null", "create_at": date("2013-09-24"), "description": "Instant human! Just add books & coffee. \nPersonal assistant, dabbler in words, alive.", "followers_count": 388, "friends_count": 707, "statues_count": 1733 }, "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("2015-11-23T16:14:09.000Z"), "id": 668945663967797248, "text": "@periodicovzlano “La Tumba”, realidad d presos políticos en Venezuela #LaTumbaEsTortura https://t.co/P1HJuYlQHx https://t.co/Dm6ZeQpAJI", "in_reply_to_status": 668945469301792768, "in_reply_to_user": 290395312, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "LaTumbaEsTortura" }}, "user_mentions": {{ 290395312 }}, "user": { "id": 116936744, "name": "Pedro Correa", "screen_name": "pcorrea7", "lang": "en", "location": "Washington, DC", "create_at": date("2010-02-23"), "description": "Freelancer videographer/editor news and Documentaries, @Telemundo @Univision @Caracol @CNNE", "followers_count": 973, "friends_count": 990, "statues_count": 24390 }, "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("2015-11-23T16:14:09.000Z"), "id": 668945664080908288, "text": "No matter how hard the past, you can always begin again ❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3748260914, "name": "Jocelyn Acosta", "screen_name": "_jocelynacosta", "lang": "en", "location": "Sacramento, CA", "create_at": date("2015-10-01"), "description": "null", "followers_count": 49, "friends_count": 118, "statues_count": 53 }, "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("2015-11-23T16:14:09.000Z"), "id": 668945664349503488, "text": "People going crazy #BlackFridayIn3Words", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BlackFridayIn3Words" }}, "user": { "id": 1041459750, "name": "Adree", "screen_name": "dearestdolphin", "lang": "en", "location": "pacific ocean", "create_at": date("2012-12-27"), "description": "i peed in my shot glass\n#blacklivesmatter", "followers_count": 1388, "friends_count": 1207, "statues_count": 38414 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Broadview, IL", "id": "422cf154312cd404", "name": "Broadview", "place_type": "city", "bounding_box": rectangle("-87.883955,41.834043 -87.812541,41.871475") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1708446, "cityName": "Broadview" } }
+{ "create_at": datetime("2015-11-23T16:14:09.000Z"), "id": 668945664630374400, "text": "#Puyallup, WA #Hospitality #Job: Prep Cook at BJ's Restaurants, Inc. https://t.co/rAztLu2b7J #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.3030362,47.1569453"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Puyallup", "Hospitality", "Job", "Jobs", "Hiring" }}, "user": { "id": 2377573225, "name": "BJ's Restaurants", "screen_name": "WorkAtBJs", "lang": "en", "location": "null", "create_at": date("2014-03-07"), "description": "null", "followers_count": 251, "friends_count": 0, "statues_count": 15908 }, "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("2015-11-23T16:14:09.000Z"), "id": 668945664785563648, "text": "dej loaf is my spirit animal", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2218317878, "name": "jade", "screen_name": "jademackinseyy", "lang": "en", "location": "Arkansas, USA", "create_at": date("2013-11-27"), "description": "| arkansas | nineteen | ♌️❁☽", "followers_count": 636, "friends_count": 1098, "statues_count": 5693 }, "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("2015-11-23T16:14:09.000Z"), "id": 668945665117036544, "text": "Spaceballs second time in 2015 (at @FlixBrewhouseIa in Des Moines, IA w/ @johnpemble) https://t.co/HUr9K4ABwD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.70188686,41.63138091"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1428278132, 14079361 }}, "user": { "id": 2202944930, "name": "Stephanie Maxwell", "screen_name": "StephMMaxwell", "lang": "en", "location": "Des Moines, IA", "create_at": date("2013-11-19"), "description": "I love media and marketing, coffee and Hawaii.", "followers_count": 17, "friends_count": 71, "statues_count": 104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Urbandale, IA", "id": "85ec017be8926e26", "name": "Urbandale", "place_type": "city", "bounding_box": rectangle("-93.814689,41.614643 -93.697511,41.658733") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1979950, "cityName": "Urbandale" } }
+{ "create_at": datetime("2015-11-23T16:14:09.000Z"), "id": 668945665163173888, "text": "All the Big Media ABC, CBS, NBC, FOX, MSNBC & all Newspaper prints are #AfraidOfTrump by calling him out for what He is. Afraid of lawsuits", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AfraidOfTrump" }}, "user": { "id": 59221321, "name": "'Demola Adesida", "screen_name": "demolysis", "lang": "en", "location": "null", "create_at": date("2009-07-22"), "description": "Am a Social Conscious Progressive person. I believe in fairness for all irrespective of Race, Color, Sexualities & Ethnicity. Boy about town. I'm unassuming", "followers_count": 19, "friends_count": 91, "statues_count": 221 }, "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("2015-11-23T16:14:09.000Z"), "id": 668945665792307200, "text": "@DaCcess_co is now on #AskWes #facebook https://t.co/NubarOqMN0 https://t.co/oEoIC7H66d", "in_reply_to_status": -1, "in_reply_to_user": 2663534678, "favorite_count": 0, "coordinate": point("-70.256653,43.659142"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AskWes", "facebook" }}, "user_mentions": {{ 2663534678 }}, "user": { "id": 1869087644, "name": "Susan Potts", "screen_name": "SusanPotts19", "lang": "en", "location": "NYC", "create_at": date("2013-09-15"), "description": "I love being a writer. What I can't stand is the paperwork.", "followers_count": 2123, "friends_count": 2563, "statues_count": 35526 }, "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("2015-11-23T16:14:09.000Z"), "id": 668945665947365377, "text": "@callie0308 Callie PLEASE STOP you are PERFECXCTGCGXSYSYE", "in_reply_to_status": 668945544518066176, "in_reply_to_user": 1628905836, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1628905836 }}, "user": { "id": 732069410, "name": "marlee", "screen_name": "marleesacks", "lang": "en", "location": "null", "create_at": date("2012-08-01"), "description": "null", "followers_count": 657, "friends_count": 1338, "statues_count": 25610 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:09.000Z"), "id": 668945666031427588, "text": "I'm not like them other bitches and I'm not bout to play no games 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": 241959199, "name": "jasmine✝", "screen_name": "___LilJASMINE", "lang": "en", "location": "out of sight ", "create_at": date("2011-01-23"), "description": "17. SENIOR. Focused. Blessed. RIP COKE. FREE MOO.", "followers_count": 4571, "friends_count": 1983, "statues_count": 46311 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden City, MI", "id": "6e4934d73c0cb460", "name": "Garden City", "place_type": "city", "bounding_box": rectangle("-83.379378,42.310149 -83.310862,42.340625") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2631420, "cityName": "Garden City" } }
+{ "create_at": datetime("2015-11-23T16:14:09.000Z"), "id": 668945666161307648, "text": "A date to winter wonderland would 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": 435510639, "name": "cass", "screen_name": "cassieatwood1", "lang": "en", "location": "null", "create_at": date("2011-12-12"), "description": "null", "followers_count": 375, "friends_count": 203, "statues_count": 11668 }, "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("2015-11-23T16:14:10.000Z"), "id": 668945667071561728, "text": "Throwback? https://t.co/9jQdXSNgJq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 87259827, "name": "♡Al♡", "screen_name": "AllyKaliszewski", "lang": "en", "location": "erie", "create_at": date("2009-11-03"), "description": "semi intelligent individual with a shitty sense of humor", "followers_count": 432, "friends_count": 289, "statues_count": 16611 }, "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("2015-11-23T16:14:10.000Z"), "id": 668945667189022722, "text": "Still so much season ahead. Still doesn't mean I'm not upset. Got to get that defense in order", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35829920, "name": "Kendall ", "screen_name": "kendalltrenaud", "lang": "en", "location": "Bloomington, IN", "create_at": date("2009-04-27"), "description": "Singing Hoosier_Music Educator_Marching Hundred Member_Big Red Basketball Band_Kappa Kappa Psi", "followers_count": 509, "friends_count": 668, "statues_count": 6572 }, "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("2015-11-23T16:14:10.000Z"), "id": 668945667478261760, "text": "#PhoenixLostFound Lost Dog (Husky) (Youngtown): Lost Husky very friendly he's White with a few Brown spots and has… https://t.co/R3zkv8Nqpm", "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": 14064 }, "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:10.000Z"), "id": 668945667608350721, "text": "90's music will get you right ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3307431098, "name": "A$AP BOYDZIE", "screen_name": "NoahJames_570", "lang": "en", "location": "The 517", "create_at": date("2015-08-05"), "description": "Making It Out Of Jackson , Student Athlete.", "followers_count": 544, "friends_count": 696, "statues_count": 1809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jackson, MI", "id": "06ab95cfe367eee7", "name": "Jackson", "place_type": "city", "bounding_box": rectangle("-84.493659,42.176663 -84.364192,42.276421") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26075, "countyName": "Jackson", "cityID": 2641420, "cityName": "Jackson" } }
+{ "create_at": datetime("2015-11-23T16:14:10.000Z"), "id": 668945667662807040, "text": "@grossguano POOFIST", "in_reply_to_status": 668896648370585601, "in_reply_to_user": 308270813, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user_mentions": {{ 308270813 }}, "user": { "id": 706333, "name": "Saucy Butterpants", "screen_name": "egypturnash", "lang": "en", "location": "Seattle, WA", "create_at": date("2007-01-25"), "description": "Lackwit gadabout. Unnaturally-colored hair. Draws.\n\nPronouns: she/her/hers.", "followers_count": 1179, "friends_count": 284, "statues_count": 41076 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University District, Seattle", "id": "366de7e7a74e7264", "name": "University District", "place_type": "neighborhood", "bounding_box": rectangle("-122.322392,47.647451 -122.286466,47.673313") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2015-11-23T16:14:10.000Z"), "id": 668945667662966784, "text": "@LilMamaCrying #LilMamaCrying https://t.co/N5dY907zLU", "in_reply_to_status": -1, "in_reply_to_user": 4297832352, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "LilMamaCrying" }}, "user_mentions": {{ 4297832352 }}, "user": { "id": 577743579, "name": "Malcom Flex", "screen_name": "BlackTyrell_", "lang": "en", "location": "Your bed", "create_at": date("2012-05-11"), "description": "LITNATION; We Stay Ready To Squab; Not Handsome; CHS Senior", "followers_count": 3237, "friends_count": 346, "statues_count": 29735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manchester, NH", "id": "5e6cc87c609e3a59", "name": "Manchester", "place_type": "city", "bounding_box": rectangle("-71.512722,42.890919 -71.375439,43.051545") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33011, "countyName": "Hillsborough", "cityID": 3345140, "cityName": "Manchester" } }
+{ "create_at": datetime("2015-11-23T16:14:10.000Z"), "id": 668945667742638081, "text": "Baby daddy, baby daddy ���� #MCM @That_Guy_Shoota https://t.co/l1t5S2Tgon", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MCM" }}, "user_mentions": {{ 157399569 }}, "user": { "id": 768667406, "name": "❤️", "screen_name": "Nisha_Shae", "lang": "en", "location": "my Thoughts", "create_at": date("2012-08-19"), "description": "You can't judge me off of what I post bcuz I post a lot of shit so that makes me bi-postER you will never kno the real ME‼️‼️ -Nisha Shae", "followers_count": 899, "friends_count": 850, "statues_count": 14842 }, "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("2015-11-23T16:14:10.000Z"), "id": 668945667843338240, "text": "I keep it real shawty I can't be committed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2909950095, "name": "Adaaa☯", "screen_name": "Amlrx3", "lang": "en", "location": "null", "create_at": date("2014-12-07"), "description": "Dubington", "followers_count": 509, "friends_count": 242, "statues_count": 16485 }, "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("2015-11-23T16:14:10.000Z"), "id": 668945667872571392, "text": "@annakriistine_ I know��", "in_reply_to_status": 668945215441399808, "in_reply_to_user": 968686854, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 968686854 }}, "user": { "id": 413532403, "name": "Tia", "screen_name": "_TiaYost", "lang": "en", "location": "null", "create_at": date("2011-11-15"), "description": "winging life one day at a time", "followers_count": 954, "friends_count": 660, "statues_count": 14422 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anoka, MN", "id": "6660cfc4fdc7284d", "name": "Anoka", "place_type": "city", "bounding_box": rectangle("-93.42293,45.179248 -93.357522,45.239954") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27003, "countyName": "Anoka", "cityID": 2701720, "cityName": "Anoka" } }
+{ "create_at": datetime("2015-11-23T16:14:10.000Z"), "id": 668945668187144192, "text": "*adds to cyber Monday shopping list* https://t.co/upB8SfJqw2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1867688880, "name": "a$hmocha", "screen_name": "HerChillASH", "lang": "en", "location": "Chicago ", "create_at": date("2013-09-15"), "description": "✨HadiyaWorld✨ LongLivePizza #DoItFor32 somewhere boolin', coolin', keepin' to myself.", "followers_count": 697, "friends_count": 515, "statues_count": 10656 }, "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:10.000Z"), "id": 668945668241801216, "text": "Aziz Sancar commends the multicultural atmosphere of Chapel Hill, also its sports. He loves the 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": 1968176760, "name": "Stephanie Lamm", "screen_name": "slamm_5", "lang": "en", "location": "lost on public transportation", "create_at": date("2013-10-17"), "description": "Assistant City Editor @DailyTarHeel. Multimedia. Data. Policy. @unc 2017.", "followers_count": 708, "friends_count": 1435, "statues_count": 2487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chapel Hill, NC", "id": "23ebe811ea93fd41", "name": "Chapel Hill", "place_type": "city", "bounding_box": rectangle("-79.083153,35.869376 -78.968887,35.969507") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37135, "countyName": "Orange", "cityID": 3711800, "cityName": "Chapel Hill" } }
+{ "create_at": datetime("2015-11-23T16:14:10.000Z"), "id": 668945668375977987, "text": "I'm at PK's House in OH https://t.co/7FvFKOy1pX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.43339229,41.61515531"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 49618314, "name": "Paul Keller", "screen_name": "pk44094", "lang": "en", "location": "null", "create_at": date("2009-06-22"), "description": "null", "followers_count": 45, "friends_count": 152, "statues_count": 2532 }, "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("2015-11-23T16:14:10.000Z"), "id": 668945668459884544, "text": "Sacramento is a strange place, very different from Los Angeles. By I do really love fall here. I'm… https://t.co/vegxqQhRAn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.42386141,38.56061295"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 749940942, "name": "Rebecca Boisclair", "screen_name": "Beccatonix", "lang": "en", "location": "Los Angeles", "create_at": date("2012-08-10"), "description": "Biggest Pentatonix fan, girl scout, IB student, mezzo-soprano, animal lover, supporter of animal rights, books are better, future nurse and Deaf interpreter.", "followers_count": 27, "friends_count": 51, "statues_count": 824 }, "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:10.000Z"), "id": 668945668698824704, "text": "Keep on receiving them scholarships", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 450101734, "name": "Chibu Nwafor", "screen_name": "RED_B0TT0M_B0SS", "lang": "en", "location": "Abuja, Nigeria✈Dallas Flexas", "create_at": date("2011-12-29"), "description": "Igbo boy in the States. •Ijeoma", "followers_count": 878, "friends_count": 548, "statues_count": 17874 }, "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("2015-11-23T16:14:10.000Z"), "id": 668945668967415809, "text": "@sailorbee @danikaharrod BIRD THE FUCK DOWN", "in_reply_to_status": 668944481111031808, "in_reply_to_user": 36206861, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 36206861, 1176067598 }}, "user": { "id": 2583550982, "name": "Hey Asuka!", "screen_name": "notcimi", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "real weeb shit", "followers_count": 301, "friends_count": 150, "statues_count": 17508 }, "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("2015-11-23T16:14:10.000Z"), "id": 668945669063884800, "text": "@brendakaycomer Love the hammer. Thanks.", "in_reply_to_status": 668855834261237760, "in_reply_to_user": 331926636, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 331926636 }}, "user": { "id": 79784897, "name": "Jason Halle", "screen_name": "jasonhalle", "lang": "en", "location": "Pompano Beach, FL", "create_at": date("2009-10-04"), "description": "Blogger, Writer, Social #Justice and #Probate Reform Advocate, \nDiversity, Books, NOH8.", "followers_count": 46332, "friends_count": 49076, "statues_count": 68515 }, "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("2015-11-23T16:14:10.000Z"), "id": 668945669197983744, "text": "3 https://t.co/0ZZlEtgB0R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2215263800, "name": "dësus❦", "screen_name": "agbcabeyo", "lang": "en", "location": "null", "create_at": date("2013-11-25"), "description": "preachers preach to different crowds.", "followers_count": 11516, "friends_count": 541, "statues_count": 130088 }, "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("2015-11-23T16:14:10.000Z"), "id": 668945669298720768, "text": "bout to heat it up and throw some ice cream 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": 415282632, "name": "Kerri", "screen_name": "_MyNameIsKerri_", "lang": "en", "location": "LT | BK", "create_at": date("2011-11-17"), "description": "Kingsway Dragon Alumni '14. 19. LIU Brooklyn '18.⚽️", "followers_count": 630, "friends_count": 1216, "statues_count": 8398 }, "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:10.000Z"), "id": 668945669315362820, "text": "thank you for showing me true love exists. happy \"official\" year to my everything ��❤️�� @ Cental Park NY https://t.co/grKDsR8fSz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.96869513,40.78047176"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 455394350, "name": "☻/❁/☯", "screen_name": "courtneyleonee", "lang": "en", "location": "null", "create_at": date("2012-01-04"), "description": "dogs heal my soul", "followers_count": 320, "friends_count": 251, "statues_count": 17327 }, "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("2015-11-23T16:14:10.000Z"), "id": 668945669407698944, "text": "I find money owed to you, current or past business & organizations. Free search. 15%finders fee if money found. PM me today @pchenderson_LV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2768557979 }}, "user": { "id": 2768557979, "name": "Paula Henderson", "screen_name": "pchenderson_LV", "lang": "en", "location": "Las Vegas, NV 89106", "create_at": date("2014-09-12"), "description": "I alert YOU to #money owed to you. Request free search for current or past business, 15% suggested finders fee of monies found. DM me today", "followers_count": 1952, "friends_count": 4464, "statues_count": 5897 }, "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("2015-11-23T16:14:10.000Z"), "id": 668945669693009921, "text": "@MaryKayCabot leave the kid alone. Spreading rumors.", "in_reply_to_status": 668916738780696576, "in_reply_to_user": 35107066, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 35107066 }}, "user": { "id": 2369439054, "name": "Pam Baron", "screen_name": "baron_pam", "lang": "en", "location": "Cleveland, Ohio", "create_at": date("2014-03-02"), "description": "Medical device", "followers_count": 591, "friends_count": 1917, "statues_count": 5416 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rocky River, OH", "id": "ad1fd8a17d633e77", "name": "Rocky River", "place_type": "city", "bounding_box": rectangle("-81.874568,41.449742 -81.823152,41.490668") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3968056, "cityName": "Rocky River" } }
+{ "create_at": datetime("2015-11-23T16:14:10.000Z"), "id": 668945669835526144, "text": "2 weeks ima have a truck on some shit . Nigga grind young nigga", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1155639926, "name": "The Kid That Did....", "screen_name": "Jose_The_Pilot", "lang": "en", "location": "Peen's World , DreamVille", "create_at": date("2013-02-06"), "description": "Straight Outta Wharton Tx", "followers_count": 354, "friends_count": 221, "statues_count": 15214 }, "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("2015-11-23T16:14:10.000Z"), "id": 668945669852372992, "text": "⚔ YNTGR ⚔ https://t.co/eXjhFWM51O", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2892906001, "name": "Khavon Thomas⚡️", "screen_name": "KhavonT", "lang": "en", "location": "Somewhere making money ", "create_at": date("2014-11-07"), "description": "Young Nigga Tryna Get Rich {YNTGR} ‼️", "followers_count": 235, "friends_count": 207, "statues_count": 1427 }, "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:10.000Z"), "id": 668945670070517760, "text": "Happy birthday mike ! Enjoy it and ball out this year������ @mikeugarte24", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1031905957 }}, "user": { "id": 2182149603, "name": "Brendon Scarlett", "screen_name": "brendons24", "lang": "en", "location": "null", "create_at": date("2013-11-15"), "description": "FBG NLMB South brunswick C/O 2017", "followers_count": 430, "friends_count": 595, "statues_count": 1316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kendall Park, NJ", "id": "037919e78aa84204", "name": "Kendall Park", "place_type": "city", "bounding_box": rectangle("-74.586075,40.394292 -74.541524,40.433525") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3436660, "cityName": "Kendall Park" } }
+{ "create_at": datetime("2015-11-23T16:14:10.000Z"), "id": 668945670124912640, "text": "@deegotfans lmao", "in_reply_to_status": 668945550905995264, "in_reply_to_user": 3068369732, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3068369732 }}, "user": { "id": 3169364190, "name": ".$", "screen_name": "alayzia7", "lang": "en", "location": "yo momma house", "create_at": date("2015-04-23"), "description": "$pliff", "followers_count": 281, "friends_count": 172, "statues_count": 4230 }, "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("2015-11-23T16:14:10.000Z"), "id": 668945670158422016, "text": "\"We have been lied to. Racism is completely false.\"\n\n| @tj_sings \n\n#BreakingBread \n#TracingRacism", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BreakingBread", "TracingRacism" }}, "user_mentions": {{ 427953421 }}, "user": { "id": 92308073, "name": "Jason Chesnut", "screen_name": "crazypastor", "lang": "en", "location": "Baltimore, Maryland", "create_at": date("2009-11-24"), "description": "| jesus-follower | liberal/radical | anti-racist | feminist | aspiring theologian | ordained in the @ELCA | works @TheSlateProject | #BlackLivesMatter |", "followers_count": 3303, "friends_count": 1572, "statues_count": 17097 }, "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:10.000Z"), "id": 668945670217207808, "text": "Buddy the Eld what's your favorite color?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1648357802, "name": "Macus Murrphy", "screen_name": "murphhh44", "lang": "en", "location": "Gilbert, AZ", "create_at": date("2013-08-05"), "description": "Mesquite High School Football, Hockey #21", "followers_count": 567, "friends_count": 371, "statues_count": 14416 }, "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("2015-11-23T16:14:10.000Z"), "id": 668945670448005120, "text": "@iRUN_DMC when you was in black", "in_reply_to_status": 668945462699892736, "in_reply_to_user": 61952830, "favorite_count": 0, "coordinate": point("-75.24498559,39.90271552"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 61952830 }}, "user": { "id": 843574214, "name": "[aв]", "screen_name": "MrTable4One", "lang": "en", "location": "The Attic Future Was Counting Money In.", "create_at": date("2012-09-24"), "description": "This page contains subject matter and language that may be disturbing to some viewers, viewer discretion is advised.", "followers_count": 437, "friends_count": 437, "statues_count": 41613 }, "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:11.000Z"), "id": 668945671102324736, "text": "@ofgeography I mean. I am already full-time weeping about goalies. So. That won't be difficult to achieve.", "in_reply_to_status": 668945433985708034, "in_reply_to_user": 746055972, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 746055972 }}, "user": { "id": 17582746, "name": "K. A. K.", "screen_name": "keannka", "lang": "en", "location": "Fair Haven, NJ", "create_at": date("2008-11-23"), "description": "hockey (rangers/pens/CBJ/stars/NWHL), baseball (red sox), teaching (middle grades/higher ed), feminism (intersectional)", "followers_count": 233, "friends_count": 784, "statues_count": 7789 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fair Haven, NJ", "id": "cb532703cd8de015", "name": "Fair Haven", "place_type": "city", "bounding_box": rectangle("-74.054422,40.347982 -74.018992,40.370794") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3422440, "cityName": "Fair Haven" } }
+{ "create_at": datetime("2015-11-23T16:14:11.000Z"), "id": 668945671341383680, "text": "Calling all the basic bitches! Calling all the basic bitches!\n\nYou're basic.....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 991959709, "name": "King Junta", "screen_name": "Jeffd_Up", "lang": "en", "location": "Columbus, OH", "create_at": date("2012-12-05"), "description": "If I get off to a rocky start, I'll always have Rocky finish.\nOHIO STATE. METS. RAIDERS. CAVALIERS. DEVILS. REAL MADRID. \nMy views are my own.", "followers_count": 154, "friends_count": 124, "statues_count": 13071 }, "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("2015-11-23T16:14:11.000Z"), "id": 668945671379021825, "text": "@anna_603 \nAmner Guzman\nPittsburg CA\n8/17/00\nSophomore PHS\n\nHobbies\n-Photography\n-Listening to Kanye \n-business https://t.co/bGcAPFmy5Z", "in_reply_to_status": 668943140796010496, "in_reply_to_user": 799814348, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 799814348 }}, "user": { "id": 2564077320, "name": "Amner Guzman", "screen_name": "amnerg73", "lang": "en", "location": "Pittsburg, CA", "create_at": date("2014-06-12"), "description": "entrepreneurship, Fata Morgana, photography ,Phs.", "followers_count": 209, "friends_count": 108, "statues_count": 692 }, "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("2015-11-23T16:14:11.000Z"), "id": 668945671454593025, "text": "Vote and share! Check out DORITOS® Crash the Super Bowl WebFlix N Chill https://t.co/J0y5OTc9AE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2244562368, "name": "ASparC", "screen_name": "Amarieesc", "lang": "en", "location": "null", "create_at": date("2013-12-13"), "description": "null", "followers_count": 55, "friends_count": 173, "statues_count": 50 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:11.000Z"), "id": 668945671727263744, "text": "James Bay on repeat tonight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1319022312, "name": "RAG", "screen_name": "ragan_22", "lang": "en", "location": "null", "create_at": date("2013-03-31"), "description": "prolly something bout Bieber", "followers_count": 338, "friends_count": 523, "statues_count": 2084 }, "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("2015-11-23T16:14:11.000Z"), "id": 668945671777423360, "text": "@DrGbates Well said, Doc. #LeadwithGiants", "in_reply_to_status": 668945480399720448, "in_reply_to_user": 3090707622, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LeadwithGiants" }}, "user_mentions": {{ 3090707622 }}, "user": { "id": 296427687, "name": "Kyle Murray", "screen_name": "TheKyleMurray", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-05-10"), "description": "Social Media Manager. Tar Heel from NC. Focus on #healthcare #seniorcare #homecare. Dodgeballer.", "followers_count": 1549, "friends_count": 1376, "statues_count": 4027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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:11.000Z"), "id": 668945671823753216, "text": "Me when my lacrosse stick comes in . https://t.co/KYPCEZxrW0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 597003176, "name": "playboy.my", "screen_name": "mjorge720", "lang": "en", "location": "the field", "create_at": date("2012-06-01"), "description": "character is what you do when no one is watching .", "followers_count": 383, "friends_count": 487, "statues_count": 7747 }, "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("2015-11-23T16:14:11.000Z"), "id": 668945672255705088, "text": "@Divergent338 WHAT ABOUT DENSALIA", "in_reply_to_status": 668943208110407681, "in_reply_to_user": 2670549326, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2670549326 }}, "user": { "id": 55325084, "name": "Jess", "screen_name": "dayisfading", "lang": "en", "location": "the curtains of the waterfall ", "create_at": date("2009-07-09"), "description": "east tennessee girl on the crystal coast. vol for life. orange&white til the end. go preds go. pharmacist. wannabe writer, professional fangirl.", "followers_count": 192, "friends_count": 152, "statues_count": 177759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morehead City, NC", "id": "01fea1ed0b8cdbb6", "name": "Morehead City", "place_type": "city", "bounding_box": rectangle("-76.90017,34.705977 -76.670032,34.762852") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37031, "countyName": "Carteret", "cityID": 3744320, "cityName": "Morehead City" } }
+{ "create_at": datetime("2015-11-23T16:14:11.000Z"), "id": 668945672297517056, "text": "@yestaleen very sexy pic https://t.co/93JEx3MhxS", "in_reply_to_status": -1, "in_reply_to_user": 1034299759, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1034299759 }}, "user": { "id": 2941175348, "name": "Larry Shober", "screen_name": "ShoberLs", "lang": "en", "location": "Burleson, TX", "create_at": date("2014-12-23"), "description": "null", "followers_count": 42, "friends_count": 1337, "statues_count": 5849 }, "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("2015-11-23T16:14:11.000Z"), "id": 668945672389787649, "text": "@bailey_findley5 cmonnn", "in_reply_to_status": 668945497399435264, "in_reply_to_user": 2497361660, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2497361660 }}, "user": { "id": 2773817511, "name": "neeks", "screen_name": "NicoCirc_", "lang": "en", "location": "wheelin", "create_at": date("2014-09-17"), "description": "livin the Dream -PSHS- /i occasionally put hockey pucks in nets/ \\_ . _/\nbailey findley is a beaut @bailey_findley5", "followers_count": 285, "friends_count": 284, "statues_count": 6612 }, "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("2015-11-23T16:14:11.000Z"), "id": 668945672482127872, "text": "@MouthFullOf_Kay she told me ������", "in_reply_to_status": 668945618736287744, "in_reply_to_user": 73707271, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 73707271 }}, "user": { "id": 2632255061, "name": "candace♡", "screen_name": "Xocandace_", "lang": "en", "location": "null", "create_at": date("2014-06-22"), "description": "... @simply_keishawn❤️", "followers_count": 1209, "friends_count": 985, "statues_count": 14584 }, "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("2015-11-23T16:14:11.000Z"), "id": 668945672670941184, "text": "RUBIO IS THE NEW CHOICE OF THE RINO ELITE SORRY JEB YOU TRIED #MakeAmericaGreatAgain https://t.co/DTSzBNYCWV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MakeAmericaGreatAgain" }}, "user": { "id": 235337115, "name": "JOHN STRONG", "screen_name": "STRONG_OPED", "lang": "en", "location": "null", "create_at": date("2011-01-07"), "description": "CURMUDGEON\nOLD GEEZER SNARK\n#PJNET #TEAPARTY #CCOT #TCOT\nRATIONAL, CONSERVATIVE, REALIST.", "followers_count": 7436, "friends_count": 7742, "statues_count": 48180 }, "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("2015-11-23T16:14:11.000Z"), "id": 668945672717123584, "text": "@jaymegurd WHAT idk how to feel about this!", "in_reply_to_status": 668945438515576832, "in_reply_to_user": 729586370, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 729586370 }}, "user": { "id": 57833247, "name": "Kate Casey", "screen_name": "kateecasey", "lang": "en", "location": "Instagram.com/kate_casey ", "create_at": date("2009-07-17"), "description": "I'm Kate. 20. NYC. snapchat: kate_casey", "followers_count": 653, "friends_count": 282, "statues_count": 14431 }, "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("2015-11-23T16:14:11.000Z"), "id": 668945672750669824, "text": "damn right https://t.co/JszgNV1MTG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 309776116, "name": "Sc: indeniyal ✨", "screen_name": "__indeNIYAl", "lang": "en", "location": "baltimore.", "create_at": date("2011-06-02"), "description": "college gal ✨", "followers_count": 1637, "friends_count": 1133, "statues_count": 87033 }, "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:11.000Z"), "id": 668945672952004608, "text": "Jesus no https://t.co/L8ILjokg4H", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 997127672, "name": "bre gotti❄️", "screen_name": "noigbre", "lang": "en", "location": "guccis world. ", "create_at": date("2012-12-08"), "description": "imma bad lil thick jit. rip to my 9/28/15❤️", "followers_count": 1408, "friends_count": 824, "statues_count": 99948 }, "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("2015-11-23T16:14:11.000Z"), "id": 668945674071879680, "text": "@DaRisk_co is now on #AskShawnAndCamila #facebook https://t.co/CAKIlr8vPm", "in_reply_to_status": -1, "in_reply_to_user": 2684961746, "favorite_count": 0, "coordinate": point("-83.920914,35.960491"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AskShawnAndCamila", "facebook" }}, "user_mentions": {{ 2684961746 }}, "user": { "id": 1511701219, "name": "Patricia Cooper", "screen_name": "PatCooperUnique", "lang": "en", "location": "Chicago", "create_at": date("2013-06-12"), "description": "It's the #soul's duty to be loyal to its own #desires. It must #abandon itself to its master #passion.", "followers_count": 1601, "friends_count": 1769, "statues_count": 37887 }, "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("2015-11-23T16:14:11.000Z"), "id": 668945674210295808, "text": "aren't they all lol https://t.co/zpQ5zfXCbz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2878418428, "name": "1⃣1⃣", "screen_name": "_ChopGotTheMop", "lang": "en", "location": "in the fredo ", "create_at": date("2014-11-15"), "description": "whats best for everyone else aint always whats best for you ❗ | ️Virgogang ♍ |", "followers_count": 1031, "friends_count": 815, "statues_count": 50291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newburgh, NY", "id": "0047e81d7db6c89a", "name": "Newburgh", "place_type": "city", "bounding_box": rectangle("-74.116963,41.487948 -74.004603,41.519289") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36071, "countyName": "Orange", "cityID": 3650034, "cityName": "Newburgh" } }
+{ "create_at": datetime("2015-11-23T16:14:11.000Z"), "id": 668945674361147392, "text": "@essymarie32 new phone who dis", "in_reply_to_status": 668939952428724224, "in_reply_to_user": 119570454, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 119570454 }}, "user": { "id": 850277216, "name": "onion", "screen_name": "LMA01234", "lang": "en", "location": "ALTUM", "create_at": date("2012-09-27"), "description": "*parody account* for booking @animeclubprez@aol.com", "followers_count": 308, "friends_count": 144, "statues_count": 2724 }, "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("2015-11-23T16:14:11.000Z"), "id": 668945674378022912, "text": "I'm about to be up all night. shoutout to that 4 hour nap ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 381833686, "name": "❥", "screen_name": "cvssidy_", "lang": "en", "location": "Sc: @mookeybaby", "create_at": date("2011-09-28"), "description": "keep your heart 3stacks", "followers_count": 1745, "friends_count": 346, "statues_count": 71584 }, "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("2015-11-23T16:14:11.000Z"), "id": 668945674449227776, "text": "Want to work at CVS Health? We're #hiring in #Irvine, CA! Click for details: https://t.co/tEcFw97vla #Healthcare #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.7946942,33.6839473"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Irvine", "Healthcare", "Job", "Jobs", "CareerArc" }}, "user": { "id": 21298660, "name": "LA Healthcare Jobs", "screen_name": "tmj_lax_health", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-02-19"), "description": "Follow this account for geo-targeted Healthcare job tweets in Los Angeles, CA. Need help? Tweet us at @CareerArc!", "followers_count": 840, "friends_count": 307, "statues_count": 2066 }, "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("2015-11-23T16:14:11.000Z"), "id": 668945674503778305, "text": "@kj_fetishmodel @adrianachechik �� Interesting", "in_reply_to_status": 668945559772921856, "in_reply_to_user": 2337874352, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2337874352, 1219304198 }}, "user": { "id": 364379239, "name": "Joe Rios", "screen_name": "TheWildCardJoe", "lang": "en", "location": "Arvada,CO", "create_at": date("2011-08-29"), "description": "Navy vet, badass video game player, movie buff, football fanatic, world traveler. Adult film connoisseur #FREAKYCREW member part time scoundrel #P4YP King", "followers_count": 1354, "friends_count": 426, "statues_count": 28138 }, "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("2015-11-23T16:14:11.000Z"), "id": 668945674537467904, "text": "@nicoleeefloress yes (:", "in_reply_to_status": 668945633168879616, "in_reply_to_user": 841327039, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 841327039 }}, "user": { "id": 933186763, "name": "Sandra.", "screen_name": "sandralion", "lang": "en", "location": "Washington", "create_at": date("2012-11-07"), "description": "null", "followers_count": 413, "friends_count": 423, "statues_count": 10160 }, "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("2015-11-23T16:14:11.000Z"), "id": 668945674935754752, "text": "Palo Alto Medical Foundation: Director of Physician Services - Mountain View (#MountainView, CA) https://t.co/J6n8fR1Nsv #Healthcare #Job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.0838511,37.3860517"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MountainView", "Healthcare", "Job" }}, "user": { "id": 20827274, "name": "SF Healthcare Jobs", "screen_name": "tmj_sfo_health", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-02-13"), "description": "Follow this account for geo-targeted Healthcare job tweets in San Francisco, CA. Need help? Tweet us at @CareerArc!", "followers_count": 766, "friends_count": 335, "statues_count": 1204 }, "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("2015-11-23T16:14:11.000Z"), "id": 668945675023814657, "text": "Yes!, indeed! ��. Tonight with some MNF‼️�� https://t.co/FAPA3gzIs9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2190690676, "name": "ron robes", "screen_name": "RobesRon", "lang": "en", "location": "Alabama (coastal)", "create_at": date("2013-11-21"), "description": "Living life through my passions; family, friends, music, art, travel, food AND good drink!✨", "followers_count": 1012, "friends_count": 950, "statues_count": 2770 }, "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("2015-11-23T16:14:12.000Z"), "id": 668945675149844480, "text": "Gone fuck that bitch and get it over with", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 770404082, "name": "Keepin Up With toria", "screen_name": "TrustnoneToriaa", "lang": "en", "location": "TRAYS WORLD.♡", "create_at": date("2012-08-20"), "description": "My Life I Live It How I Want | Diamonds Are My Bestfriend | BOOJIE OR WHATEVER | 2⃣0⃣1⃣5⃣ | Money And School is my main focus✏️", "followers_count": 847, "friends_count": 744, "statues_count": 54645 }, "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("2015-11-23T16:14:12.000Z"), "id": 668945675317551104, "text": "@Jeanette_V99 I deadass had bad dreams all night and I didn't even do it", "in_reply_to_status": 668942513768656897, "in_reply_to_user": 2668791123, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2668791123 }}, "user": { "id": 1384767758, "name": "nwalon", "screen_name": "nwalon", "lang": "en", "location": "Staten Island, NY", "create_at": date("2013-04-27"), "description": "Oscar Mazanek❤️", "followers_count": 434, "friends_count": 292, "statues_count": 14790 }, "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:12.000Z"), "id": 668945675376177153, "text": "Interested in a #Nursing #job near #Rochester, MI? This could be a great fit: https://t.co/wnOPmHXk9m #RN #Detroit https://t.co/AaelNwxCg9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.1492553,42.6807334"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "Rochester", "RN", "Detroit" }}, "user": { "id": 1076984880, "name": "Crittenton Jobs", "screen_name": "CrittentonJobs", "lang": "en", "location": "Rochester, Michigan", "create_at": date("2013-01-10"), "description": "Working for Crittenton Hospital Medical Center is more than just a job. It’s a career. Follow us for current opportunities.", "followers_count": 225, "friends_count": 249, "statues_count": 3311 }, "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("2015-11-23T16:14:12.000Z"), "id": 668945675468431360, "text": "We're #hiring! Click to apply: Seasonal Cosmetics - Beauty Advisor, Part Time: Macy's Tacoma Mall - https://t.co/yfB4WT667D #Tacoma, WA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.4678568,47.2164165"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Tacoma" }}, "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": 607, "friends_count": 0, "statues_count": 9111 }, "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("2015-11-23T16:14:12.000Z"), "id": 668945675598467072, "text": "@gabbs98_ can't. the ice melts everytime.", "in_reply_to_status": 668945316456955905, "in_reply_to_user": 2168869243, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2168869243 }}, "user": { "id": 3041880397, "name": "holly he", "screen_name": "hollyhmxb", "lang": "en", "location": "null", "create_at": date("2015-02-25"), "description": "next time you're at starbucks, why not order a double shot... of COMPASSION", "followers_count": 202, "friends_count": 202, "statues_count": 1183 }, "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("2015-11-23T16:14:12.000Z"), "id": 668945675732807684, "text": "Indiana lost and I am going to eat dinner alone and mourn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36208995, "name": "Could be Wyatt", "screen_name": "WHarmon18", "lang": "en", "location": "null", "create_at": date("2009-04-28"), "description": "all the world is doing its best night and day to make me everybody else", "followers_count": 425, "friends_count": 889, "statues_count": 20329 }, "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("2015-11-23T16:14:12.000Z"), "id": 668945675745382402, "text": "My mom do this every year ... Then Christmas get here she in the mood .....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24040812, "name": "COIN STAR ⭐️", "screen_name": "Nerdiction", "lang": "en", "location": "Florida, USA", "create_at": date("2009-03-12"), "description": "FENG SHUI YA MENTAL .. yuup i love me some @nickiminaj", "followers_count": 4018, "friends_count": 1094, "statues_count": 258550 }, "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("2015-11-23T16:14:12.000Z"), "id": 668945676105977856, "text": "You danced all night girl you deserve it ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2324423660, "name": "QueenRee", "screen_name": "lilaiko", "lang": "en", "location": "null", "create_at": date("2014-02-02"), "description": "HTX. ♊", "followers_count": 569, "friends_count": 417, "statues_count": 14406 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:12.000Z"), "id": 668945676152139776, "text": "HAHAAAHA https://t.co/comrbGT0Zy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 2486618702, "name": "MELLSSSSS", "screen_name": "mamimellss", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-05-09"), "description": "'XOXOXO", "followers_count": 406, "friends_count": 296, "statues_count": 15385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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:12.000Z"), "id": 668945676169052161, "text": "Gawd @ Coco Asian Bistro & Bar https://t.co/X5wePYOfkZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.13111955,26.09808877"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 23851715, "name": "Tahhh Tenacious", "screen_name": "Chim3ric__", "lang": "en", "location": "marinating in fancy sauce", "create_at": date("2009-03-11"), "description": "Tah. 24. Homo. taken. #BIFC ig: chim3ric__ snapchat: tahx10", "followers_count": 933, "friends_count": 878, "statues_count": 47415 }, "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("2015-11-23T16:14:12.000Z"), "id": 668945676173078528, "text": "I can only agree with two lol https://t.co/iYw4do7v1B", "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", "followers_count": 1013, "friends_count": 650, "statues_count": 10641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:12.000Z"), "id": 668945676198395904, "text": "How could a guy ever hit a woman?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 478795340, "name": "Tyler", "screen_name": "ItsPeake", "lang": "en", "location": "Hoonigan ", "create_at": date("2012-01-30"), "description": "Tyler from Les Stanford | 18 | Snapchat: Tpeake69 | @Kkoveleski ❤️", "followers_count": 1876, "friends_count": 1196, "statues_count": 21882 }, "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("2015-11-23T16:14:12.000Z"), "id": 668945676324106241, "text": "caparison still the best juice smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3242447263, "name": "1/10✨", "screen_name": "mjeyl_", "lang": "en", "location": "Eden Prairie, MN", "create_at": date("2015-06-11"), "description": "sc: xoxo_mz& m_mohamed18 // ig: mjeylani_ ||NEW ACCOUNT||\n#blacklivesmatter", "followers_count": 157, "friends_count": 172, "statues_count": 725 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eden Prairie, MN", "id": "a1a25de9f5efa69b", "name": "Eden Prairie", "place_type": "city", "bounding_box": rectangle("-93.520913,44.799239 -93.397928,44.893162") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2718116, "cityName": "Eden Prairie" } }
+{ "create_at": datetime("2015-11-23T16:14:12.000Z"), "id": 668945676370231297, "text": "@RayvenJuanengo yes thank", "in_reply_to_status": 668940617376927744, "in_reply_to_user": 66055966, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 66055966 }}, "user": { "id": 208259217, "name": "Ryan", "screen_name": "ridrizaj", "lang": "en", "location": "San Diego, CA", "create_at": date("2010-10-26"), "description": "sorry i was late i can’t conceptualize time", "followers_count": 215, "friends_count": 263, "statues_count": 23184 }, "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:12.000Z"), "id": 668945676437319680, "text": "This #CustomerService #job might be a great fit for you: Client Services Specialist - https://t.co/8dFnIVW91D #SanDiego, CA #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.1572551,32.7153292"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CustomerService", "job", "SanDiego", "Hiring" }}, "user": { "id": 71946353, "name": "TMJ-SAN CstSrv Jobs", "screen_name": "tmj_san_cstsrv", "lang": "en", "location": "San Diego, CA", "create_at": date("2009-09-05"), "description": "Follow this account for geo-targeted Customer Service job tweets in San Diego, CA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 330, "friends_count": 284, "statues_count": 109 }, "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:12.000Z"), "id": 668945676567318529, "text": "Need to get faded up", "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": "''", "screen_name": "WhataburgerLife", "lang": "en", "location": "GHS 2016", "create_at": date("2011-01-27"), "description": "you can tell your kids you knew me when you go see my movies/shows", "followers_count": 555, "friends_count": 328, "statues_count": 20470 }, "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("2015-11-23T16:14:12.000Z"), "id": 668945676680757249, "text": "@teddyp10 of #DBS & #dbsfamily ��: @jordankeyser @ Duke's Barber Shop https://t.co/A7yS2QfVjm", "in_reply_to_status": -1, "in_reply_to_user": 44525882, "favorite_count": 0, "coordinate": point("-73.78069106,42.65998885"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DBS", "dbsfamily" }}, "user_mentions": {{ 44525882, 55588606 }}, "user": { "id": 436919330, "name": "✄Duke's Barber Shop✄", "screen_name": "dukesofalbany", "lang": "en", "location": "Albany, NY", "create_at": date("2011-12-14"), "description": "The Finest of Master Barbering", "followers_count": 542, "friends_count": 408, "statues_count": 6277 }, "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": 36001, "countyName": "Albany", "cityID": 3601000, "cityName": "Albany" } }
+{ "create_at": datetime("2015-11-23T16:14:12.000Z"), "id": 668945676688998402, "text": "@shannope_ honestly I can't feel anything anymore.", "in_reply_to_status": 668945583395049472, "in_reply_to_user": 863888744, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 863888744 }}, "user": { "id": 228254313, "name": "jos", "screen_name": "teenvonnegut", "lang": "en", "location": "UOP '19", "create_at": date("2010-12-18"), "description": "she died doing what she loved: dying.", "followers_count": 170, "friends_count": 109, "statues_count": 5793 }, "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("2015-11-23T16:14:12.000Z"), "id": 668945676714119168, "text": "Traigo un kilo de purita cocaina", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 1364934336, "name": "edgar", "screen_name": "chiri1234_la", "lang": "en", "location": "Canoga Park, CA", "create_at": date("2013-04-19"), "description": "me gusta estar alegre", "followers_count": 247, "friends_count": 307, "statues_count": 7348 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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:12.000Z"), "id": 668945676722532352, "text": "This #IT #job might be a great fit for you: Software Engineer I (JUNE 2016 CCP) - https://t.co/jKb4jmW703 #FindYourFit #Topeka, KS #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.6890185,39.0558235"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "IT", "job", "FindYourFit", "Topeka", "Hiring" }}, "user": { "id": 2444385740, "name": "Payless ShoeSource", "screen_name": "Payless_Jobs", "lang": "en", "location": "Topeka, KS", "create_at": date("2014-04-14"), "description": "You know who you are. And you know what moves you. That's why your chosen career should fit you like your favorite pair of shoes #findyourfit", "followers_count": 32, "friends_count": 1, "statues_count": 187 }, "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("2015-11-23T16:14:12.000Z"), "id": 668945676777033728, "text": "@westonbrowne1 love ya too dude thanks for always being there", "in_reply_to_status": 668942121655635972, "in_reply_to_user": 841041918, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 841041918 }}, "user": { "id": 1724093857, "name": "Matthew Blake", "screen_name": "m_blake8", "lang": "en", "location": "Fair Oaks, CA", "create_at": date("2013-09-02"), "description": "Just trying to be happy", "followers_count": 466, "friends_count": 386, "statues_count": 2547 }, "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("2015-11-23T16:14:12.000Z"), "id": 668945677020471296, "text": "Yeah, two minutes on photoshop. SMH https://t.co/9MArjopZ5s", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174920192, "name": "tyler!", "screen_name": "muhfuggintylah", "lang": "en", "location": "null", "create_at": date("2010-08-04"), "description": "this is a parody account* anything tweeted is not affiliated with Tyler. (unless its good)", "followers_count": 667, "friends_count": 501, "statues_count": 7372 }, "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:12.000Z"), "id": 668945677041459200, "text": "It was an amazing show with amazing people https://t.co/rlRjH8nM3Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2408023667, "name": "micayla bragg", "screen_name": "micaylabragg", "lang": "en", "location": "null", "create_at": date("2014-03-13"), "description": "FL | JW", "followers_count": 32, "friends_count": 58, "statues_count": 28 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florida Ridge, FL", "id": "02a67e042018b506", "name": "Florida Ridge", "place_type": "city", "bounding_box": rectangle("-80.4474,27.557127 -80.356133,27.607531") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12061, "countyName": "Indian River", "cityID": 1223050, "cityName": "Florida Ridge" } }
+{ "create_at": datetime("2015-11-23T16:14:12.000Z"), "id": 668945677842571266, "text": "Spineless bitches & backless dresses", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 340761012, "name": "Gabrielllle", "screen_name": "gabz_95", "lang": "en", "location": "Lake View, Chicago", "create_at": date("2011-07-22"), "description": "photographer // Chicago", "followers_count": 756, "friends_count": 514, "statues_count": 10955 }, "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:12.000Z"), "id": 668945677901127681, "text": "@RileyAdrienne yesss I'm excited :)", "in_reply_to_status": 668945320865218560, "in_reply_to_user": 2500546592, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2500546592 }}, "user": { "id": 1308852097, "name": "amanda", "screen_name": "zeigleramanda", "lang": "en", "location": "washington ", "create_at": date("2013-03-27"), "description": "@Connor_DeMarco / what would Leslie Knope do?", "followers_count": 586, "friends_count": 271, "statues_count": 21182 }, "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("2015-11-23T16:14:12.000Z"), "id": 668945678006030336, "text": "I'm not even big on turkey. I will smash a plethora of sides though....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26159589, "name": "Khitam", "screen_name": "khitamjabr", "lang": "en", "location": "KC, MO", "create_at": date("2009-03-23"), "description": "Confessions of a Palestinian-American Muslim....I am the next supreme. Wrangler of @ericajoymusic", "followers_count": 467, "friends_count": 662, "statues_count": 33879 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Kansas City, MO", "id": "19869ae7b22efc3b", "name": "North Kansas City", "place_type": "city", "bounding_box": rectangle("-94.587373,39.123228 -94.529967,39.152953") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29047, "countyName": "Clay", "cityID": 2953102, "cityName": "North Kansas City" } }
+{ "create_at": datetime("2015-11-23T16:14:12.000Z"), "id": 668945678089916416, "text": "Robert Half Finance & Accounting: Director/Mgr Financial Reporting (#MAUMEE, OH) https://t.co/gt8gyyEfsD #Finance #RHFAJobs #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.6538244,41.5628294"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MAUMEE", "Finance", "RHFAJobs", "Job", "Jobs" }}, "user": { "id": 791258761, "name": "RH Finance Jobs", "screen_name": "RHFAJobs", "lang": "en", "location": "null", "create_at": date("2012-08-30"), "description": "null", "followers_count": 173, "friends_count": 0, "statues_count": 32553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maumee, OH", "id": "ccf3f9123267d97d", "name": "Maumee", "place_type": "city", "bounding_box": rectangle("-83.718691,41.527192 -83.606563,41.593643") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39095, "countyName": "Lucas", "cityID": 3948342, "cityName": "Maumee" } }
+{ "create_at": datetime("2015-11-23T16:14:12.000Z"), "id": 668945678228258820, "text": "I'm really hoping Rih's music hits because she kept us waiting for entirely too long", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1499563302, "name": "Earth, Wind, & Shade", "screen_name": "KevyD_PhD", "lang": "en", "location": "Emory Alum | Austin, TX | AΦA", "create_at": date("2013-06-10"), "description": "Everytime you see me I look like I hit the lotto twice #igotareallybigteam #folkswannapopoff", "followers_count": 400, "friends_count": 698, "statues_count": 11584 }, "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:12.000Z"), "id": 668945678245232640, "text": "@alejandrobetLOZ @Mikeespencerr you hear alejandros new remix \"Diego Im drunk\"", "in_reply_to_status": 668945072713519104, "in_reply_to_user": 1721001840, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1721001840, 2832819633 }}, "user": { "id": 2780731668, "name": "Peter", "screen_name": "petey_moose", "lang": "en", "location": "Brooklyn", "create_at": date("2014-08-30"), "description": "Xavier '17", "followers_count": 105, "friends_count": 93, "statues_count": 628 }, "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:12.000Z"), "id": 668945678412869632, "text": "#Khamenei #Putin #ISIS #Assed, They are terrorist they must go.. https://t.co/2TlNiy8SK6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Khamenei", "Putin", "ISIS", "Assed" }}, "user": { "id": 56932891, "name": "فرشا خاوران", "screen_name": "MYTTT", "lang": "en", "location": "null", "create_at": date("2009-07-14"), "description": "Human Right Activists", "followers_count": 564, "friends_count": 1618, "statues_count": 3513 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:12.000Z"), "id": 668945678450585601, "text": "See our latest #SanFrancisco, CA #job and click to apply: Android Developer - https://t.co/gVIE9or3ge #NettempsJobs #IT #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.4274582,37.7982061"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SanFrancisco", "job", "NettempsJobs", "IT", "Hiring", "CareerArc" }}, "user": { "id": 20827150, "name": "San Francisco IT Job", "screen_name": "tmj_sfo_it", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-02-13"), "description": "Follow this account for geo-targeted Software Dev. - General/IT job tweets in San Francisco, CA. Need help? Tweet us at @CareerArc!", "followers_count": 699, "friends_count": 360, "statues_count": 1018 }, "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:12.000Z"), "id": 668945678513520640, "text": "Join the Advantage Sales and Marketing team! See our latest #Sales #job opening here: https://t.co/BF4Bo9wbEo #FortCollins, CO #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.084423,40.5852602"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "job", "FortCollins", "Hiring" }}, "user": { "id": 22362640, "name": "TMJ-CO Sales Jobs", "screen_name": "tmj_co_sales", "lang": "en", "location": "Colorado", "create_at": date("2009-03-01"), "description": "Follow this account for geo-targeted Sales job tweets in Colorado Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 367, "friends_count": 303, "statues_count": 285 }, "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("2015-11-23T16:14:12.000Z"), "id": 668945678836584448, "text": "Uws and associated by 97th Amsterdam https://t.co/Yc3e2CSe6z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.93925904,40.74552473"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16004672, "name": "lostjuly", "screen_name": "lostjuly", "lang": "en", "location": "queens, bayside, NY", "create_at": date("2008-08-26"), "description": "I'm bad at this game", "followers_count": 17, "friends_count": 32, "statues_count": 769 }, "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("2015-11-23T16:14:12.000Z"), "id": 668945678870011905, "text": "So you gone use my shit..... thats cute ☺☺☺☺☺☺ https://t.co/5oCpanKJaA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2872572826, "name": "Tayy ♈♈", "screen_name": "__tayywadd", "lang": "en", "location": "null", "create_at": date("2014-11-11"), "description": "HTX | I'm an artist | RIP Nana |", "followers_count": 441, "friends_count": 472, "statues_count": 3061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:12.000Z"), "id": 668945678870016000, "text": "\"Victory is a thousand times sweeter when you're the underdog.\" -Jenny Han", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2224355208, "name": "k. murray", "screen_name": "kaelin_bryce", "lang": "en", "location": "Paloma Creek, TX", "create_at": date("2013-11-30"), "description": "#9", "followers_count": 766, "friends_count": 588, "statues_count": 382 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paloma Creek, TX", "id": "0123115a0c1b23ed", "name": "Paloma Creek", "place_type": "city", "bounding_box": rectangle("-96.946655,33.220749 -96.929913,33.22994") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4854868, "cityName": "Paloma Creek" } }
+{ "create_at": datetime("2015-11-23T16:14:12.000Z"), "id": 668945679016792064, "text": "When your manager tells you he'll promote you to women's team lead if you eat a grape & swallow it. Ughh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 86548110, "name": "april", "screen_name": "gothpotato", "lang": "en", "location": "Chicago", "create_at": date("2009-10-31"), "description": "bio major. Shedd aquarium. ⚰", "followers_count": 803, "friends_count": 341, "statues_count": 12976 }, "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:12.000Z"), "id": 668945679134404609, "text": "@TomCrean that took two timeouts to come up with?", "in_reply_to_status": -1, "in_reply_to_user": 32127835, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 32127835 }}, "user": { "id": 342519947, "name": "Cooper Hill", "screen_name": "chris_moooney", "lang": "en", "location": "null", "create_at": date("2011-07-25"), "description": "I can do all things through Christ who strengthens me. Philippians 4:13", "followers_count": 531, "friends_count": 635, "statues_count": 11407 }, "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("2015-11-23T16:14:12.000Z"), "id": 668945679172116480, "text": "@Cherir_lamour light depression?", "in_reply_to_status": 668877946556448774, "in_reply_to_user": 29135024, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 29135024 }}, "user": { "id": 24187647, "name": "Emile Session", "screen_name": "ES37", "lang": "en", "location": "Crown Heights", "create_at": date("2009-03-13"), "description": "All progress is precarious and the solution of one problem brings us face to face with another problem.", "followers_count": 146, "friends_count": 196, "statues_count": 10914 }, "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:13.000Z"), "id": 668945679499194368, "text": "@RAIDERS YOU NEED TO BEAT TITANS ON SUNDAY !!! MUST WIN !!! #BeGreat", "in_reply_to_status": -1, "in_reply_to_user": 16332223, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BeGreat" }}, "user_mentions": {{ 16332223 }}, "user": { "id": 2211892472, "name": "Jordan Atallah", "screen_name": "atll1", "lang": "en", "location": "California", "create_at": date("2013-11-23"), "description": "#TeamFollowBack", "followers_count": 61, "friends_count": 122, "statues_count": 2755 }, "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("2015-11-23T16:14:13.000Z"), "id": 668945679528554496, "text": "@WWERoadDogg Life throws so much at you, you have to take it as it comes!", "in_reply_to_status": 668788674113511424, "in_reply_to_user": 229333456, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 229333456 }}, "user": { "id": 49725851, "name": "TheGamingPegasus187", "screen_name": "GaminPegasus187", "lang": "en", "location": "Great Falls, Montana", "create_at": date("2009-06-22"), "description": "I'm that Gaming Pegasus from the youtubes. Podcaster, pro wrestling and MMA fan. And sworn enemy of stupidity #PegasusforBigBrother18", "followers_count": 449, "friends_count": 2006, "statues_count": 27212 }, "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("2015-11-23T16:14:13.000Z"), "id": 668945679549636609, "text": "@VegasJill77 so annoying... #periscope deleting educational scopes! watch on @KatchHQ: no senseless censorship there https://t.co/wq4MvJfmYC", "in_reply_to_status": 668944508004909056, "in_reply_to_user": 30414800, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "periscope" }}, "user_mentions": {{ 30414800, 1355770982 }}, "user": { "id": 1258367311, "name": "Zhana Vrangalova PhD", "screen_name": "DrZhana", "lang": "en", "location": "NYU", "create_at": date("2013-03-10"), "description": "Sex researcher, writer, speaker. NYU prof #HumanSexuality. @CasualSexProj creator. I tweet abt new sex research & broadcast daily on http://t.co/qMgGDjk66m.", "followers_count": 6538, "friends_count": 2368, "statues_count": 8642 }, "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:13.000Z"), "id": 668945679788609536, "text": "Now if I could shake the blues... ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 99692128, "name": "Amber L. Lillard", "screen_name": "AmberLBlackwell", "lang": "en", "location": "Colorado Springs, CO", "create_at": date("2009-12-27"), "description": "living life thru a camera lens", "followers_count": 89, "friends_count": 349, "statues_count": 5837 }, "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("2015-11-23T16:14:13.000Z"), "id": 668945679939571712, "text": "ain't none to spill brud�� https://t.co/ybijpYoqsG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2823433650, "name": "laTrenty️", "screen_name": "iam_tjohnson21", "lang": "en", "location": "null", "create_at": date("2014-09-20"), "description": "THS 21 || SC: iam_tjohnson21", "followers_count": 437, "friends_count": 349, "statues_count": 11312 }, "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("2015-11-23T16:14:13.000Z"), "id": 668945679994089472, "text": "@TuriansCanDance same", "in_reply_to_status": 668945438125502466, "in_reply_to_user": 213227102, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 213227102 }}, "user": { "id": 809208018, "name": "Meeseeks Look at Me!", "screen_name": "sorryjzargo", "lang": "en", "location": "Campbellsville, KY", "create_at": date("2012-09-07"), "description": "Existence is pain for a Social Justice Meeseeks; he/him", "followers_count": 234, "friends_count": 550, "statues_count": 31893 }, "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("2015-11-23T16:14:13.000Z"), "id": 668945680056979456, "text": "@NJDavidD point is EVERYTHING goes down from time to time, enterprise or otherwise.", "in_reply_to_status": 668944959689646080, "in_reply_to_user": 101582811, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 101582811 }}, "user": { "id": 550091576, "name": "Mike Brandes, CTS", "screen_name": "MikeBrandesAV", "lang": "en", "location": "Minneapolis, MN", "create_at": date("2012-04-10"), "description": "A/V, IT and other thoughts. Freelance Writer. Consultant. Former scratch golfer and alligator trapper", "followers_count": 937, "friends_count": 142, "statues_count": 7223 }, "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("2015-11-23T16:14:13.000Z"), "id": 668945680078012417, "text": "What a win for D Manning. #MauiInvitational #rockchalk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MauiInvitational", "rockchalk" }}, "user": { "id": 109325211, "name": "Brad Weiler", "screen_name": "bweiler10", "lang": "en", "location": "Topeka,KS", "create_at": date("2010-01-28"), "description": "I am a Father of two. A husband to a great wife. instagram accounts @cweiler4 @bweiler10 bweiler1@yahoo.com Kansas Jayhawks Sporting KC", "followers_count": 249, "friends_count": 410, "statues_count": 2882 }, "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("2015-11-23T16:14:13.000Z"), "id": 668945680224808960, "text": "Can you recommend anyone for this #Hospitality #job? https://t.co/93sJakXssQ #Pleasanton, CA #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.927267,37.694832"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Pleasanton", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 22489947, "name": "TMJ-SJC HRTA Jobs", "screen_name": "tmj_sjc_hrta", "lang": "en", "location": "San Jose, CA", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in San Jose, CA. Need help? Tweet us at @CareerArc!", "followers_count": 358, "friends_count": 295, "statues_count": 749 }, "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("2015-11-23T16:14:13.000Z"), "id": 668945680459689985, "text": "@FIROfficial hi guy's it's a 1d fan and I have a question what are your names Justin Charlie Michael or Robby any of these names bye ��������������", "in_reply_to_status": -1, "in_reply_to_user": 42261441, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 42261441 }}, "user": { "id": 1631415144, "name": "Nadia Anaya", "screen_name": "nadia_2401", "lang": "en", "location": "null", "create_at": date("2013-07-29"), "description": "null", "followers_count": 87, "friends_count": 417, "statues_count": 40 }, "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.887803 -88.081435,41.942768") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1711332, "cityName": "Carol Stream" } }
+{ "create_at": datetime("2015-11-23T16:14:13.000Z"), "id": 668945680790982656, "text": "#Adele #TaylorSwift @Adele be killing @taylorswift13 #Adele is so refined unlike that wanna B model fake ass chic https://t.co/UsWtEZfhuo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Adele", "TaylorSwift", "Adele" }}, "user_mentions": {{ 184910040, 17919972 }}, "user": { "id": 148085398, "name": "Gossip Guy", "screen_name": "aucae12", "lang": "en", "location": "null", "create_at": date("2010-05-25"), "description": "Writing truth, don't want everyone to know behave urself, grow up, do good, B A positive member of society, not a pot smoking fuck head, it's not cool", "followers_count": 18, "friends_count": 98, "statues_count": 652 }, "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("2015-11-23T16:14:13.000Z"), "id": 668945680820527104, "text": "breaks my heart to hear when such awful things happen to such amazing people!! love you both!!��@katiecarruso @gracee_corcoran", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2566925996, 2608763919 }}, "user": { "id": 2680652502, "name": "✨Neve✨", "screen_name": "neve_spellman", "lang": "en", "location": "Drexel Hill, PA", "create_at": date("2014-07-25"), "description": "the happiest people don't have the best of everything, they just make the best of everything. 6/11/15", "followers_count": 235, "friends_count": 210, "statues_count": 2670 }, "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("2015-11-23T16:14:13.000Z"), "id": 668945681055281153, "text": "To the professor that just canceled class on the day before thanksgiving... 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": 548218763, "name": "Danny Owens", "screen_name": "HelloDannyOwens", "lang": "en", "location": "Tacoma", "create_at": date("2012-04-08"), "description": "Photographer & other things. Contact: hello@dannyowens.co", "followers_count": 480, "friends_count": 248, "statues_count": 4178 }, "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("2015-11-23T16:14:13.000Z"), "id": 668945681063768068, "text": "This painting is taking forever.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 100701816, "name": "Kelso ❤", "screen_name": "Cosmokelso0o", "lang": "en", "location": "Silent Hill", "create_at": date("2009-12-30"), "description": "Rebuilding...", "followers_count": 171, "friends_count": 153, "statues_count": 19880 }, "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("2015-11-23T16:14:13.000Z"), "id": 668945681080446976, "text": "This summer was crazy tbh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 310599097, "name": "TriniJack", "screen_name": "Jackymxxn", "lang": "en", "location": "In the cut", "create_at": date("2011-06-03"), "description": "You already know how we rockin'. | Tyrone & Michelle | XX | 122021995", "followers_count": 784, "friends_count": 583, "statues_count": 25649 }, "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:13.000Z"), "id": 668945681327910912, "text": "@Lovelyhose1 I want to see you with me make love to me I'm Luis", "in_reply_to_status": -1, "in_reply_to_user": 3091644454, "favorite_count": 0, "coordinate": point("-117.0889478,32.6299194"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3091644454 }}, "user": { "id": 3666385459, "name": "Luis Varelajr", "screen_name": "LVarelajr", "lang": "en", "location": "null", "create_at": date("2015-09-23"), "description": "I love IT my dj to love ther at the bar wily wollys2$sp no tuesday night", "followers_count": 39, "friends_count": 312, "statues_count": 63 }, "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("2015-11-23T16:14:13.000Z"), "id": 668945681927671808, "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": 372080257, "name": "Geezy✈️", "screen_name": "TrentGeezy", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-09-11"), "description": "765 ✈️ 812 ✈️ 310", "followers_count": 9833, "friends_count": 457, "statues_count": 66067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Lafayette, IN", "id": "832ac0728b3e26ac", "name": "West Lafayette", "place_type": "city", "bounding_box": rectangle("-86.999431,40.406761 -86.86053,40.511402") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18157, "countyName": "Tippecanoe", "cityID": 1882862, "cityName": "West Lafayette" } }
+{ "create_at": datetime("2015-11-23T16:14:13.000Z"), "id": 668945682112380929, "text": "I got NSYNC! Can We Guess If You Were An NSYNC Or Backstreet Boys Fan? https://t.co/8EdYR3cJ9I via kelleylocke @buzzfeed THEY WON.", "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": 33697653, "name": "Sarina May Waldon♣️", "screen_name": "sarinamay93", "lang": "en", "location": "Sioux City,IA.", "create_at": date("2009-04-20"), "description": "How YOU Doin?~Jimberlake,TV,Music,My Dogs,NYC,Reeses,Chicken Strips,TBell,Lazy Days,Bonus Features, Concerts,Chapstick.~NO, you don't lick your boom boom down.", "followers_count": 1424, "friends_count": 1777, "statues_count": 70622 }, "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("2015-11-23T16:14:13.000Z"), "id": 668945682175127552, "text": "I really need a lot of black stuff for 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": 2864704191, "name": "Lil Andrea ❤", "screen_name": "Annie_FineAss", "lang": "en", "location": "null", "create_at": date("2014-11-06"), "description": "I'm Way To Cute To Be Somebody's Boo ❤", "followers_count": 577, "friends_count": 894, "statues_count": 13559 }, "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:13.000Z"), "id": 668945682343067649, "text": "shoukdve never have broke it .. https://t.co/bPExWjSoFZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2769101295, "name": "❣", "screen_name": "iimtaee_7", "lang": "en", "location": "null", "create_at": date("2014-09-13"), "description": "fw myself !", "followers_count": 1027, "friends_count": 567, "statues_count": 24778 }, "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("2015-11-23T16:14:13.000Z"), "id": 668945682582122497, "text": "My trust is 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": 1078400179, "name": "Javi", "screen_name": "javieerrr_", "lang": "en", "location": "6", "create_at": date("2013-01-10"), "description": "90210", "followers_count": 399, "friends_count": 250, "statues_count": 1403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wallington, NJ", "id": "def44d7d147e501a", "name": "Wallington", "place_type": "city", "bounding_box": rectangle("-74.121151,40.841495 -74.096269,40.863791") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3476490, "cityName": "Wallington" } }
+{ "create_at": datetime("2015-11-23T16:14:13.000Z"), "id": 668945682590486528, "text": "Me and my boi from yesterday ���� @ Jackson https://t.co/c96g4h2FRo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.3536,40.1054"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2566142033, "name": "Evan", "screen_name": "evan_purdy21", "lang": "en", "location": "Freehold ", "create_at": date("2014-05-26"), "description": "DDE basketball - #21 Freehold Football", "followers_count": 243, "friends_count": 177, "statues_count": 830 }, "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("2015-11-23T16:14:13.000Z"), "id": 668945682590523392, "text": "Like....I cannot deal.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 394942033, "name": "Morgo C❤️✨", "screen_name": "__morgo", "lang": "en", "location": "Pikesville, MD 21208", "create_at": date("2011-10-20"), "description": "Mother of a princess, kenli✨12.10.12✨ chapter 22❤️♐❤️ SC: morgo.c", "followers_count": 673, "friends_count": 194, "statues_count": 49863 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lochearn, MD", "id": "35b58c251fb1789f", "name": "Lochearn", "place_type": "city", "bounding_box": rectangle("-76.748415,39.319362 -76.711027,39.377081") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2447450, "cityName": "Lochearn" } }
+{ "create_at": datetime("2015-11-23T16:14:13.000Z"), "id": 668945682607308801, "text": "@MyMy__xo thanks!! ��", "in_reply_to_status": 668940190845698049, "in_reply_to_user": 2236687556, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2236687556 }}, "user": { "id": 107759365, "name": "November 23rd", "screen_name": "taliaaalynn", "lang": "en", "location": "Beaver Falls, PA", "create_at": date("2010-01-23"), "description": "senior cheer captain. proverbs 31:25. ✝ ... @_ChuckJackson_ ❤️", "followers_count": 1234, "friends_count": 956, "statues_count": 24916 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaver Falls, PA", "id": "579173dcb66e54d5", "name": "Beaver Falls", "place_type": "city", "bounding_box": rectangle("-80.335207,40.739691 -80.309549,40.790082") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42007, "countyName": "Beaver", "cityID": 4204792, "cityName": "Beaver Falls" } }
+{ "create_at": datetime("2015-11-23T16:14:13.000Z"), "id": 668945682674225152, "text": "@jhouse5 @Titleist445 Go YouTube Divine Sounds' \"What People Do for Money.\" Ferocious groove with cheeky, prescient lyrics. #hiphop", "in_reply_to_status": 668944507413463040, "in_reply_to_user": 15026258, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiphop" }}, "user_mentions": {{ 15026258, 26780618 }}, "user": { "id": 2260488314, "name": "Chuck Monan", "screen_name": "monan_chuck", "lang": "en", "location": "Little Rock USA", "create_at": date("2013-12-24"), "description": "Preaching Minister at PVCC....Pigskin Preacher on 103.7 The Buzz", "followers_count": 599, "friends_count": 191, "statues_count": 10075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maumelle, AR", "id": "13e46df66b64d972", "name": "Maumelle", "place_type": "city", "bounding_box": rectangle("-92.434688,34.815745 -92.369176,34.891708") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 544600, "cityName": "Maumelle" } }
+{ "create_at": datetime("2015-11-23T16:14:13.000Z"), "id": 668945682691186688, "text": "If this isn't a wake up call for these players, I don't know what will be. #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": 107805484, "name": "Chase Arthur", "screen_name": "KingMeArthur", "lang": "en", "location": "Jeffersonville, IN, USA", "create_at": date("2010-01-23"), "description": "Mayor's Office Intern for the City of Jeffersonville. Coach, Swimmer, Traveler, and Overall Student of Life. *Tweets are solely my views and interests alone.*", "followers_count": 378, "friends_count": 956, "statues_count": 11806 }, "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("2015-11-23T16:14:13.000Z"), "id": 668945682720419845, "text": "@__briannaa11 shut up you're gorges the way you are.", "in_reply_to_status": 668944387330588672, "in_reply_to_user": 2580526088, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2580526088 }}, "user": { "id": 2580643081, "name": "am@ri$$", "screen_name": "quannaa11", "lang": "en", "location": "Burleson, TX", "create_at": date("2014-06-21"), "description": "TJ Leach..❤️ ♥️❣", "followers_count": 451, "friends_count": 370, "statues_count": 4411 }, "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("2015-11-23T16:14:13.000Z"), "id": 668945683009765376, "text": "how can cowboy boots not bring a smile to 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": 1191626425, "name": "Kate", "screen_name": "caitlynswansonn", "lang": "en", "location": "BA", "create_at": date("2013-02-17"), "description": "at least my coffee won't be cold in hell", "followers_count": 274, "friends_count": 267, "statues_count": 1284 }, "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("2015-11-23T16:14:14.000Z"), "id": 668945683500658688, "text": "Ahh, cool crisp Delaware evenings can be beautiful. Thank you for the cozy picture @leafophoto… https://t.co/DPZMevizqa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.1634369,38.7555809"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 35883216 }}, "user": { "id": 2366446525, "name": "Live Delaware", "screen_name": "LiveDelaware", "lang": "en", "location": "Delaware", "create_at": date("2014-02-28"), "description": "Promoting & Supporting locally owned and operated businesses in Delaware. | Non-profits too. | Keep it local! #netde", "followers_count": 2110, "friends_count": 2102, "statues_count": 3376 }, "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("2015-11-23T16:14:14.000Z"), "id": 668945683605495809, "text": "My moms trying to get me obsessed with a 15 year old contestant on The Voice. Help.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 68483968, "name": "Amy Elizabeth", "screen_name": "amy_elizabethx3", "lang": "en", "location": "null", "create_at": date("2009-08-24"), "description": "Believe that life is worth living and your belief will help create the fact. 9.2.13", "followers_count": 269, "friends_count": 343, "statues_count": 6444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madison Center, CT", "id": "01444f50023c13c0", "name": "Madison Center", "place_type": "city", "bounding_box": rectangle("-72.672145,41.2485 -72.524154,41.360874") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 944630, "cityName": "Madison Center" } }
+{ "create_at": datetime("2015-11-23T16:14:14.000Z"), "id": 668945683647496192, "text": "Definitely had to redo that ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 596961185, "name": "❣", "screen_name": "ForeverRiyaah", "lang": "en", "location": "South Carolina, USA", "create_at": date("2012-06-01"), "description": "❃ Instagram; @Kurlzies ❃ sc; riyah ✨5 beautiful angels, rest easy Merah, Eli, Nahthan, Gabe, & Elaine you'll never be forgotten babies. I love you ❤️ #Home", "followers_count": 1736, "friends_count": 917, "statues_count": 62731 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Grove, SC", "id": "1edcae527018c24b", "name": "Oak Grove", "place_type": "city", "bounding_box": rectangle("-81.198937,33.945051 -81.105911,34.011381") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45063, "countyName": "Lexington", "cityID": 4551720, "cityName": "Oak Grove" } }
+{ "create_at": datetime("2015-11-23T16:14:14.000Z"), "id": 668945683878014976, "text": "the fact that my coworker is in her mid-twenties and still talks crap and starts drama is funny ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 136491244, "name": "Elynnnnn", "screen_name": "ElynRene", "lang": "en", "location": "Pleasant Hill, IA", "create_at": date("2010-04-23"), "description": "null", "followers_count": 804, "friends_count": 179, "statues_count": 25991 }, "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("2015-11-23T16:14:14.000Z"), "id": 668945683886563328, "text": "@iDhanzo lol gotta change my lock screen for this", "in_reply_to_status": 668547882270740481, "in_reply_to_user": 319642780, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 319642780 }}, "user": { "id": 183709692, "name": "Albus Severus Potter", "screen_name": "Iam_UT", "lang": "en", "location": "El Dorado", "create_at": date("2010-08-27"), "description": "YOU MAY BEAT ME DOWN, BUT LIKE A DICK, I RISE http://rantsofsilence.wordpress.com", "followers_count": 3860, "friends_count": 1595, "statues_count": 174719 }, "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("2015-11-23T16:14:14.000Z"), "id": 668945684008046592, "text": "@YukonVet DaisyMayLoverPants! Oh, how I've missed you! #puglove #droakleyyukonvet", "in_reply_to_status": 668932077543251968, "in_reply_to_user": 2445617514, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "puglove", "droakleyyukonvet" }}, "user_mentions": {{ 2445617514 }}, "user": { "id": 876814650, "name": "Lisa Townsend", "screen_name": "ltownsend167", "lang": "en", "location": "null", "create_at": date("2012-10-12"), "description": "null", "followers_count": 61, "friends_count": 202, "statues_count": 348 }, "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("2015-11-23T16:14:14.000Z"), "id": 668945684213719040, "text": "Like he legit came up to me and was like, I have to do this I'm like wait what lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 515765922, "name": "G", "screen_name": "gabbydangelo_", "lang": "en", "location": "null", "create_at": date("2012-03-05"), "description": "null", "followers_count": 1139, "friends_count": 365, "statues_count": 70858 }, "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("2015-11-23T16:14:14.000Z"), "id": 668945684222099457, "text": "\"what color are my eyes\" https://t.co/Ge12C0jGDu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2800357295, "name": "Anna Radican", "screen_name": "aradicalll", "lang": "en", "location": "Noblesville, IN", "create_at": date("2014-10-02"), "description": "Don't stand up then sit back down just because you have doubt.", "followers_count": 300, "friends_count": 438, "statues_count": 6097 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Noblesville, IN", "id": "013e3bc05a18abdc", "name": "Noblesville", "place_type": "city", "bounding_box": rectangle("-86.090354,39.993436 -85.932944,40.10716") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18057, "countyName": "Hamilton", "cityID": 1854180, "cityName": "Noblesville" } }
+{ "create_at": datetime("2015-11-23T16:14:14.000Z"), "id": 668945684243066881, "text": "#AllWhiteJury deliberating in case of #BlackWomanSerialRapist #DanielHoltzclaw in racist #OK. Facing 36 counts of #rape", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AllWhiteJury", "BlackWomanSerialRapist", "DanielHoltzclaw", "OK", "rape" }}, "user": { "id": 256352051, "name": "Magnificent2", "screen_name": "Indomitable69", "lang": "en", "location": "Whup Republican Ass Blvd.", "create_at": date("2011-02-22"), "description": "#Independent #Liberal #Leo #INTJ #RacistsBANNED4Life #Cowboys4Life #Obama4Life What is your deepest fear?", "followers_count": 5061, "friends_count": 4923, "statues_count": 104236 }, "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("2015-11-23T16:14:14.000Z"), "id": 668945684322717696, "text": "Early Xmas gift to myself. Thanks @NoLayingUp https://t.co/pCAinPzRzk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1151892168 }}, "user": { "id": 1221645823, "name": "Matt Herr", "screen_name": "AFootintheRough", "lang": "en", "location": "New York, NY", "create_at": date("2013-02-26"), "description": "The only way of finding out a man's character is to play golf with him. - P.G Wodehouse", "followers_count": 80, "friends_count": 424, "statues_count": 921 }, "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("2015-11-23T16:14:14.000Z"), "id": 668945684532502531, "text": "my mom is literally a spawn of satan", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 371528299, "name": "taylor", "screen_name": "Oasying", "lang": "en", "location": "carthage '19", "create_at": date("2011-09-10"), "description": "lover of Jesus, puppies, & sweet tea • @finally_erik :) • IG: Oasying", "followers_count": 558, "friends_count": 69, "statues_count": 33736 }, "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:14.000Z"), "id": 668945684691685376, "text": "I will see what I can do. Tomorrow evening if not afternoon. I have more time Wednesday if we don't get it finished. https://t.co/Opbanry2K1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 80979854, "name": "Thomas Price", "screen_name": "TBrave", "lang": "en", "location": "Northern California - NW", "create_at": date("2009-10-08"), "description": "Northwest Native, Father of 4, Nature Boy, Husband & Friend.\nAka. @ThomastheBrave, @TLP_LtY, @PacNWCascades\n#SocialMedia #Marketing #Guru, #WWDB, #LocaltoYou!", "followers_count": 1256, "friends_count": 2422, "statues_count": 9129 }, "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:14.000Z"), "id": 668945684779827200, "text": "I do not want to go to this city council meeting at all", "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": "christian", "screen_name": "omfgchristian", "lang": "en", "location": "null", "create_at": date("2011-11-29"), "description": "controversial but not in a bad way", "followers_count": 448, "friends_count": 353, "statues_count": 19817 }, "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("2015-11-23T16:14:14.000Z"), "id": 668945684880564225, "text": "I keep her with me so I can have someone to travel with❤️ https://t.co/6UJZQtkFKr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 894113143, "name": "Lindsey Golnick", "screen_name": "LGolnick", "lang": "en", "location": "null", "create_at": date("2012-10-20"), "description": "California☀ A Day Without Laughter Is A Day Wasted", "followers_count": 201, "friends_count": 281, "statues_count": 1794 }, "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:14.000Z"), "id": 668945685098733570, "text": "ANDY GRAMMER IS BACK ON DWTS TONIGHT GUYS MY BAE @andygrammer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15138534 }}, "user": { "id": 764463434, "name": "Sarah James", "screen_name": "sarahjames1994", "lang": "en", "location": "null", "create_at": date("2012-08-17"), "description": "i love The Vamps One Direction BTR Olivia Somerlyn Demi Lovato", "followers_count": 1344, "friends_count": 2061, "statues_count": 45891 }, "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("2015-11-23T16:14:14.000Z"), "id": 668945685652242432, "text": "@Dave4FtMyers Would love to see how hes going to get that crap past a republican congress. Oh ya, Hussein O will write EO @arrowsmithwoman", "in_reply_to_status": 668939852839170048, "in_reply_to_user": 338291677, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 338291677, 67514751 }}, "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": 2033, "friends_count": 2243, "statues_count": 16359 }, "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:14.000Z"), "id": 668945685992091648, "text": "So @jennschiffer was saying she needed to buy an \"electric knife\" and I immediately thought of like a knife-taser instead of carving knife.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 12524622 }}, "user": { "id": 17177251, "name": "spacer.bmp", "screen_name": "brianloveswords", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2008-11-04"), "description": "chief garbage monster @Bocoup; figurehead @brooklyn_js; probably not three cats in a trench coat. He/him", "followers_count": 3633, "friends_count": 1900, "statues_count": 18499 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kingston, NY", "id": "1385e8d7843ce9d7", "name": "Kingston", "place_type": "city", "bounding_box": rectangle("-74.051642,41.875827 -73.961854,41.957802") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36111, "countyName": "Ulster", "cityID": 3639727, "cityName": "Kingston" } }
+{ "create_at": datetime("2015-11-23T16:14:14.000Z"), "id": 668945686013083648, "text": "Let me say this again. THATS THE FUCKING PLAY YOU DRAW UP!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 252133591, "name": "Josh Lee", "screen_name": "jeddylee_14", "lang": "en", "location": "Greensburg, IN", "create_at": date("2011-02-14"), "description": "Indiana University.", "followers_count": 256, "friends_count": 432, "statues_count": 4796 }, "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:14.000Z"), "id": 668945686038188032, "text": "conflating WMD in Iraq with Intel on #Daesh is patently ridiculous and offensive, @mitchellreports. #NBCNightlyNews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Daesh", "NBCNightlyNews" }}, "user_mentions": {{ 89820928 }}, "user": { "id": 14253400, "name": "Will Pollock", "screen_name": "bywillpollock", "lang": "en", "location": "Atlanta & Charlottesville", "create_at": date("2008-03-29"), "description": "timid writers rarely make history | now: @pizzaforgood (http://bit.ly/1ho3IPA) & @leavingtriscuit (http://bit.ly/trskyy); humorist. occasional daft prick.", "followers_count": 1514, "friends_count": 870, "statues_count": 21916 }, "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("2015-11-23T16:14:14.000Z"), "id": 668945686054981632, "text": "Here now waiting for the show to start. https://t.co/c4GLSTHSIn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2902491161, "name": "Lisa Krauss", "screen_name": "LisaKraussWhite", "lang": "en", "location": "Hickory, North Carolina", "create_at": date("2014-12-02"), "description": "Huge Walking Dead fan. Big Sis,Cool Aunt, SciFi geek, animal lover. Love beach,water,scuba Peace & Love for everyone.Don't just say it, show it. #BrightSideRulz", "followers_count": 731, "friends_count": 747, "statues_count": 4182 }, "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("2015-11-23T16:14:14.000Z"), "id": 668945686365380608, "text": "I literally win every time in spongebob trivia crack. I know my spongebob", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 398950633, "name": "Katie Nelson", "screen_name": "KatieNelson17", "lang": "en", "location": "null", "create_at": date("2011-10-26"), "description": "Snap chat- katie07123", "followers_count": 932, "friends_count": 501, "statues_count": 10459 }, "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("2015-11-23T16:14:14.000Z"), "id": 668945686436548608, "text": "@MalerieHixson @AdysonBargstadt dying over here ��", "in_reply_to_status": 668931959343583232, "in_reply_to_user": 451129667, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 451129667, 510826197 }}, "user": { "id": 987665804, "name": "Megan Poppert", "screen_name": "MeganPoppert", "lang": "en", "location": "null", "create_at": date("2012-12-03"), "description": "UNL ΧΩ❤️God is so good❤️", "followers_count": 348, "friends_count": 250, "statues_count": 3224 }, "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("2015-11-23T16:14:14.000Z"), "id": 668945686461677569, "text": "My son and his equally idiotic friend decided holidays r more festive if you spray paint driveway red. Tomorrow, they get to know sandpaper.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 173910394, "name": "jeffrey t jordon", "screen_name": "jtjordon", "lang": "en", "location": "san diego", "create_at": date("2010-08-02"), "description": "null", "followers_count": 651, "friends_count": 446, "statues_count": 2896 }, "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:14.000Z"), "id": 668945686742867968, "text": "@drscott_atlanta how could anyone be so barbaric? So sickening and disgusting ��", "in_reply_to_status": 668937488719826944, "in_reply_to_user": 61015562, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 61015562 }}, "user": { "id": 3291165858, "name": "Lynn DeVito", "screen_name": "DevitoLynn", "lang": "en", "location": "null", "create_at": date("2015-07-24"), "description": "Proud American, loyal friend and animal lover", "followers_count": 104, "friends_count": 238, "statues_count": 557 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsford, NY", "id": "01431a8402105db6", "name": "Pittsford", "place_type": "city", "bounding_box": rectangle("-77.580355,43.026793 -77.479304,43.124356") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3658354, "cityName": "Pittsford" } }
+{ "create_at": datetime("2015-11-23T16:14:14.000Z"), "id": 668945686856130560, "text": "@TorreyRockwell @SeanSean252 @WayneDupreeShow @Rockprincess818 @realDonaldTrump https://t.co/Ui6Tw8w7IT", "in_reply_to_status": 668579906943610881, "in_reply_to_user": 3918610461, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3918610461, 1863840511, 282695161, 715044396, 25073877 }}, "user": { "id": 467726691, "name": "Cheri Jacobus", "screen_name": "CheriJacobus", "lang": "en", "location": "New York, NY", "create_at": date("2012-01-18"), "description": "GOP consultant, TV pundit, wrote for The Hill, The Guardian. Ran campaigns, worked on Capitol Hill, RNC, adjunct professor GWU Grad School of Political Mgmt", "followers_count": 6544, "friends_count": 583, "statues_count": 24303 }, "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("2015-11-23T16:14:14.000Z"), "id": 668945686864490496, "text": ", miraah�� https://t.co/4zkRcrpoVX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 2901679012, "name": "B A P❤✨", "screen_name": "miraahworld_", "lang": "en", "location": "A❤", "create_at": date("2014-12-01"), "description": "w/ taylor forevaeva❣ kai✨", "followers_count": 1243, "friends_count": 958, "statues_count": 12244 }, "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:14.000Z"), "id": 668945686940024832, "text": "I'm so sick of this ugly ass picture of lil nana crying all over 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": 231223647, "name": ".", "screen_name": "_DAIYANight", "lang": "en", "location": "PHL✈️[D]MV", "create_at": date("2010-12-27"), "description": "#HigherGoals #HighHeels⠀ ⠀•High Level•⠀ ⠀ ⠀⠀ ⠀ ⠀⠀ ⠀⠀#HowardU ⠀ ⠀⠀ ⠀ ⠀ ⠀ ⠀ ⠀ ⠀⠀ ⠀⠀#crowned. ⠀ ⠀⠀ ⠀ ⠀⠀ ⠀ ⠀ ⠀⠀ ⠀⠀ ⠀ ⠀⠀'19", "followers_count": 1693, "friends_count": 1311, "statues_count": 48602 }, "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("2015-11-23T16:14:14.000Z"), "id": 668945687023738880, "text": "@JayyM20 @kaileenyy_ brb I'm trying to catch and be cool like y'all", "in_reply_to_status": 668945519255777280, "in_reply_to_user": 179208939, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 179208939, 3083530704 }}, "user": { "id": 446500088, "name": "bel", "screen_name": "sanchieson", "lang": "en", "location": "cloud surfing ", "create_at": date("2011-12-25"), "description": "dreams of living life like rappers do", "followers_count": 1420, "friends_count": 1003, "statues_count": 67197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Artesia, NM", "id": "016770364f36ba41", "name": "Artesia", "place_type": "city", "bounding_box": rectangle("-104.464491,32.784418 -104.38739,32.86448") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35015, "countyName": "Eddy", "cityID": 3505220, "cityName": "Artesia" } }
+{ "create_at": datetime("2015-11-23T16:14:14.000Z"), "id": 668945687028080640, "text": "\"Are you working hard or you hardly work?\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 88764223, "name": "lepeti", "screen_name": "LePetitToc", "lang": "en", "location": "aqui, ali e em todo lugar", "create_at": date("2009-11-09"), "description": "absolute beginner", "followers_count": 2028, "friends_count": 642, "statues_count": 62932 }, "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("2015-11-23T16:14:14.000Z"), "id": 668945687166324736, "text": "@SpiffyDiffySwag find something to do bro.", "in_reply_to_status": 668945175968792580, "in_reply_to_user": 840882548, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 840882548 }}, "user": { "id": 381584302, "name": "⚡2⚡3⚡9⚡", "screen_name": "ADDISON239", "lang": "en", "location": "26.351603,-81.772762", "create_at": date("2011-09-28"), "description": "#TeamReallyReal #Facebook KING ADDISON #INSTAGRAM 239FortMyers #SnapChat king-addison", "followers_count": 2492, "friends_count": 2974, "statues_count": 9873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Estero, FL", "id": "47d6c4ee241dfc33", "name": "Estero", "place_type": "city", "bounding_box": rectangle("-81.840335,26.389186 -81.754749,26.456982") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12071, "countyName": "Lee", "cityID": 1221150, "cityName": "Estero" } }
+{ "create_at": datetime("2015-11-23T16:14:14.000Z"), "id": 668945687220998144, "text": "I'm at Archibald, LA in 5, LA https://t.co/iG9o0ynMvO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-91.77673087,32.35289906"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 92890677, "name": "Lee Jones", "screen_name": "LeeJ07", "lang": "en", "location": "Monroe,LA", "create_at": date("2009-11-26"), "description": "Handsome southern charmer who loves to travel(At least, that's what I tell everyone off the record. I may have a few secrets!!).", "followers_count": 449, "friends_count": 2004, "statues_count": 20346 }, "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": 22041, "countyName": "Franklin" } }
+{ "create_at": datetime("2015-11-23T16:14:14.000Z"), "id": 668945687267012608, "text": "Reasons I rather drink beer https://t.co/G03lPk5VXn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 632513030, "name": "King D.", "screen_name": "Sweet_Love_Dom", "lang": "en", "location": "Houston, Tx.", "create_at": date("2012-07-10"), "description": "In my heart forever Damazo Tello. My brother. See you soon. ig : domg713", "followers_count": 1222, "friends_count": 755, "statues_count": 43848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:14.000Z"), "id": 668945687321534464, "text": "@DATBOIMJC @LatiNo_Chill FOH", "in_reply_to_status": 668944833348681728, "in_reply_to_user": 454166006, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 454166006, 426226006 }}, "user": { "id": 533630397, "name": "sleezy", "screen_name": "IllDoPeYOu", "lang": "en", "location": "Schertz, TX", "create_at": date("2012-03-22"), "description": "null", "followers_count": 630, "friends_count": 226, "statues_count": 77483 }, "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("2015-11-23T16:14:14.000Z"), "id": 668945687548030976, "text": "@jwiesey_baby https://t.co/LPw4sMn3Gh", "in_reply_to_status": -1, "in_reply_to_user": 413614695, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 413614695 }}, "user": { "id": 57202022, "name": ".bailey.", "screen_name": "bailey_zwahrr", "lang": "en", "location": "null", "create_at": date("2009-07-15"), "description": "⭐️Instagram: bailey_zwahrr | Speardog⭐️", "followers_count": 346, "friends_count": 433, "statues_count": 13528 }, "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("2015-11-23T16:14:14.000Z"), "id": 668945687564816384, "text": "https://t.co/8MPKyYgvtk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 25722965, "name": "Wayne Winters", "screen_name": "minnman47", "lang": "en", "location": "Conway Ar", "create_at": date("2009-03-21"), "description": "world affairs in the crosshairs...trigger finger on the pulse of society", "followers_count": 6707, "friends_count": 6729, "statues_count": 164805 }, "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("2015-11-23T16:14:15.000Z"), "id": 668945687694831616, "text": "バス待ってたら天使いた����\n変顔してもかわいさやばい〜 https://t.co/AjsmVhTaBV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user": { "id": 808809013, "name": "Sayaka", "screen_name": "_sayakyan", "lang": "ja", "location": "Chico, CA", "create_at": date("2012-09-07"), "description": "CSU Chico, ALCI student !", "followers_count": 434, "friends_count": 311, "statues_count": 85 }, "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("2015-11-23T16:14:15.000Z"), "id": 668945687724298240, "text": "(@ Fashion Valley in San Diego, CA) https://t.co/fJLc6iM2Yi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.16717978,32.76786063"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1086785532, "name": "سالم أبوردن", "screen_name": "salem_aborden", "lang": "ar", "location": "Kuwait - USA ", "create_at": date("2013-01-13"), "description": "اللهم يسر أموري", "followers_count": 371, "friends_count": 313, "statues_count": 5356 }, "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:15.000Z"), "id": 668945688076513280, "text": "Life becomes more meaningful when you realize the simple fact that you'll never get the same moment twice", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2262050766, "name": "Lil k.", "screen_name": "BayardKaila", "lang": "en", "location": "Auburn, WA", "create_at": date("2013-12-25"), "description": "• ♌️ • 18 • Enjoying life • Blessed •", "followers_count": 320, "friends_count": 580, "statues_count": 3856 }, "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("2015-11-23T16:14:15.000Z"), "id": 668945688210771968, "text": "I just wanna check like the 3rd ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1085064973, "name": "Im the plug", "screen_name": "The_Champ23", "lang": "en", "location": "#GRINDING4TASHAWN", "create_at": date("2013-01-12"), "description": "So Viking PullUp Boy in Dirtyville,Va", "followers_count": 1378, "friends_count": 943, "statues_count": 40460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Danville, VA", "id": "a58a872123811d63", "name": "Danville", "place_type": "city", "bounding_box": rectangle("-79.519308,36.540813 -79.317136,36.673579") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51590, "countyName": "Danville", "cityID": 5121344, "cityName": "Danville" } }
+{ "create_at": datetime("2015-11-23T16:14:15.000Z"), "id": 668945688781324288, "text": "I got 4 boys listening to #PURPOSE ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PURPOSE" }}, "user": { "id": 3950721389, "name": "4 more days✨", "screen_name": "Ashhhhlayyy", "lang": "en", "location": "Starbucks", "create_at": date("2015-10-13"), "description": "• Matthew 7:1• Beyonce & Chance & JB❤️", "followers_count": 276, "friends_count": 250, "statues_count": 8868 }, "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:15.000Z"), "id": 668945688789520384, "text": "�� Gods were 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": 62199389, "name": "S❃", "screen_name": "sharmne", "lang": "en", "location": "San Diego, CA", "create_at": date("2009-08-01"), "description": "sc: sharmne", "followers_count": 524, "friends_count": 342, "statues_count": 19142 }, "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("2015-11-23T16:14:15.000Z"), "id": 668945688844046336, "text": "@its_lexxy https://t.co/BaEbEXqO8x", "in_reply_to_status": -1, "in_reply_to_user": 713066844, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 713066844 }}, "user": { "id": 1562223944, "name": "ᗷᖇEᑎT", "screen_name": "brittanyvanae16", "lang": "en", "location": "Wills Point, TX", "create_at": date("2013-07-01"), "description": "That's the past, let it go.", "followers_count": 437, "friends_count": 634, "statues_count": 6041 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wills Point, TX", "id": "4036df46900590ba", "name": "Wills Point", "place_type": "city", "bounding_box": rectangle("-96.034442,32.69598 -95.987913,32.727338") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48467, "countyName": "Van Zandt", "cityID": 4879564, "cityName": "Wills Point" } }
+{ "create_at": datetime("2015-11-23T16:14:15.000Z"), "id": 668945688936497152, "text": "No laziness exists when there's a half a chunk of cheese in the fridge. When there's laundry to fold, it's a whole 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": 146513248, "name": "i carried a punkin", "screen_name": "duh_nellll", "lang": "en", "location": "buffalo, ny", "create_at": date("2010-05-21"), "description": "I was born under a chicken wing, raised in a bison's den", "followers_count": 1287, "friends_count": 1663, "statues_count": 39211 }, "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("2015-11-23T16:14:15.000Z"), "id": 668945689028640769, "text": "�� https://t.co/b5B5DeMtMZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "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": 74332 }, "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:15.000Z"), "id": 668945689347510272, "text": "@lindapelaccio @Heritage_Radio absolutely!", "in_reply_to_status": 668909426871545856, "in_reply_to_user": 114485547, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 114485547, 50476981 }}, "user": { "id": 16113204, "name": "katieparla", "screen_name": "katieparla", "lang": "en", "location": "Rome, Istanbul, Dirty Jerz", "create_at": date("2008-09-03"), "description": "I write about foods & drinks & the people that make them. KP's Rome & KP's Istanbul (apps); Eating & Drinking in Rome (ebook); Tasting Rome (cookbook).", "followers_count": 11492, "friends_count": 2530, "statues_count": 22710 }, "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("2015-11-23T16:14:15.000Z"), "id": 668945689368510464, "text": "@KrystenWilkers2 and I are kinda praying aunt charity is gonna come in clutch for the big ticket this year again����", "in_reply_to_status": -1, "in_reply_to_user": 1365687990, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1365687990 }}, "user": { "id": 606563886, "name": "breahღ", "screen_name": "Breah_Pelfrey", "lang": "en", "location": "Baker County, Florida", "create_at": date("2012-06-12"), "description": "no I'm not Lindsay Lohan", "followers_count": 566, "friends_count": 503, "statues_count": 18079 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fleming Island, FL", "id": "00a5cfeb77a0a6f1", "name": "Fleming Island", "place_type": "city", "bounding_box": rectangle("-81.748593,30.041009 -81.689034,30.147301") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12019, "countyName": "Clay", "cityID": 1222660, "cityName": "Fleming Island" } }
+{ "create_at": datetime("2015-11-23T16:14:15.000Z"), "id": 668945689595027457, "text": "@keep__smyelin you're correct, I don't feel it though. Maybe it's theirs but I don't feel it. #ChatMS", "in_reply_to_status": 668945375387123712, "in_reply_to_user": 1355393934, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ChatMS" }}, "user_mentions": {{ 1355393934 }}, "user": { "id": 16245157, "name": "Glenn D", "screen_name": "bccdny", "lang": "en", "location": "New York", "create_at": date("2008-09-11"), "description": "I take lots of photos of the food I eat & other stuff too #justshoot. Let's cure MS! Support the #DeterminedDomilicis", "followers_count": 1037, "friends_count": 1996, "statues_count": 15155 }, "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("2015-11-23T16:14:15.000Z"), "id": 668945689595027458, "text": "@NetNetB_ �������� Corny Right! Dfl", "in_reply_to_status": 668944910813409280, "in_reply_to_user": 70615882, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 70615882 }}, "user": { "id": 2723671546, "name": "Karla Shantele'", "screen_name": "IKarlaBee", "lang": "en", "location": "null", "create_at": date("2014-07-24"), "description": "https://soundcloud.com/soundsofamillionaire/ty-minus-the-bullshit", "followers_count": 811, "friends_count": 342, "statues_count": 42795 }, "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:15.000Z"), "id": 668945689779441664, "text": "Hate wasting my time on people who don't deserve it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 492842272, "name": "Tori", "screen_name": "Tori6394", "lang": "en", "location": "San Antonio, TX", "create_at": date("2012-02-14"), "description": "SC: toree_2017 | TJG", "followers_count": 1044, "friends_count": 866, "statues_count": 23299 }, "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:15.000Z"), "id": 668945689854906368, "text": "You can have it all, my empire of dirt!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 345077036, "name": "Russell Urban", "screen_name": "russurban7", "lang": "en", "location": "New Jersey", "create_at": date("2011-07-29"), "description": "20 years old.", "followers_count": 395, "friends_count": 285, "statues_count": 7783 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Runnemede, NJ", "id": "dca35046296d178c", "name": "Runnemede", "place_type": "city", "bounding_box": rectangle("-75.102633,39.840473 -75.048314,39.863385") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34007, "countyName": "Camden", "cityID": 3465160, "cityName": "Runnemede" } }
+{ "create_at": datetime("2015-11-23T16:14:15.000Z"), "id": 668945690081538049, "text": "Not when I'm so close to 2 huge goals 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": 1883332765, "name": "Bri .♈️", "screen_name": "michelle_lovee2", "lang": "en", "location": "Durham, NC", "create_at": date("2013-09-19"), "description": "NorthCarolinaCentral'17", "followers_count": 1246, "friends_count": 2029, "statues_count": 11426 }, "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("2015-11-23T16:14:15.000Z"), "id": 668945690299625472, "text": "@GPNHS enviro students sharing results from WDET Our Environment, Our Waterways event! @CJ_Skowronski @katecalmurray https://t.co/XInrbmfNWt", "in_reply_to_status": -1, "in_reply_to_user": 840273373, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 840273373, 3060768433, 309981615 }}, "user": { "id": 707257860, "name": "Maureen Bur", "screen_name": "maureen_bur", "lang": "en", "location": "Motor City", "create_at": date("2012-07-20"), "description": "Educator at Grosse Pointe Public Schools | Views are my own", "followers_count": 592, "friends_count": 948, "statues_count": 2724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grosse Pointe Farms, MI", "id": "7db67e376ddb31f2", "name": "Grosse Pointe Farms", "place_type": "city", "bounding_box": rectangle("-82.915006,42.384205 -82.883058,42.426377") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2635520, "cityName": "Grosse Pointe Farms" } }
+{ "create_at": datetime("2015-11-23T16:14:15.000Z"), "id": 668945690492575744, "text": "Let just talk about this session that's gonna take place we we get back �� https://t.co/THGn8WHJaZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4202067604, "name": "Dawit ♓️", "screen_name": "6awit", "lang": "en", "location": "Boro", "create_at": date("2015-11-16"), "description": "null", "followers_count": 68, "friends_count": 61, "statues_count": 190 }, "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.17314,33.884965") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1377652, "cityName": "Tucker" } }
+{ "create_at": datetime("2015-11-23T16:14:15.000Z"), "id": 668945690513514499, "text": "@RichDaKid7 ok", "in_reply_to_status": 668906933223604225, "in_reply_to_user": 704584813, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 704584813 }}, "user": { "id": 1028804886, "name": "jo", "screen_name": "xx_jordaan", "lang": "en", "location": "null", "create_at": date("2012-12-22"), "description": "#GWSTaeJon #RIPAuntieSharon #LupusPrincessAlia", "followers_count": 3230, "friends_count": 1582, "statues_count": 159610 }, "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("2015-11-23T16:14:15.000Z"), "id": 668945691100774400, "text": "I love crean to death. wanted to be a member of his staff for quite some time but after that end to the game. Fire him after he shakes hands", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1166635730, "name": "Nick Bragg", "screen_name": "Nick_Bragg4", "lang": "en", "location": "null", "create_at": date("2013-02-10"), "description": "OVO MillerGolf", "followers_count": 203, "friends_count": 350, "statues_count": 7361 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Noblesville, IN", "id": "013e3bc05a18abdc", "name": "Noblesville", "place_type": "city", "bounding_box": rectangle("-86.090354,39.993436 -85.932944,40.10716") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18057, "countyName": "Hamilton", "cityID": 1854180, "cityName": "Noblesville" } }
+{ "create_at": datetime("2015-11-23T16:14:15.000Z"), "id": 668945691243380736, "text": "Cowbell on 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": 3225462300, "name": "Zach G", "screen_name": "zrgustin18", "lang": "en", "location": "Holt, MI", "create_at": date("2015-05-24"), "description": "null", "followers_count": 68, "friends_count": 291, "statues_count": 2754 }, "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("2015-11-23T16:14:15.000Z"), "id": 668945691255939072, "text": "Chandler jones is an inspiration", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 517087858, "name": "FettY", "screen_name": "Ross_ins", "lang": "en", "location": "Long Beach", "create_at": date("2012-03-06"), "description": "ALLYSON ❤️#LAKERGANG #NYR #FLYEAGLESFLY", "followers_count": 592, "friends_count": 250, "statues_count": 13150 }, "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("2015-11-23T16:14:15.000Z"), "id": 668945691696238592, "text": "Shop at @vegancollection and 25% of your purchase goes to Kitten Rescue. https://t.co/84yN8wZ7lt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 52871994 }}, "user": { "id": 18405034, "name": "KittenRescue", "screen_name": "KittenRescue", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2008-12-27"), "description": "Kitten Rescue is a Los Angeles-based non-profit organization dedicated to saving homeless and abandoned cats and dogs.", "followers_count": 10464, "friends_count": 7480, "statues_count": 4876 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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:16.000Z"), "id": 668945691855740928, "text": "Fuck. https://t.co/2ArZ8cvRSJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3526166537, "name": "Spacemonkey17", "screen_name": "skye_swartley", "lang": "en", "location": "Pennsylvania, USA", "create_at": date("2015-09-02"), "description": "life is life so live it long and wisely.", "followers_count": 13, "friends_count": 108, "statues_count": 147 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lansford, PA", "id": "140baa41a003175a", "name": "Lansford", "place_type": "city", "bounding_box": rectangle("-75.906059,40.822821 -75.860654,40.842705") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42025, "countyName": "Carbon", "cityID": 4241464, "cityName": "Lansford" } }
+{ "create_at": datetime("2015-11-23T16:14:16.000Z"), "id": 668945692036112384, "text": "@ohMYitsjiax listen I don't even have a cat", "in_reply_to_status": 668945589200150528, "in_reply_to_user": 32796097, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 32796097 }}, "user": { "id": 123742825, "name": "super shredder ॐ", "screen_name": "LeonEfff", "lang": "en", "location": "NYC Basshead", "create_at": date("2010-03-16"), "description": "NYC - Basshead - Open your eyes- Nothing is what it seems - ♓️", "followers_count": 830, "friends_count": 641, "statues_count": 46041 }, "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:16.000Z"), "id": 668945692388368384, "text": "My sister: I bought a record player\nMe: wow you're so hipster", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1705714842, "name": "Kat", "screen_name": "kgpowell99", "lang": "en", "location": "Texas", "create_at": date("2013-08-27"), "description": "Smile, life is way too short to be unhappy", "followers_count": 278, "friends_count": 218, "statues_count": 8255 }, "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("2015-11-23T16:14:16.000Z"), "id": 668945692669296640, "text": "What's with all the tears ? We been doing this shit for years ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 504851265, "name": "Angel Hendrixxx", "screen_name": "__angelnicolee", "lang": "en", "location": "null", "create_at": date("2012-02-26"), "description": "July23rd ♕ . Rip Raven . bestfriend @brittanyy_tiana", "followers_count": 2259, "friends_count": 925, "statues_count": 100996 }, "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("2015-11-23T16:14:16.000Z"), "id": 668945692891582469, "text": "@michelle0043 yessss in pe �� Lmfao!", "in_reply_to_status": 668945492923953152, "in_reply_to_user": 944851416, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 944851416 }}, "user": { "id": 2437942099, "name": "Mami Yankee", "screen_name": "elizabuttpls", "lang": "en", "location": "null", "create_at": date("2014-04-10"), "description": "null", "followers_count": 550, "friends_count": 629, "statues_count": 34854 }, "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("2015-11-23T16:14:16.000Z"), "id": 668945693080326144, "text": "marvins room - drake, 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": 3300089843, "name": "(-:", "screen_name": "casssiegurl", "lang": "en", "location": "Oxnard, CA", "create_at": date("2015-05-26"), "description": "leo", "followers_count": 283, "friends_count": 328, "statues_count": 3568 }, "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("2015-11-23T16:14:16.000Z"), "id": 668945693357273090, "text": "@__undenyable thanks bby��!", "in_reply_to_status": 668945208156028929, "in_reply_to_user": 2156424521, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2156424521 }}, "user": { "id": 560515924, "name": "BIRTHDAYGIRL❤️", "screen_name": "rayligion_", "lang": "en", "location": "null", "create_at": date("2012-04-22"), "description": "bibby you intelligent, you could've been the president!", "followers_count": 1007, "friends_count": 868, "statues_count": 52726 }, "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("2015-11-23T16:14:16.000Z"), "id": 668945693386539008, "text": "Embarrassing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 183873851, "name": "Austin Compton", "screen_name": "AustinCompton33", "lang": "en", "location": "Bloom Baby ", "create_at": date("2010-08-27"), "description": "317 | IU | ΘX Kelley School of Business 4-Time Football State Champion", "followers_count": 556, "friends_count": 531, "statues_count": 5961 }, "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("2015-11-23T16:14:16.000Z"), "id": 668945693764022272, "text": "I don't hold my tongue for nobody but there's certain shit you just don't say to a mf just outta respect ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1960173650, "name": "Tymera", "screen_name": "thug_life_ty", "lang": "en", "location": "daddy house ", "create_at": date("2013-10-14"), "description": "Princess of the world| LongLiveShamari ❤", "followers_count": 2133, "friends_count": 2344, "statues_count": 26857 }, "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:16.000Z"), "id": 668945693772521472, "text": "I'm at Carnegie Institution of Science - @carnegiescience in Washington, DC https://t.co/aZSIbvy5Gt https://t.co/Q74SdvtBkh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.03624324,38.90945264"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23814999 }}, "user": { "id": 29352296, "name": "ARMIE", "screen_name": "armie1", "lang": "en", "location": "Washington, DC ", "create_at": date("2009-04-06"), "description": "Hispanic issues interest me, as do politics, sports & taking pics/The essence of knowledge is, having it, to apply it; not having it, to confess your ignorance", "followers_count": 530, "friends_count": 2016, "statues_count": 6563 }, "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("2015-11-23T16:14:16.000Z"), "id": 668945693784956928, "text": "I forgot it was sushi Monday ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 466134880, "name": "Houston Miller", "screen_name": "HoustonMiller88", "lang": "en", "location": "Keller Highschool ", "create_at": date("2012-01-16"), "description": "Keller Indians 2016 DE/OLB #88 #FearTheSpear •Texas Tech Football Commit• #WreckEm If you want it... Work for it.", "followers_count": 1149, "friends_count": 729, "statues_count": 4210 }, "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("2015-11-23T16:14:16.000Z"), "id": 668945693940301825, "text": "@rachfeld Cullen appreciates your favorite pastime!!! https://t.co/IVYFIDryht", "in_reply_to_status": -1, "in_reply_to_user": 24558534, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24558534 }}, "user": { "id": 181681129, "name": "Lauren Gallagher", "screen_name": "lauuuurennn1", "lang": "en", "location": "Drexel University", "create_at": date("2010-08-22"), "description": "If you don’t like how the table is set, turn over the table.", "followers_count": 597, "friends_count": 692, "statues_count": 6961 }, "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:16.000Z"), "id": 668945694376501248, "text": "Tom Crean's coaching inadequacies are glaring in that loss to Wake. Complete lack of understanding team defensive concepts. #FireCrean", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FireCrean" }}, "user": { "id": 368702705, "name": "Jordan O", "screen_name": "jordanoyer12", "lang": "en", "location": "South Bend, IN", "create_at": date("2011-09-05"), "description": "Sales Representative at Premier Manufacturing. I am a Dol-Fan who has also bled Cream and Crimson his entire life. Go Hoosiers!", "followers_count": 57, "friends_count": 81, "statues_count": 466 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgetown, IN", "id": "b6a2e631184de1bf", "name": "Georgetown", "place_type": "city", "bounding_box": rectangle("-86.236322,41.716119 -86.216797,41.760422") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18043, "countyName": "Floyd", "cityID": 1827324, "cityName": "Georgetown" } }
+{ "create_at": datetime("2015-11-23T16:14:16.000Z"), "id": 668945694577827840, "text": "I'm at Southtown Yoga in Saint Louis, MO https://t.co/AvP6aaQMfh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.21225993,38.61726117"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19621850, "name": "Danielle Hohmeier", "screen_name": "daniellesmyname", "lang": "en", "location": "St. Louis, MO", "create_at": date("2009-01-27"), "description": "Brand & Content Strategy at @Atomicdust. @Shaw_STL resident. Weakness for coffee, JCrew, TV & all things #STL.", "followers_count": 3631, "friends_count": 3868, "statues_count": 31100 }, "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.320464,38.533149 -90.175132,38.774349") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29510, "countyName": "St. Louis", "cityID": 2965000, "cityName": "St. Louis" } }
+{ "create_at": datetime("2015-11-23T16:14:16.000Z"), "id": 668945694745624582, "text": "Marquette 10, Ben Simmons 0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "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": 39714, "friends_count": 1412, "statues_count": 89301 }, "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:16.000Z"), "id": 668945694787375104, "text": "So thankful❤️ https://t.co/hVIjiHtGw9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.819613,32.929994"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 321753475, "name": "Katie Pratt❁", "screen_name": "Seventeen30kate", "lang": "en", "location": "neverland", "create_at": date("2011-06-21"), "description": "Life goes on☻ Marcus Pope❤️", "followers_count": 450, "friends_count": 407, "statues_count": 5360 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Galleria Dallas", "id": "07d9c936f9087004", "name": "Galleria Dallas", "place_type": "poi", "bounding_box": rectangle("-96.8196131,32.9299939 -96.819613,32.929994") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2015-11-23T16:14:16.000Z"), "id": 668945694984568833, "text": "@Baileywinters97 he sounds like a great guy", "in_reply_to_status": 668905590962286592, "in_reply_to_user": 1633936447, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1633936447 }}, "user": { "id": 1101093116, "name": "Conor Kubicki", "screen_name": "conorkubicki", "lang": "en", "location": "null", "create_at": date("2013-01-18"), "description": "Strong and content, I travel the open road", "followers_count": 104, "friends_count": 160, "statues_count": 426 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edwardsville, KS", "id": "7bb25bcd7f8698a9", "name": "Edwardsville", "place_type": "city", "bounding_box": rectangle("-94.848542,39.04553 -94.78765,39.08519") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20209, "countyName": "Wyandotte", "cityID": 2020000, "cityName": "Edwardsville" } }
+{ "create_at": datetime("2015-11-23T16:14:16.000Z"), "id": 668945695454470144, "text": "�������� https://t.co/1potHi7G1Z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 27172416, "name": "Packers 7-3", "screen_name": "NattyDreadLocs", "lang": "en", "location": "United States", "create_at": date("2009-03-27"), "description": "#BlackLivesMatter #withMalala #MizzouHungerStrike #ConcernedStudent1950 #ConcernedAlum #StayWoke #USMNT", "followers_count": 516, "friends_count": 309, "statues_count": 61167 }, "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("2015-11-23T16:14:16.000Z"), "id": 668945695554953216, "text": "@VianneyCafe Tuesday Lunch Menu: Annual Thanksgiving Dinner. Turkey, 2 sides, and a dinner roll. #blessed #feast #LiturgyDay", "in_reply_to_status": -1, "in_reply_to_user": 859639836, "favorite_count": 0, "coordinate": point("-90.47133771,38.56810953"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "blessed", "feast", "LiturgyDay" }}, "user_mentions": {{ 859639836 }}, "user": { "id": 859639836, "name": "Vianney Cafe", "screen_name": "VianneyCafe", "lang": "en", "location": "Kirkwood, Missouri ", "create_at": date("2012-10-03"), "description": "St. John Vianney High School Cafe, Kirkwood, Missouri - Only the pure in heart can make a good soup.", "followers_count": 482, "friends_count": 43, "statues_count": 754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Des Peres, MO", "id": "b81c5ca7b5eecf7a", "name": "Des Peres", "place_type": "city", "bounding_box": rectangle("-90.474007,38.568344 -90.425156,38.6187") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2919270, "cityName": "Des Peres" } }
+{ "create_at": datetime("2015-11-23T16:14:16.000Z"), "id": 668945695869652992, "text": "#gingerass https://t.co/4hNEPjDBjL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "gingerass" }}, "user": { "id": 329674212, "name": "❣The Big Bad Wolf❣", "screen_name": "bunkydm", "lang": "en", "location": "thebigbadwolf-blog.tumblr.com", "create_at": date("2011-07-05"), "description": "Master 2 RL collared❤️Little Red R.H❤️ I❤️what is mine and protect her heart 25+100% NSFW.DM for pack only #LLA Omega of the Wolfpack #Teamdark", "followers_count": 2409, "friends_count": 805, "statues_count": 43690 }, "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("2015-11-23T16:14:16.000Z"), "id": 668945695882084352, "text": "My momma told me to watch out for girls like you!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 82214299, "name": "Mr. Hollywood", "screen_name": "chandlerward27", "lang": "en", "location": "Utah, USA", "create_at": date("2009-10-13"), "description": "Sunshine living in the 801. ☀️", "followers_count": 1164, "friends_count": 735, "statues_count": 5186 }, "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("2015-11-23T16:14:16.000Z"), "id": 668945695970193408, "text": "@CandaceHavens Congratulations! You'll have fun buying little dresses!", "in_reply_to_status": 668939303339343872, "in_reply_to_user": 15831105, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15831105 }}, "user": { "id": 17819092, "name": "Gail Pennington", "screen_name": "GailPennington", "lang": "en", "location": "St. Louis, Mo.", "create_at": date("2008-12-02"), "description": "St. Louis Post-Dispatch TV critic, retro-antique dealer, cat lover", "followers_count": 6293, "friends_count": 298, "statues_count": 15982 }, "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("2015-11-23T16:14:16.000Z"), "id": 668945696012283904, "text": "���������� @LebaneseJesus @Ana_McLusty https://t.co/Vfjdaq5rMg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 21836124, 2367581341 }}, "user": { "id": 578269269, "name": "Strawberry Cheekcake", "screen_name": "Ohhstephhh", "lang": "en", "location": "Miami, FL", "create_at": date("2012-05-12"), "description": "Beard & beer loving woman @_TheRealGemini ❤️", "followers_count": 529, "friends_count": 404, "statues_count": 34841 }, "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("2015-11-23T16:14:17.000Z"), "id": 668945696301576192, "text": "@BretBaier Your show is the closest to Truth then any other show on Fox. Thanks, Bret, For Being Patriot.", "in_reply_to_status": 668944638888251392, "in_reply_to_user": 18646108, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18646108 }}, "user": { "id": 2916017550, "name": "(G)Colonel Of Tank", "screen_name": "GGGGGGG700", "lang": "en", "location": "null", "create_at": date("2014-12-01"), "description": "Faith, Hope, and Country #CruzCrew #DoLifeBig", "followers_count": 1050, "friends_count": 926, "statues_count": 20662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sapulpa, OK", "id": "0141cf041521033a", "name": "Sapulpa", "place_type": "city", "bounding_box": rectangle("-96.180993,35.940671 -96.029529,36.075337") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40037, "countyName": "Creek", "cityID": 4065400, "cityName": "Sapulpa" } }
+{ "create_at": datetime("2015-11-23T16:14:17.000Z"), "id": 668945696330940416, "text": "@IndianaMBB Why? Why throw the ball full court? Why? @TomCrean you had 3.2 seconds......", "in_reply_to_status": -1, "in_reply_to_user": 791598918, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 791598918, 32127835 }}, "user": { "id": 210855916, "name": "Scott Babb", "screen_name": "eastsoftballdad", "lang": "en", "location": "Joshua 24:15", "create_at": date("2010-11-01"), "description": "Husband, Father, Coach", "followers_count": 114, "friends_count": 298, "statues_count": 395 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Taylorsville, IN", "id": "01761dd60a3d6264", "name": "Taylorsville", "place_type": "city", "bounding_box": rectangle("-85.970282,39.258782 -85.941072,39.31796") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18005, "countyName": "Bartholomew", "cityID": 1875104, "cityName": "Taylorsville" } }
+{ "create_at": datetime("2015-11-23T16:14:17.000Z"), "id": 668945696939085824, "text": "Seems legit, beer for the pain and glue for the bleeding.. https://t.co/y1HtvV7Bmk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3245688319, "name": "Trey Ramirez", "screen_name": "RTREYR1", "lang": "en", "location": "Moses Lake, WA", "create_at": date("2015-06-14"), "description": "null", "followers_count": 175, "friends_count": 159, "statues_count": 2037 }, "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("2015-11-23T16:14:17.000Z"), "id": 668945697429917696, "text": "@queenchey165 I do I do", "in_reply_to_status": 668943161989943296, "in_reply_to_user": 4153869563, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4153869563 }}, "user": { "id": 3437312080, "name": "Queen Kayla", "screen_name": "kaylarayee", "lang": "en", "location": "null", "create_at": date("2015-08-23"), "description": "null", "followers_count": 104, "friends_count": 199, "statues_count": 2219 }, "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("2015-11-23T16:14:17.000Z"), "id": 668945697434173441, "text": "I entered 2 win $3000 and a design consultation in @jonathanadler 's Refresh, Renew, Re-do Sweeps! https://t.co/kvnByNfYB3 #RefreshRenewRedo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RefreshRenewRedo" }}, "user_mentions": {{ 67609370 }}, "user": { "id": 331297112, "name": "*Cookie*", "screen_name": "Brwnbby426", "lang": "en", "location": "North Carolina", "create_at": date("2011-07-07"), "description": "#TeamTaurus Proud mother of three daughters, one of which attends #UNCC. Loves reggae and go go. #Naturalsista. #TeamFibromyalgia. I am who I am. Love me 4 me", "followers_count": 548, "friends_count": 1092, "statues_count": 4379 }, "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("2015-11-23T16:14:17.000Z"), "id": 668945697438306304, "text": "this adorable lady at the place I volunteer just gave me the biggest hug. my day is made", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2411063031, "name": "nat cat", "screen_name": "nataliefanella_", "lang": "en", "location": "null", "create_at": date("2014-03-14"), "description": "a$ap mob.", "followers_count": 511, "friends_count": 431, "statues_count": 2490 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elk Grove Village, IL", "id": "227d358f6ed8d29e", "name": "Elk Grove Village", "place_type": "city", "bounding_box": rectangle("-88.0605,41.983375 -87.935116,42.040488") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1723256, "cityName": "Elk Grove Village" } }
+{ "create_at": datetime("2015-11-23T16:14:17.000Z"), "id": 668945697614467072, "text": "Anderson .Paak - Am I Wrong - Listen Mobile | Audiomack https://t.co/QtzfZfzcOp @AndersonPaak @pomobeats @ScHoolBoyQ @1Omarion", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 97737655, 1228329978, 22736017, 31652181 }}, "user": { "id": 512966597, "name": "Max Kidder", "screen_name": "Maxthekidder", "lang": "en", "location": "Tallahassee,Fl Las Vegas,NV", "create_at": date("2012-03-03"), "description": "Owner of Mad Max Mgmt. Mgr of Sonnie Lavoy, ADRO, and DJ Suupaa", "followers_count": 155, "friends_count": 177, "statues_count": 4875 }, "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("2015-11-23T16:14:17.000Z"), "id": 668945698004402176, "text": "https://t.co/Z5rN4u4xOZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 231302647, "name": "Ben Hulsey", "screen_name": "_BenH_", "lang": "en", "location": "Seattle, WA", "create_at": date("2010-12-27"), "description": "IBEW member, pro labor, lefty, ex cook", "followers_count": 77, "friends_count": 256, "statues_count": 1732 }, "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:17.000Z"), "id": 668945698046414848, "text": "This https://t.co/FnpT9OcLAq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2465026147, "name": "ali carter-hendrix", "screen_name": "alijahborjas_", "lang": "en", "location": "TMHS", "create_at": date("2014-04-26"), "description": "RIP Tyler. 11-2-15", "followers_count": 365, "friends_count": 400, "statues_count": 5422 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:17.000Z"), "id": 668945698218471424, "text": "A 2010 Volkswagen Passat was just scanned near Fresno, CA 93703 https://t.co/Pa268hmek6 #ios #usedcar #myvinny", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.79,36.77"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ios", "usedcar", "myvinny" }}, "user": { "id": 2361533634, "name": "Vinny Scans", "screen_name": "Vinny_Scans", "lang": "en", "location": "null", "create_at": date("2014-02-25"), "description": "In the market for a #usedcar? Ever wonder what your #car is worth? @Vinny_Says can help.", "followers_count": 782, "friends_count": 1, "statues_count": 440835 }, "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("2015-11-23T16:14:17.000Z"), "id": 668945698402897921, "text": "Run that shit fam!", "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": 11347 }, "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:17.000Z"), "id": 668945698516267011, "text": "@MaddyAllain https://t.co/DcrZeM8mO8", "in_reply_to_status": -1, "in_reply_to_user": 957614400, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 957614400 }}, "user": { "id": 2219521667, "name": "emma", "screen_name": "emma_anderosn14", "lang": "en", "location": "fairly local", "create_at": date("2013-12-11"), "description": "wfhs '17 cross country and in my spare time, a professional dog lover", "followers_count": 272, "friends_count": 389, "statues_count": 4398 }, "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("2015-11-23T16:14:17.000Z"), "id": 668945698797297664, "text": "When it comes to receiving refugees, where's Brazil? Huge country! China? Russia?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21833728, "name": "Brasilmagic", "screen_name": "Brasilmagic", "lang": "en", "location": "Washington DC, USA", "create_at": date("2009-02-24"), "description": "Brazilian & American, Sociologist. Co-host of Road to Reason. Not a fan of libertarians & tea partiers. #Atheist #UniteBlue #Science #Technology #Health", "followers_count": 13411, "friends_count": 12083, "statues_count": 81246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Bethesda, MD", "id": "d3c4d514e13810f3", "name": "North Bethesda", "place_type": "city", "bounding_box": rectangle("-77.155886,39.012634 -77.084619,39.074304") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2456337, "cityName": "North Bethesda" } }
+{ "create_at": datetime("2015-11-23T16:14:17.000Z"), "id": 668945698902159360, "text": "lol luv physics", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 808034641, "name": "ebb", "screen_name": "e_blanchfield", "lang": "en", "location": "null", "create_at": date("2012-09-06"), "description": "ths '16", "followers_count": 220, "friends_count": 199, "statues_count": 4446 }, "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("2015-11-23T16:14:17.000Z"), "id": 668945698969096192, "text": "One of the happiest nights of the year is tonight �� #TeamCrikey #DWTS https://t.co/QA7sAAm2g8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TeamCrikey", "DWTS" }}, "user": { "id": 354549724, "name": "Lucca Petrucci", "screen_name": "lucca_petrucci", "lang": "en", "location": "null", "create_at": date("2011-08-13"), "description": "Business Marketing Student. Social Media Intern. Disney College Program Alumni. Adventure Seeker.", "followers_count": 123, "friends_count": 295, "statues_count": 378 }, "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("2015-11-23T16:14:17.000Z"), "id": 668945699690692608, "text": "Plugging in the Christmas tree... #CAGP15 @ Gaylord Palms Resort Florida https://t.co/EAsXUieNWf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.52695871,28.34276301"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CAGP15" }}, "user": { "id": 405714281, "name": "Central FL Top 5", "screen_name": "CentralFLTop5", "lang": "en", "location": "Central Florida & beyond!", "create_at": date("2011-11-05"), "description": "#CentralFlorida for residents and vistors alike!\nContact: info@centralfloridatop5.com #news #events #business #tech #culture #charity #food #tourism #LoveFL", "followers_count": 10098, "friends_count": 6292, "statues_count": 35322 }, "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": 12097, "countyName": "Osceola" } }
+{ "create_at": datetime("2015-11-23T16:14:17.000Z"), "id": 668945699736698880, "text": "@S_R_P_S Thanks for following & adding me to list of Women Worth Listening To. Sometimes it seems that no one hears me.", "in_reply_to_status": -1, "in_reply_to_user": 739267692, "favorite_count": 0, "coordinate": point("-74.03843627,41.08051125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 739267692 }}, "user": { "id": 106860313, "name": "Suzanne Du Charme", "screen_name": "SuzanneDuCharme", "lang": "en", "location": "New York Metro", "create_at": date("2010-01-20"), "description": "Third generation native New Yorker. Loving Wife*Actor*Dog Mom*Arts Education Supporter*Nicely Nutty", "followers_count": 426, "friends_count": 996, "statues_count": 9139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chestnut Ridge, NY", "id": "0385d78e2458d3c9", "name": "Chestnut Ridge", "place_type": "city", "bounding_box": rectangle("-74.081706,41.063498 -74.032612,41.102245") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36087, "countyName": "Rockland", "cityID": 3615400, "cityName": "Chestnut Ridge" } }
+{ "create_at": datetime("2015-11-23T16:14:17.000Z"), "id": 668945699740983297, "text": "@BigDawsTv yes", "in_reply_to_status": 668944692919209984, "in_reply_to_user": 593161327, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 593161327 }}, "user": { "id": 1367233044, "name": "Manny", "screen_name": "Manny_Kun", "lang": "en", "location": "Murfreesboro, TN", "create_at": date("2013-04-20"), "description": "SSBM | LoL | Esports | Anime | Team UMR |", "followers_count": 739, "friends_count": 553, "statues_count": 36174 }, "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("2015-11-23T16:14:17.000Z"), "id": 668945699770253313, "text": "love being a barista \nhate old men flirting with me and thinking it'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": 2853727579, "name": "ciara", "screen_name": "itsciararaye", "lang": "en", "location": "washington", "create_at": date("2014-10-12"), "description": "party on wayne party on garth", "followers_count": 503, "friends_count": 256, "statues_count": 10471 }, "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("2015-11-23T16:14:17.000Z"), "id": 668945699967512576, "text": "I need to go back to Alabama", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2993399543, "name": "NO_FLYzone✈️", "screen_name": "MccoyKolby", "lang": "en", "location": "ripUNCLEANDREA FOOTBALL=LIFE", "create_at": date("2015-01-22"), "description": "null", "followers_count": 282, "friends_count": 325, "statues_count": 571 }, "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("2015-11-23T16:14:17.000Z"), "id": 668945700055568384, "text": "Hour left of work. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 492408276, "name": "Pasketti", "screen_name": "Anch0rdD0wn", "lang": "en", "location": "wisco", "create_at": date("2012-02-14"), "description": "I'm in my lane, get out my way. #WhootieGang", "followers_count": 488, "friends_count": 242, "statues_count": 33864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Pleasant, WI", "id": "009327fe192849b0", "name": "Mount Pleasant", "place_type": "city", "bounding_box": rectangle("-87.910314,42.667745 -87.786808,42.755807") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55101, "countyName": "Racine", "cityID": 5554875, "cityName": "Mount Pleasant" } }
+{ "create_at": datetime("2015-11-23T16:14:17.000Z"), "id": 668945700101750785, "text": "hbd to my hot frat boys!!!!!! I miss y'all!! take off ur sunglasses, y'all are too sweet to look like douches�� https://t.co/dh0aynaowj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 619786359, "name": "perri chan", "screen_name": "perrichan_", "lang": "en", "location": "bartlett", "create_at": date("2012-06-26"), "description": "ig: @perrileilanichan | sc: @perrileilani", "followers_count": 1023, "friends_count": 433, "statues_count": 20345 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945700273696768, "text": "@DaireneAponte lmao nah not you", "in_reply_to_status": 668933537177841664, "in_reply_to_user": 547034407, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 547034407 }}, "user": { "id": 289705621, "name": "Excuse my French ", "screen_name": "coke_boy236", "lang": "en", "location": "781", "create_at": date("2011-04-28"), "description": "null", "followers_count": 214, "friends_count": 260, "statues_count": 3225 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945700546220032, "text": "Evening Drizzle @ CenturyLink Field https://t.co/VrsPg8ChU7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.33183255,47.59506553"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 422136728, "name": "Brad Leege", "screen_name": "bradleege", "lang": "en", "location": "Madison, WI", "create_at": date("2011-11-26"), "description": "Mobile at @Mapbox. Traveler. #MiaSanMia #EBFG Opinions are my own. !RT.equals(endorsement)", "followers_count": 496, "friends_count": 595, "statues_count": 8420 }, "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:18.000Z"), "id": 668945700793749506, "text": "@Dannation_ who? @andrewpowell113", "in_reply_to_status": 668945525861822464, "in_reply_to_user": 413529269, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 413529269, 2704712626 }}, "user": { "id": 50208968, "name": "Greg Percoco", "screen_name": "GregPercoco", "lang": "en", "location": "null", "create_at": date("2009-06-23"), "description": "i get goin'", "followers_count": 693, "friends_count": 406, "statues_count": 24722 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945701141770241, "text": "Dark or light?\nI always go w dark ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1212820765, "name": "nikki", "screen_name": "BritneyNicole01", "lang": "en", "location": "null", "create_at": date("2013-02-23"), "description": "caution tape around my heart", "followers_count": 147, "friends_count": 139, "statues_count": 9788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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:18.000Z"), "id": 668945701162778624, "text": "@serg_reyes what're we gonna do?!", "in_reply_to_status": 668940848562724865, "in_reply_to_user": 1654685371, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1654685371 }}, "user": { "id": 51679315, "name": "Jordan DiGiulio", "screen_name": "JordanDiGiulio", "lang": "en", "location": "Reno", "create_at": date("2009-06-28"), "description": "22. Livin' the dream since '9⃣2⃣ IG-jordandigiulio", "followers_count": 713, "friends_count": 375, "statues_count": 25297 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945701334831105, "text": "I hate overthinking about somebody", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2153019437, "name": "angel", "screen_name": "angelpereira430", "lang": "en", "location": "null", "create_at": date("2013-10-25"), "description": "I post up and that's confident", "followers_count": 284, "friends_count": 59, "statues_count": 3665 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945701389266944, "text": "I THOUGHT I LOST MY BIGGO PHONE FOR A SECOND #MTVSTARS #VideoMTV2015 5 Seconds Of Summer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MTVSTARS", "VideoMTV2015" }}, "user": { "id": 217480023, "name": "ana :-) TL", "screen_name": "STRlPPEDMGC", "lang": "en-gb", "location": "rowyso Dallas & slfl Dallas ", "create_at": date("2010-11-19"), "description": "the blood in my veins is made up of mistake", "followers_count": 16861, "friends_count": 4931, "statues_count": 116540 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945701599059970, "text": "Your relationship should erase your tears, not your smile. True love doesn't hurt, it heals.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1363131684, "name": "Autumn", "screen_name": "AutumnMuffley", "lang": "en", "location": "null", "create_at": date("2013-04-18"), "description": "▪️Romans 8:18▪️", "followers_count": 245, "friends_count": 161, "statues_count": 2377 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945702064558081, "text": "@TokenPsycho I believe in paying it forward forgive me if I don't make sense... It was done to me when I was your age. Us single women...", "in_reply_to_status": 668945034939604992, "in_reply_to_user": 1327917463, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1327917463 }}, "user": { "id": 22644848, "name": "Houston Socialite", "screen_name": "mshouston", "lang": "en", "location": "ÜT: 29.849112,-95.564121", "create_at": date("2009-03-03"), "description": "Life and times of a Socialite in HTown. Lover of Real hip hop & studing #MMA. I am the hostess of http://www.htownsocialite.com", "followers_count": 778, "friends_count": 584, "statues_count": 27496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:18.000Z"), "id": 668945702114844672, "text": "Rest in peace, little guy ❤ https://t.co/9HGk3LQg4P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 750110616, "name": "Ondine", "screen_name": "fille_du_fleuve", "lang": "en", "location": "null", "create_at": date("2012-08-10"), "description": "going with the flow, coaxing opportunity", "followers_count": 225, "friends_count": 473, "statues_count": 13035 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945702463119360, "text": "if we talk and you still buddy buddy wit your ex it's a complete dub", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1145980616, "name": "g", "screen_name": "gvercetti", "lang": "en", "location": "null", "create_at": date("2013-02-03"), "description": "sunkissed punjabi - ig: g.vercetti", "followers_count": 2142, "friends_count": 754, "statues_count": 56223 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945702563815425, "text": "#firetomcream. Great example of his idiotic coaching", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "firetomcream" }}, "user": { "id": 1312554018, "name": "Monty Burdette", "screen_name": "mlb_hoosier", "lang": "en", "location": "Central Florida", "create_at": date("2013-03-28"), "description": "null", "followers_count": 17, "friends_count": 53, "statues_count": 492 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945702630924289, "text": "@S18Hanna thank you!!!���� love and miss u tooo!!", "in_reply_to_status": 668944103284080640, "in_reply_to_user": 1653567492, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1653567492 }}, "user": { "id": 2394454220, "name": "Marley Kae", "screen_name": "MarleyGrealis", "lang": "en", "location": "null", "create_at": date("2014-03-17"), "description": "Fashion Illustrator", "followers_count": 162, "friends_count": 221, "statues_count": 1206 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945702660083712, "text": "@jorelodell setting up his @arrichannel #AlexaMini #Carina for today's shoot with @chatchett28.… https://t.co/gQZO6TECgn", "in_reply_to_status": -1, "in_reply_to_user": 2677617092, "favorite_count": 0, "coordinate": point("-118.09666166,34.17787564"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AlexaMini", "Carina" }}, "user_mentions": {{ 2677617092, 70743241, 67127767 }}, "user": { "id": 2193294266, "name": "Alex Fuerst", "screen_name": "Alex_Fuerst_DP", "lang": "en", "location": "Dallas/Ft. Worth, Texas", "create_at": date("2013-11-13"), "description": "Professional DP + Cinematographer who's passionate about visual storytelling, technology & filmmaking. Trained by ASC members & world class filmmakers.", "followers_count": 216, "friends_count": 305, "statues_count": 986 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945703058690048, "text": "@tim_guthrie @drrewdrriscoll @WORLDSTARVlNE literally almost tagged both of you in this an hour ago", "in_reply_to_status": 668944054139400193, "in_reply_to_user": 17407263, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17407263, 306890314, 1204567627 }}, "user": { "id": 277733441, "name": "SAVY", "screen_name": "savannahhall13", "lang": "en", "location": "null", "create_at": date("2011-04-05"), "description": "Practice Reckless Optimism- Harto", "followers_count": 144, "friends_count": 475, "statues_count": 435 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marysville, OH", "id": "79e69adde79a23b4", "name": "Marysville", "place_type": "city", "bounding_box": rectangle("-83.401316,40.199677 -83.298207,40.274078") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39159, "countyName": "Union", "cityID": 3948160, "cityName": "Marysville" } }
+{ "create_at": datetime("2015-11-23T16:14:18.000Z"), "id": 668945703100633088, "text": "Cavs game with my man ��❤️ @ Quicken Loans Arena https://t.co/giylfTeRPd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.68801044,41.49663581"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 760750356, "name": "Tate Ritchie", "screen_name": "Tatee_Ritchiee", "lang": "en", "location": "Cleveland, OH", "create_at": date("2012-08-15"), "description": "LEC | Running | Accounting | CLE ❤️ - - - Life is only as good as your mindset", "followers_count": 834, "friends_count": 362, "statues_count": 8441 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945703209570304, "text": "This reminded me of @c3135t32120 https://t.co/cfalR1RVzr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1247112415 }}, "user": { "id": 2460715406, "name": "jarod mendoza", "screen_name": "jarodmendoza45", "lang": "en", "location": "dont worry about it ", "create_at": date("2014-04-23"), "description": "hi I'm Jarod", "followers_count": 386, "friends_count": 387, "statues_count": 2217 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945703217991680, "text": "Album done. Release date Friday. https://t.co/IDcU8Wd9Ek", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 23597952, "name": "Ben Famous", "screen_name": "MrBenFamous", "lang": "en", "location": "null", "create_at": date("2009-03-10"), "description": "http://GDMWorldwide.com • Designer/Musician", "followers_count": 514, "friends_count": 414, "statues_count": 13270 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945703457193985, "text": "@DouchyLafell I hope I can quote this and troll ya later :p", "in_reply_to_status": 668945546204332033, "in_reply_to_user": 3024398673, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3024398673 }}, "user": { "id": 271143081, "name": "Cole Florian", "screen_name": "ColeFloFlo", "lang": "en", "location": "New York", "create_at": date("2011-03-23"), "description": "Go Bills and Blues #BillsMafia #CFC", "followers_count": 685, "friends_count": 349, "statues_count": 24156 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945703499128837, "text": "@stephaniemuhs LMAO A TAMP", "in_reply_to_status": 668943810890752000, "in_reply_to_user": 33034694, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 33034694 }}, "user": { "id": 1143628850, "name": "αnnα lαchímíα", "screen_name": "annalachimiaaa", "lang": "en", "location": "null", "create_at": date("2013-02-02"), "description": "21 | culinary student | foodie | in love with my best friend❤️ | NYC bound |", "followers_count": 174, "friends_count": 231, "statues_count": 16287 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945703645802496, "text": "https://t.co/FZhKyWv72e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "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": 13864 }, "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:18.000Z"), "id": 668945703813517312, "text": "Baybrook flow before this meeting ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 540432047, "name": "199X", "screen_name": "Mhernandeez_", "lang": "en", "location": "Htx.", "create_at": date("2012-03-29"), "description": "20 | #stickypromo cocktail waitress ❤️", "followers_count": 2730, "friends_count": 1425, "statues_count": 52906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Webster, TX", "id": "35e409cdfaf2527a", "name": "Webster", "place_type": "city", "bounding_box": rectangle("-95.164345,29.497368 -95.08994,29.555012") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4876948, "cityName": "Webster" } }
+{ "create_at": datetime("2015-11-23T16:14:18.000Z"), "id": 668945703939530752, "text": "always in the best mood with sam ☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1737800516, "name": "emilee riley", "screen_name": "RileyEmilee", "lang": "en", "location": "null", "create_at": date("2013-09-06"), "description": "always winning ❥ @hostetter99", "followers_count": 486, "friends_count": 359, "statues_count": 6873 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945704136609792, "text": "\"I'm not good for you , you're no good 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": 2467955806, "name": "lauren ❥", "screen_name": "lauren_bass_", "lang": "en", "location": "Rocky Mount, NC", "create_at": date("2014-04-07"), "description": ":((", "followers_count": 332, "friends_count": 352, "statues_count": 12237 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945704279105536, "text": "MacNeil", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4328969354, "name": "William Greenway", "screen_name": "wdgj1952", "lang": "en", "location": "null", "create_at": date("2015-11-22"), "description": "null", "followers_count": 1, "friends_count": 25, "statues_count": 6 }, "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("2015-11-23T16:14:18.000Z"), "id": 668945704337866752, "text": "lies ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1724125482, "name": "ᗩlexx", "screen_name": "Alexxandraa_00", "lang": "en", "location": "null", "create_at": date("2013-09-02"), "description": "Sounds Goods, Feels Good.", "followers_count": 589, "friends_count": 993, "statues_count": 13090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hobbs, NM", "id": "fc045378889ef7d6", "name": "Hobbs", "place_type": "city", "bounding_box": rectangle("-103.201769,32.680274 -103.093307,32.803199") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35025, "countyName": "Lea", "cityID": 3532520, "cityName": "Hobbs" } }
+{ "create_at": datetime("2015-11-23T16:14:19.000Z"), "id": 668945704556056577, "text": "When you gonna stop running them jewels and run for office, @KillerMike? https://t.co/1E50nq4ENo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21265120 }}, "user": { "id": 402949882, "name": "Dan Mistich", "screen_name": "drmistich", "lang": "en", "location": "Georgia, USA", "create_at": date("2011-11-01"), "description": "I snore. You don't have to remind me.", "followers_count": 622, "friends_count": 1603, "statues_count": 5691 }, "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("2015-11-23T16:14:19.000Z"), "id": 668945704832733184, "text": "Soccer was chill today", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 971308332, "name": "Lil K.O", "screen_name": "bryan09lopez", "lang": "en", "location": "Salem, OR", "create_at": date("2012-11-25"), "description": "Senior @ Mck/CowboyNation/ MMA fighter/ U.S Air Force ✈️ / @rondarousey is my girl/ 13 KO's with a great right uppercut!", "followers_count": 728, "friends_count": 704, "statues_count": 27784 }, "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("2015-11-23T16:14:19.000Z"), "id": 668945705164140544, "text": "@SMACKHighWA do you even go to school? *sophomores", "in_reply_to_status": 668941206290890752, "in_reply_to_user": 1590874910, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1590874910 }}, "user": { "id": 4294548014, "name": "Lena Beana", "screen_name": "kalenafaith", "lang": "en", "location": "Lake Stevens, WA", "create_at": date("2015-11-19"), "description": "i say sweet beets and nice rice a little too much ~ just love ~ Junior @ LSHS", "followers_count": 38, "friends_count": 58, "statues_count": 27 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Stevens, WA", "id": "d44cb984bf75455e", "name": "Lake Stevens", "place_type": "city", "bounding_box": rectangle("-122.086456,48.002337 -122.048347,48.034111") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5337900, "cityName": "Lake Stevens" } }
+{ "create_at": datetime("2015-11-23T16:14:19.000Z"), "id": 668945705289932800, "text": "Happy birthday @Starkweather94! Hope to see you this week!! Have fun! ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 184761974 }}, "user": { "id": 24093144, "name": "αѕнℓєу fяαикℓιи", "screen_name": "ashleyybrook", "lang": "en", "location": "IL", "create_at": date("2009-03-12"), "description": "null", "followers_count": 1006, "friends_count": 923, "statues_count": 17685 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Herrin, IL", "id": "ef04eacbee05f667", "name": "Herrin", "place_type": "city", "bounding_box": rectangle("-89.06759,37.773879 -88.999855,37.847052") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17199, "countyName": "Williamson", "cityID": 1734358, "cityName": "Herrin" } }
+{ "create_at": datetime("2015-11-23T16:14:19.000Z"), "id": 668945705789218817, "text": "Planning out your future really isn't fun", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 320375971, "name": "Courtney Bosetti", "screen_name": "courttbos", "lang": "en", "location": "Akron, OH", "create_at": date("2011-06-19"), "description": "A life without love isn't a life worth living. PR student, Pittsburgh gal, Day Dreamer, Giraffe Lover. Rocco is pretty chill too.", "followers_count": 369, "friends_count": 307, "statues_count": 8502 }, "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("2015-11-23T16:14:19.000Z"), "id": 668945706099585024, "text": "@ustin611 it's an accounting night class, it's not even about learning. It's about surviving", "in_reply_to_status": 668945402004152322, "in_reply_to_user": 1481046247, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1481046247 }}, "user": { "id": 464288431, "name": "Trill Cosby", "screen_name": "BreydonGates", "lang": "en", "location": "null", "create_at": date("2012-01-14"), "description": "22. DISTANCE RUNNER. SKATEBOARDING. RADDAD. IG: greydonbates SC:bgates93", "followers_count": 353, "friends_count": 303, "statues_count": 5638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portsmouth, OH", "id": "8a6345658f3277de", "name": "Portsmouth", "place_type": "city", "bounding_box": rectangle("-83.02068,38.726819 -82.932079,38.786816") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39145, "countyName": "Scioto", "cityID": 3964304, "cityName": "Portsmouth" } }
+{ "create_at": datetime("2015-11-23T16:14:19.000Z"), "id": 668945706279800832, "text": ".@JaradB went all the way down to @RAYGUNshirts to buy a @DrakeUGriff shirt and they closed at 6. Y not extended hours for our future Prez?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14812896, 16656254, 3826398072 }}, "user": { "id": 433886273, "name": "Lilianna Bernstein", "screen_name": "LhappyB", "lang": "en", "location": "Des Moines, IA", "create_at": date("2011-12-10"), "description": "#DSM and @drakeneighbor resident. Aspiring trophy wife to @jaradb. #puggle mother. I spend a lot of time at @drakeuniversity. I love all of the above.", "followers_count": 273, "friends_count": 369, "statues_count": 1375 }, "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("2015-11-23T16:14:19.000Z"), "id": 668945706368049152, "text": "Фото 16 https://t.co/erHJ2K5gu7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-123.01198501,37.69979013"), "retweet_count": 0, "lang": "ru", "is_retweet": false, "user": { "id": 3202217836, "name": "Samantha Drake", "screen_name": "samadron", "lang": "en", "location": "null", "create_at": date("2015-04-24"), "description": "null", "followers_count": 11, "friends_count": 0, "statues_count": 23287 }, "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("2015-11-23T16:14:19.000Z"), "id": 668945706841825280, "text": "New phone. Text me w/ names.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 134610456, "name": "Tae", "screen_name": "Therealtiana__", "lang": "en", "location": "Cali", "create_at": date("2010-04-18"), "description": "Follow your heart where it leads you.", "followers_count": 757, "friends_count": 576, "statues_count": 21666 }, "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("2015-11-23T16:14:19.000Z"), "id": 668945707001253888, "text": "Where my Young Pappy fans at?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 332423321, "name": "Malik Harris", "screen_name": "malikharris2ft", "lang": "en", "location": "Eden Prairie, Minnesota", "create_at": date("2011-07-09"), "description": "Peace & Positivity", "followers_count": 1456, "friends_count": 1412, "statues_count": 12870 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eden Prairie, MN", "id": "a1a25de9f5efa69b", "name": "Eden Prairie", "place_type": "city", "bounding_box": rectangle("-93.520913,44.799239 -93.397928,44.893162") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2718116, "cityName": "Eden Prairie" } }
+{ "create_at": datetime("2015-11-23T16:14:19.000Z"), "id": 668945707013955584, "text": "Really I'm the plug , really I'm 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": 556430466, "name": "✨Twinziess✨", "screen_name": "twiinzballin4L", "lang": "en", "location": "null", "create_at": date("2012-04-17"), "description": "Somewhere Balling ..", "followers_count": 647, "friends_count": 351, "statues_count": 12524 }, "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("2015-11-23T16:14:19.000Z"), "id": 668945707039072256, "text": "Damn if I ever pursued u should of fcked wit me lmao", "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": "#VoteBernieSanders16", "screen_name": "lxxus", "lang": "en", "location": "ATL✈️ DC✈️ CobbCo", "create_at": date("2010-02-25"), "description": "@Sycegame OG | 5'11| TrapHouse | 156 Forsyth St SW | Serial Entrepreneur | Throw Experiences Not Parties | Manager | Owner | Investor", "followers_count": 2080, "friends_count": 499, "statues_count": 248249 }, "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("2015-11-23T16:14:19.000Z"), "id": 668945707244453888, "text": "ΖΦΒ Informational ����\nDec. 2\nEmail RhoDelta_72@yahoo.com to RSVP \n\n#UNT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "UNT" }}, "user": { "id": 915900732, "name": "Janet", "screen_name": "_JanetBrasi", "lang": "en", "location": "Denton, TX", "create_at": date("2012-10-30"), "description": "UNT'17 • Aspiring Social Worker • ΖΦΒ Sorority, Inc. • Spr'15 QU4D • BSE Ambassador • Life is really simple, but we insist on making it hard.", "followers_count": 838, "friends_count": 851, "statues_count": 40400 }, "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:19.000Z"), "id": 668945707353575425, "text": "can you tell she's my mom https://t.co/b07bd0gwxs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 253195663, "name": "Mads", "screen_name": "its_Madisonn", "lang": "en", "location": "null", "create_at": date("2011-02-16"), "description": "i go to college", "followers_count": 770, "friends_count": 437, "statues_count": 28795 }, "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:19.000Z"), "id": 668945707621945344, "text": "@mansoor99ha We'll get back to u quickly. Need to submit one? PLS fill up this form: https://t.co/hogDBF28Ie Thanks! Margo from the HSS Team", "in_reply_to_status": 668945232155774976, "in_reply_to_user": 16332734, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 310921876 }}, "user": { "id": 16332734, "name": "Hotspot Shield", "screen_name": "HotspotShield", "lang": "en", "location": "The Internet", "create_at": date("2008-09-17"), "description": "Hotspot Shield lets you browse the web securely, anonymously, and change your virtual location to access blocked content.\nFree download: http://smarturl.it/HSS", "followers_count": 68130, "friends_count": 290, "statues_count": 11366 }, "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.228635,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("2015-11-23T16:14:19.000Z"), "id": 668945707747966976, "text": "This is beautiful https://t.co/3RmEfkylMS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 166441178, "name": "☠ Jolynn ☠", "screen_name": "joleewalker", "lang": "en", "location": "Greenville, NC", "create_at": date("2010-07-13"), "description": "In Trap, we trust. My ambition is my weaponry -Niykee Heaton Never settle -Bdub |-/#LNHS15 #ECU19 #Undaunted #BoneyardReady☠ #ItsOkayToNotBeOkay", "followers_count": 369, "friends_count": 1262, "statues_count": 2436 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boone, NC", "id": "ecdf2813e684c066", "name": "Boone", "place_type": "city", "bounding_box": rectangle("-81.7384,36.183417 -81.57739,36.246767") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37189, "countyName": "Watauga", "cityID": 3707080, "cityName": "Boone" } }
+{ "create_at": datetime("2015-11-23T16:14:19.000Z"), "id": 668945707777179648, "text": "I hate you so much I think I'm going to die from it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19315251, "name": "Kaylee Foschino", "screen_name": "kayleefosch", "lang": "en", "location": "Mount Juliet, TN", "create_at": date("2009-01-21"), "description": "idk my bff Jill? #geauxtigers", "followers_count": 1282, "friends_count": 568, "statues_count": 90661 }, "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("2015-11-23T16:14:19.000Z"), "id": 668945708142206976, "text": "Taylor and I just go to the auction to check out the young guys ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2230932070, "name": "Heather ➰", "screen_name": "ShadeHeather", "lang": "en", "location": "null", "create_at": date("2013-12-17"), "description": "Never settle for being someone's other, when you know you have the potential to be someone's only.", "followers_count": 205, "friends_count": 262, "statues_count": 6961 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hagerstown, MD", "id": "fb4e19c906cdaa73", "name": "Hagerstown", "place_type": "city", "bounding_box": rectangle("-77.786448,39.592442 -77.671529,39.676503") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24043, "countyName": "Washington", "cityID": 2436075, "cityName": "Hagerstown" } }
+{ "create_at": datetime("2015-11-23T16:14:19.000Z"), "id": 668945708158861315, "text": "@SlimChoncho21 preciate that !", "in_reply_to_status": 668945589820743681, "in_reply_to_user": 1362221444, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1362221444 }}, "user": { "id": 2391137581, "name": "TeamFaith", "screen_name": "latson95", "lang": "en", "location": "Bossier City, LA ✈️ Grambling", "create_at": date("2014-03-15"), "description": "Child of a King. Mckela B. Williams ❤️", "followers_count": 398, "friends_count": 521, "statues_count": 10762 }, "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("2015-11-23T16:14:19.000Z"), "id": 668945708280492032, "text": "i dont know about nobody else , but im really happy in my relationship..❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2428522657, "name": "..", "screen_name": "makiasw", "lang": "en", "location": "#longlivetaj.", "create_at": date("2014-04-05"), "description": "I can do all things through Christ who strengthens me.. -Philippians 4:13 ❤️ i love you @blanco2__", "followers_count": 602, "friends_count": 550, "statues_count": 12664 }, "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("2015-11-23T16:14:19.000Z"), "id": 668945708456787968, "text": "Alright Jake I'm coming 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": 636259945, "name": "Nick Moffett", "screen_name": "Moffnic304", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2012-07-15"), "description": "null", "followers_count": 525, "friends_count": 407, "statues_count": 5099 }, "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("2015-11-23T16:14:19.000Z"), "id": 668945708553080832, "text": "@jaixhf I kept it with me for two years", "in_reply_to_status": 668936568443277312, "in_reply_to_user": 2334105355, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2334105355 }}, "user": { "id": 2163648338, "name": "manga-reth", "screen_name": "margawrathh", "lang": "en", "location": "null", "create_at": date("2013-10-29"), "description": "don't waste your love on someone who doesn't value it", "followers_count": 120, "friends_count": 125, "statues_count": 1504 }, "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("2015-11-23T16:14:20.000Z"), "id": 668945708683259908, "text": "What do I want for Christmas? For people to stop fucking around and work for change 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": 1136063899, "name": "The♈️Enigma", "screen_name": "KingStormRae", "lang": "en", "location": "Apollo Beach, FL", "create_at": date("2013-01-30"), "description": "We never die we multiply. Never sleeping always creeping. Forces of nature. ♈️♉️♊️♋️♌️♍️♎️♏️♐️♑️♒️♓️", "followers_count": 172, "friends_count": 149, "statues_count": 4817 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Apollo Beach, FL", "id": "de55c30bd4263e8a", "name": "Apollo Beach", "place_type": "city", "bounding_box": rectangle("-82.437901,27.735035 -82.377145,27.793269") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1201675, "cityName": "Apollo Beach" } }
+{ "create_at": datetime("2015-11-23T16:14:20.000Z"), "id": 668945708729376768, "text": "You give me purpose...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 112025417, "name": "Darnell McCray", "screen_name": "DnellsKing", "lang": "en", "location": "null", "create_at": date("2010-02-06"), "description": "GannonU '18 NYC", "followers_count": 451, "friends_count": 363, "statues_count": 13331 }, "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:20.000Z"), "id": 668945708884492288, "text": "@Trae_Th3_Truth ugh tru", "in_reply_to_status": 668945644615106560, "in_reply_to_user": 320898356, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 320898356 }}, "user": { "id": 452073282, "name": "M.", "screen_name": "Murwanduh", "lang": "en", "location": "null", "create_at": date("2012-01-01"), "description": "i'm a good person with bad habits", "followers_count": 1467, "friends_count": 712, "statues_count": 33116 }, "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("2015-11-23T16:14:20.000Z"), "id": 668945708913967104, "text": "happy bday @carolinesarno thx 4 the mmrs! btw I'm sure @VaughnEagan wishes you one too https://t.co/IelCt04xtw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 843777084, 357093566 }}, "user": { "id": 251852124, "name": "madeline fleming", "screen_name": "maddiej213", "lang": "en", "location": "UA•2016 WFU•2020", "create_at": date("2011-02-13"), "description": "I'm a glass half full kind of girl", "followers_count": 541, "friends_count": 379, "statues_count": 3889 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Upper Arlington, OH", "id": "61f73dd1739509f0", "name": "Upper Arlington", "place_type": "city", "bounding_box": rectangle("-83.104102,39.989304 -83.042267,40.063889") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3979002, "cityName": "Upper Arlington" } }
+{ "create_at": datetime("2015-11-23T16:14:20.000Z"), "id": 668945708951584768, "text": "@maaacmorris THATS MY BEST FRIEND����", "in_reply_to_status": 668945271259160576, "in_reply_to_user": 3247524126, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3247524126 }}, "user": { "id": 2614866336, "name": "Stephanie", "screen_name": "stef_gonzalez_", "lang": "en", "location": "null", "create_at": date("2014-07-09"), "description": "Dream big", "followers_count": 68, "friends_count": 197, "statues_count": 446 }, "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:20.000Z"), "id": 668945708959985664, "text": "My little brother is such a troublemaker ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3109078448, "name": "Victoria", "screen_name": "whitehouse_vic", "lang": "en", "location": "null", "create_at": date("2015-03-26"), "description": "gvhs | soccer | Antonio ❤ \nMontevallo University committ ⚽", "followers_count": 257, "friends_count": 239, "statues_count": 1286 }, "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("2015-11-23T16:14:20.000Z"), "id": 668945709362712576, "text": "@sanders_graysen https://t.co/0ef86bTly0", "in_reply_to_status": -1, "in_reply_to_user": 801502963, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 801502963 }}, "user": { "id": 615210566, "name": "BChilds", "screen_name": "itsbnastyy", "lang": "en", "location": "Daytona Beach-Americus Georgia", "create_at": date("2012-06-22"), "description": "I simply came here to party. GSW Softball #5 Insta: Brittchilds", "followers_count": 338, "friends_count": 302, "statues_count": 19637 }, "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("2015-11-23T16:14:20.000Z"), "id": 668945709534679040, "text": "Sometimes Your Word Is All You Have So Stay True To 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": 594866852, "name": "Tabb Thugga", "screen_name": "BrooklynBelen_", "lang": "en", "location": "null", "create_at": date("2012-05-30"), "description": "BROOKLYN ✈ Florida \nRIP AMC ☝\nBlessed Family \nBestfriend ? Dayzah \nTabb Thugga Taking Ya Bitch Since 1996", "followers_count": 796, "friends_count": 1243, "statues_count": 38647 }, "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("2015-11-23T16:14:20.000Z"), "id": 668945709761081344, "text": "I want a @jrjanis Jersey for Christmas ! ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 223687614 }}, "user": { "id": 1058256103, "name": "Diego Alvarez", "screen_name": "DAlvarez_95", "lang": "en", "location": "San Antonio, TX", "create_at": date("2013-01-03"), "description": "UTSA\nCyber Security/IT", "followers_count": 151, "friends_count": 360, "statues_count": 535 }, "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:20.000Z"), "id": 668945709865918464, "text": "I get so happy when I see my boyfriend after work cause I go from mad to super happy ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2393657484, "name": "rayleen", "screen_name": "rayleenanahi", "lang": "en", "location": "null", "create_at": date("2014-03-16"), "description": "new to this :(", "followers_count": 93, "friends_count": 110, "statues_count": 183 }, "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("2015-11-23T16:14:20.000Z"), "id": 668945710772068352, "text": "Codeine Crazy >", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2360362356, "name": "toddler❣", "screen_name": "nvj___", "lang": "en", "location": "detroit, mi ", "create_at": date("2014-02-24"), "description": "sunshine & kaila", "followers_count": 832, "friends_count": 1125, "statues_count": 37962 }, "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:20.000Z"), "id": 668945711325708288, "text": "Wassup", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 992076966, "name": "Josh⚜", "screen_name": "Trvvvp96", "lang": "en", "location": " #Saints⚜ #ATG", "create_at": date("2012-12-05"), "description": "40s And Some bad bitches .", "followers_count": 1092, "friends_count": 2118, "statues_count": 4701 }, "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("2015-11-23T16:14:20.000Z"), "id": 668945711577235456, "text": "got in my bed at 6:30\nto do homework and I've been so distracted that I'm just realizing my backpack is downstairs :/", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2187635144, "name": "Bella Carusone", "screen_name": "bella_carusone", "lang": "en", "location": "New York, USA", "create_at": date("2013-11-10"), "description": "Snapchat-Bella Carusone", "followers_count": 269, "friends_count": 298, "statues_count": 1703 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glens Falls, NY", "id": "011edd780200b886", "name": "Glens Falls", "place_type": "city", "bounding_box": rectangle("-73.726867,43.287378 -73.594942,43.379646") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36113, "countyName": "Warren", "cityID": 3629333, "cityName": "Glens Falls" } }
+{ "create_at": datetime("2015-11-23T16:14:20.000Z"), "id": 668945712042942466, "text": "lol three 3's already", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 138448618, "name": "Russy", "screen_name": "RussIsDope", "lang": "en", "location": "ΠΚΦ", "create_at": date("2010-04-29"), "description": "RussIsDope - The Boy", "followers_count": 921, "friends_count": 215, "statues_count": 12865 }, "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("2015-11-23T16:14:20.000Z"), "id": 668945712084869120, "text": "Incidentally, I was thinking Kayla Pittman kinda looked a lil like Mehgan too https://t.co/GTHZMhNwMc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3165560103, "name": "Chief_Cakes", "screen_name": "Still_Sylvia", "lang": "en", "location": "null", "create_at": date("2015-04-13"), "description": "Born & raised in Chicago. Young, but I'm old school...Here for sum discussion, to VENT, but mostly to LAUGH!! So don't u dare kill my vibe! Unless uz a....", "followers_count": 309, "friends_count": 231, "statues_count": 28750 }, "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:20.000Z"), "id": 668945712344932352, "text": "@BaileyEverslage I just ate ugh", "in_reply_to_status": 668945595726458880, "in_reply_to_user": 322990399, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 322990399 }}, "user": { "id": 408739667, "name": "Kayla Everslage", "screen_name": "Kayla_Everslage", "lang": "en", "location": "502 // 859", "create_at": date("2011-11-09"), "description": "|| UK'18 Delta Zeta || vincentlococo ✌️", "followers_count": 640, "friends_count": 139, "statues_count": 15858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsboro Farm, KY", "id": "00728f4d0894d30c", "name": "Brownsboro Farm", "place_type": "city", "bounding_box": rectangle("-85.600403,38.298858 -85.572238,38.315005") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2110162, "cityName": "Brownsboro Farm" } }
+{ "create_at": datetime("2015-11-23T16:14:20.000Z"), "id": 668945712395231232, "text": "I love u baby your beautiful, sexy, and I want to make love to U https://t.co/7tJsbSrE5f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 506414325, "name": "wzemrock", "screen_name": "wzemrock", "lang": "en", "location": "Massachusetts", "create_at": date("2012-02-27"), "description": "null", "followers_count": 111, "friends_count": 276, "statues_count": 276 }, "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("2015-11-23T16:14:20.000Z"), "id": 668945712411865088, "text": "@holytoledoo I was literally laughing my ass off in the dead quiet theater and looked like an asshole������", "in_reply_to_status": 668945305325297665, "in_reply_to_user": 3011944404, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3011944404 }}, "user": { "id": 1517945299, "name": "KT", "screen_name": "KTCoop13", "lang": "en", "location": "Yorba Linda, CA", "create_at": date("2013-06-14"), "description": "YLHS mustang mix", "followers_count": 443, "friends_count": 308, "statues_count": 7780 }, "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("2015-11-23T16:14:20.000Z"), "id": 668945712424448000, "text": "@HatchDoll they thought���� indeed & still gonna die alone", "in_reply_to_status": 668945514772234240, "in_reply_to_user": 311661790, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 311661790 }}, "user": { "id": 784716937, "name": "H ⚡️97", "screen_name": "HaywoodJavonte", "lang": "en", "location": "SC- Haywood_w", "create_at": date("2012-08-27"), "description": "Volleyball⚪️ FL.☀️ 18.♋️ Employed & Enjoying life☺️ James 1:4", "followers_count": 1324, "friends_count": 871, "statues_count": 17634 }, "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("2015-11-23T16:14:20.000Z"), "id": 668945712592257025, "text": "This is why we must fear Trump: He makes the insane Ted Cruz and Marco Rubio look sensible https://t.co/bcRn9Yh17R via @Salon", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16955991 }}, "user": { "id": 19508842, "name": "Jbaby56", "screen_name": "jbaby56", "lang": "en", "location": "null", "create_at": date("2009-01-25"), "description": "Sustainable Green Designer-Renewable, Energy efficient Environmentally friendly design solutions for Residential and Commercial clients.", "followers_count": 2858, "friends_count": 2555, "statues_count": 112770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glen Ridge, NJ", "id": "c4dab45ef7581417", "name": "Glen Ridge", "place_type": "city", "bounding_box": rectangle("-74.218378,40.78572 -74.191401,40.829007") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3426610, "cityName": "Glen Ridge" } }
+{ "create_at": datetime("2015-11-23T16:14:21.000Z"), "id": 668945712810340352, "text": "@Skylar_r_r ��", "in_reply_to_status": -1, "in_reply_to_user": 3035555792, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3035555792 }}, "user": { "id": 3228350234, "name": "ATM", "screen_name": "Austin_M80_", "lang": "en", "location": "tornado alley, OK", "create_at": date("2015-05-27"), "description": "| Future Firefighter | |worlds greatest wingman|. #Thunder #Bucks #CowboysNation #ATLbraves.", "followers_count": 1432, "friends_count": 1467, "statues_count": 17721 }, "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("2015-11-23T16:14:21.000Z"), "id": 668945712856621057, "text": "Cause I'm missing more than just your body", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 447376161, "name": "ʝσαииα. ♡", "screen_name": "Joannaaileen_", "lang": "en", "location": "null", "create_at": date("2011-12-26"), "description": "It’s always, and always will be about you. ❤️ 11/14 god needed a angel", "followers_count": 560, "friends_count": 931, "statues_count": 16659 }, "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:21.000Z"), "id": 668945712898580481, "text": "@OtterBox is for me to Carlos Figueroa", "in_reply_to_status": 668945584280092672, "in_reply_to_user": 15666980, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 15666980 }}, "user": { "id": 4245847813, "name": "carlos m figueroa", "screen_name": "Thefigueroa30", "lang": "en", "location": "Coral Springs, FL", "create_at": date("2015-11-15"), "description": "like to play my playstation and I like @WWE @JOHNCENA", "followers_count": 18, "friends_count": 130, "statues_count": 660 }, "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("2015-11-23T16:14:21.000Z"), "id": 668945713196216320, "text": "you mean nothin to me https://t.co/P0grzlXwAn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 128107984, "name": "KA$$", "screen_name": "kascity_", "lang": "en", "location": "null", "create_at": date("2010-03-30"), "description": "I tweet a lot so yeah", "followers_count": 651, "friends_count": 684, "statues_count": 57145 }, "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("2015-11-23T16:14:21.000Z"), "id": 668945713431212032, "text": "Damn RT @SeanTheTerrible: �� https://t.co/slFtD2HwWp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.073991,26.594766"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 184169712 }}, "user": { "id": 59598291, "name": "Relocated Filth", "screen_name": "ClassicBray", "lang": "en", "location": "Where the sun shines", "create_at": date("2009-07-23"), "description": "I don't like people behind me so don't follow me.", "followers_count": 3315, "friends_count": 492, "statues_count": 206993 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Worth, FL", "id": "000aa4f4407fa88e", "name": "Lake Worth", "place_type": "city", "bounding_box": rectangle("-80.122484,26.587884 -80.04594,26.646173") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1239075, "cityName": "Lake Worth" } }
+{ "create_at": datetime("2015-11-23T16:14:21.000Z"), "id": 668945713435426816, "text": "@bstrawbridge Well, I haven't been happy, but have now concluded there's no way he'll bring banner to B-Town...before just unlikely.", "in_reply_to_status": 668945465648553984, "in_reply_to_user": 23950857, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23950857 }}, "user": { "id": 17922314, "name": "brbinchicago", "screen_name": "brbinchicago", "lang": "en", "location": "Chicago", "create_at": date("2008-12-06"), "description": "Lawyer first, and lover of all things food, sports and travel. Living & eating my way through Lincoln Square.", "followers_count": 444, "friends_count": 856, "statues_count": 9913 }, "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:21.000Z"), "id": 668945713628356609, "text": "I'm pretty sure you'll just go online to send & receive nudes just bc I'm mad at you.. So go ahead.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3900446013, "name": "MaiT", "screen_name": "maseratimai", "lang": "en", "location": "Biloxi, MS", "create_at": date("2015-10-08"), "description": "Blasé Blasé", "followers_count": 71, "friends_count": 116, "statues_count": 232 }, "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("2015-11-23T16:14:21.000Z"), "id": 668945713665982464, "text": "#Retail #Job in #Minden, NV: Store Team Member at Tractor Supply Company https://t.co/rtqcMVnbRG #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.7793287,38.9528066"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "Job", "Minden", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 59877526, "name": "TMJ-NV Retail Jobs", "screen_name": "tmj_nv_retail", "lang": "en", "location": "Nevada", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in Nevada Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 347, "friends_count": 300, "statues_count": 69 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minden, NV", "id": "26c0810ad14db9c8", "name": "Minden", "place_type": "city", "bounding_box": rectangle("-119.799065,38.937493 -119.748749,38.97601") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32005, "countyName": "Douglas", "cityID": 3247000, "cityName": "Minden" } }
+{ "create_at": datetime("2015-11-23T16:14:21.000Z"), "id": 668945713712205824, "text": "My 6 yr old sister keeps snapchatting 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": 427098727, "name": "Mace", "screen_name": "MvceoNumba9", "lang": "en", "location": "null", "create_at": date("2011-12-02"), "description": "You get sloppy drunk, I stay Whiskey neat.", "followers_count": 742, "friends_count": 667, "statues_count": 19242 }, "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("2015-11-23T16:14:21.000Z"), "id": 668945713888411648, "text": "@carbonstealer @misskaddykins https://t.co/Z1Y0a1TGU2", "in_reply_to_status": -1, "in_reply_to_user": 239979047, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 239979047, 340161816 }}, "user": { "id": 2434833876, "name": "Alex Walker", "screen_name": "AlexWalker199", "lang": "en", "location": "null", "create_at": date("2014-04-08"), "description": "“Stab the body and it heals, but injure the heart and the wound lasts a lifetime.” \n― Mineko Iwasaki", "followers_count": 70, "friends_count": 186, "statues_count": 8687 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bradford, PA", "id": "ca239b58a2a550c7", "name": "Bradford", "place_type": "city", "bounding_box": rectangle("-78.691214,41.941184 -78.615049,41.999") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42083, "countyName": "McKean", "cityID": 4208040, "cityName": "Bradford" } }
+{ "create_at": datetime("2015-11-23T16:14:21.000Z"), "id": 668945713926131712, "text": "I <3 my home @ Nashville, Tennessee https://t.co/ENabalWYWL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.78333333,36.16666667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 557867791, "name": "Lily Tillman", "screen_name": "lilyy_tillman", "lang": "en", "location": "null", "create_at": date("2012-04-19"), "description": "belmont '19", "followers_count": 807, "friends_count": 483, "statues_count": 6778 }, "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.99451 -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("2015-11-23T16:14:21.000Z"), "id": 668945714169319424, "text": "The+mission+and+future+of+the+church+are fuelled+by+the+growth+and+potential+of+our+leaders. https://t.co/rcCx9sj9Kw via cnieuwhof", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 764005141, "name": "jacob vanskike", "screen_name": "jacobvanskike", "lang": "en", "location": "omaha", "create_at": date("2012-08-17"), "description": "| heart after the local church | worship pastor @wccomaha #wccworship | @nechristian @ncworshiparts alumnus |", "followers_count": 359, "friends_count": 266, "statues_count": 3856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plattsmouth, NE", "id": "47a0993bb01ddbad", "name": "Plattsmouth", "place_type": "city", "bounding_box": rectangle("-95.909445,40.984448 -95.875781,41.028453") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31025, "countyName": "Cass", "cityID": 3139345, "cityName": "Plattsmouth" } }
+{ "create_at": datetime("2015-11-23T16:14:21.000Z"), "id": 668945714328637440, "text": "Happy birthday to my day 1️⃣ brotha. Hope you have a lavish ass birthday @JustRoot28 https://t.co/sOnRU9XwsN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 721414008 }}, "user": { "id": 1319431982, "name": "#FreeBool", "screen_name": "nikbos22", "lang": "en", "location": "locals only", "create_at": date("2013-03-31"), "description": "Sonoma state TKE RIP C-NEE", "followers_count": 401, "friends_count": 254, "statues_count": 4571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakeside, CA", "id": "03c9c75ede037d93", "name": "Lakeside", "place_type": "city", "bounding_box": rectangle("-116.948624,32.826697 -116.872927,32.872074") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 639766, "cityName": "Lakeside" } }
+{ "create_at": datetime("2015-11-23T16:14:21.000Z"), "id": 668945714358022144, "text": "We're #hiring! Read about our latest #job opening here: Reservations Sales Agent - https://t.co/FDiVgdB9JD #Sunriver, OR #BusinessMgmt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.4386404,43.8840067"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Sunriver", "BusinessMgmt" }}, "user": { "id": 2842684723, "name": "Destination Jobs", "screen_name": "DestinationJobs", "lang": "en", "location": "null", "create_at": date("2014-10-06"), "description": "The Destination Hotels collection includes award-winning hotels & resorts in premier vacation destinations around the U.S. Check here for open jobs.", "followers_count": 249, "friends_count": 2, "statues_count": 916 }, "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": 41017, "countyName": "Deschutes", "cityID": 4171250, "cityName": "Sunriver" } }
+{ "create_at": datetime("2015-11-23T16:14:21.000Z"), "id": 668945714462990337, "text": "Adorable. With @zgeballe at the @seattleaquarium #seaotter #seaweasel @ Seattle Aquarium https://t.co/N1KRSOkIa0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.3429794,47.60783"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "seaotter", "seaweasel" }}, "user_mentions": {{ 37955503, 46970693 }}, "user": { "id": 21501082, "name": "Kaitlin Janusz", "screen_name": "kaitlinjanusz", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-02-21"), "description": "international tax consultant and attorney. internet marketing enthusiast. coffee and wine drinker. book lover. runner. knitter.", "followers_count": 245, "friends_count": 351, "statues_count": 878 }, "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:21.000Z"), "id": 668945714530140160, "text": "@SayrDeyb @annabellesanity bakit ganyan kayo ha hahahahahahahahaha", "in_reply_to_status": 668780051614203905, "in_reply_to_user": 1958783334, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 1958783334, 728153377 }}, "user": { "id": 1959751544, "name": "Has ♛", "screen_name": "HasleyVlldlgd", "lang": "en", "location": "Honolulu, HI", "create_at": date("2013-10-13"), "description": "Aviator at heart •Take me anywhere ✈️ 4P4T ❤️", "followers_count": 411, "friends_count": 143, "statues_count": 52315 }, "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("2015-11-23T16:14:21.000Z"), "id": 668945714567745537, "text": "This is something we'd do tbh @gabhayz https://t.co/inkPJEUoJZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2882814821 }}, "user": { "id": 2686821678, "name": "bonca", "screen_name": "BiancaxxMarie", "lang": "en", "location": "null", "create_at": date("2014-07-28"), "description": "nyc", "followers_count": 175, "friends_count": 169, "statues_count": 796 }, "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:21.000Z"), "id": 668945714597228544, "text": "@light_maddy ur dad is my fav ������❤️", "in_reply_to_status": -1, "in_reply_to_user": 1392569756, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1392569756 }}, "user": { "id": 3452015115, "name": "Autumn", "screen_name": "autumnwhitlatch", "lang": "en", "location": "null", "create_at": date("2015-08-26"), "description": "M.M", "followers_count": 214, "friends_count": 263, "statues_count": 997 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Barberton, OH", "id": "6d88dcc1dd35318b", "name": "Barberton", "place_type": "city", "bounding_box": rectangle("-81.650635,40.969234 -81.563534,41.042029") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3903828, "cityName": "Barberton" } }
+{ "create_at": datetime("2015-11-23T16:14:21.000Z"), "id": 668945714706296832, "text": "Holy wow that's goals https://t.co/c24UTtbelm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 324969770, "name": "Jansen Blevins", "screen_name": "JBlev_24", "lang": "en", "location": "Muncie, Indiana", "create_at": date("2011-06-27"), "description": "null", "followers_count": 507, "friends_count": 627, "statues_count": 5931 }, "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("2015-11-23T16:14:21.000Z"), "id": 668945714936958976, "text": "@jmarshall2015 https://t.co/Br4p7fbeJG", "in_reply_to_status": -1, "in_reply_to_user": 409527002, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 409527002 }}, "user": { "id": 327521782, "name": "allison", "screen_name": "allison_paigeee", "lang": "en", "location": "336////704", "create_at": date("2011-07-01"), "description": "Matthew 11:28 | Wingate University", "followers_count": 1585, "friends_count": 586, "statues_count": 35884 }, "place": { "country": "United States", "country_code": "United States", "full_name": "King, NC", "id": "9f6a388f85a93288", "name": "King", "place_type": "city", "bounding_box": rectangle("-80.40585,36.238116 -80.306974,36.343131") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37169, "countyName": "Stokes", "cityID": 3735760, "cityName": "King" } }
+{ "create_at": datetime("2015-11-23T16:14:21.000Z"), "id": 668945715108933632, "text": "Lol IU...absolute horrific and I mean horrific defense all game. With that you get a loss to a mediocre Wake Forrest team.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 473671775, "name": "Joseph Gremore", "screen_name": "Sir_Gremore12", "lang": "en", "location": "null", "create_at": date("2012-01-24"), "description": "Cubs, Pacers, Colts, Hoosiers! ISU! Sigma Chi!", "followers_count": 556, "friends_count": 361, "statues_count": 12033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Terre Haute, IN", "id": "8a61588aff8b0577", "name": "Terre Haute", "place_type": "city", "bounding_box": rectangle("-87.460551,39.394564 -87.303557,39.520714") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18167, "countyName": "Vigo", "cityID": 1875428, "cityName": "Terre Haute" } }
+{ "create_at": datetime("2015-11-23T16:14:21.000Z"), "id": 668945715188637696, "text": "@aisu10 youre getting it more than me lolol", "in_reply_to_status": 668943529654280192, "in_reply_to_user": 29287398, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 29287398 }}, "user": { "id": 3012520367, "name": "stray", "screen_name": "Slysk8s", "lang": "en", "location": "null", "create_at": date("2015-02-02"), "description": "null", "followers_count": 18, "friends_count": 29, "statues_count": 433 }, "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("2015-11-23T16:14:21.000Z"), "id": 668945715196858368, "text": "my mcm don't like me and he likes everyone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1419649076, "name": "❣", "screen_name": "venaexo", "lang": "en", "location": "null", "create_at": date("2013-05-10"), "description": "htxxxx", "followers_count": 931, "friends_count": 1731, "statues_count": 8954 }, "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("2015-11-23T16:14:21.000Z"), "id": 668945715226394624, "text": "@crimsonquarry Tom Crean or Pep Hamilton?", "in_reply_to_status": -1, "in_reply_to_user": 29423939, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 29423939 }}, "user": { "id": 278274722, "name": "Riley Roudebush", "screen_name": "Riley_Roudy24", "lang": "en", "location": "Nobletucky", "create_at": date("2011-04-06"), "description": "sports", "followers_count": 450, "friends_count": 285, "statues_count": 6409 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Noblesville, IN", "id": "013e3bc05a18abdc", "name": "Noblesville", "place_type": "city", "bounding_box": rectangle("-86.090354,39.993436 -85.932944,40.10716") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18057, "countyName": "Hamilton", "cityID": 1854180, "cityName": "Noblesville" } }
+{ "create_at": datetime("2015-11-23T16:14:21.000Z"), "id": 668945715310235648, "text": "@iStrokes_ ������ that is the longest nickname ... I like it anuk chan", "in_reply_to_status": 668938636910555136, "in_reply_to_user": 285346919, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 285346919 }}, "user": { "id": 258042684, "name": "Juice Lord BigWoe", "screen_name": "EzekielDaDragon", "lang": "en", "location": "Narnia", "create_at": date("2011-02-26"), "description": "sometimes you gotta fly alone... ice cold baby I told you im ice cold ✨ #E4HAM", "followers_count": 1380, "friends_count": 868, "statues_count": 146144 }, "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("2015-11-23T16:14:21.000Z"), "id": 668945715561918464, "text": "Can't express how grateful I am for Maya. I never make new friends and she is truly 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": 161906448, "name": "Lex", "screen_name": "LexyyLexx", "lang": "en", "location": "Between Psychotic and Iconic ", "create_at": date("2010-07-01"), "description": "Look up, the sun is just a cloud away . . . ⛅️", "followers_count": 946, "friends_count": 259, "statues_count": 52947 }, "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("2015-11-23T16:14:21.000Z"), "id": 668945715603763200, "text": "@The_Melt burger & fries was so delicious!! See ya night time I visit SF..", "in_reply_to_status": -1, "in_reply_to_user": 300449981, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 300449981 }}, "user": { "id": 368676893, "name": "ᎾX ᏝAHUN CHICCHAN", "screen_name": "jimmyb304", "lang": "en", "location": "AᏝᏝ ᎾVEREastᏔheelingBoy #ᏔV", "create_at": date("2011-09-05"), "description": "Ꮭiveitup WV~( LasV )~PA* FAMILY IS ᏔHERE IT IS@ 80's baby / Aries_ ~ Ꮎwner at Eatinirondaily™ instagram/Snap @jimmyb304 CᎾAᏝ-MINER Handsome&Healthy", "followers_count": 23799, "friends_count": 20087, "statues_count": 8058 }, "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:21.000Z"), "id": 668945715633127424, "text": "Won a 6 rd decision Friday night but broke my hand in the 2 rd https://t.co/ZDqhyrq69f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 150457697, "name": "Prince Tiger Smalls", "screen_name": "prince_tsmalls", "lang": "en", "location": "San Diego, Las Vegas, L.A.", "create_at": date("2010-05-31"), "description": "Professional Boxer . Flashy Warrior . P⭐️T⭐️S⭐️", "followers_count": 1347, "friends_count": 185, "statues_count": 18401 }, "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:21.000Z"), "id": 668945715679334400, "text": "@TyrodTaylor You ready for your best game this season?", "in_reply_to_status": -1, "in_reply_to_user": 26229939, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 26229939 }}, "user": { "id": 1063561028, "name": "David S Brenchley", "screen_name": "DSBrenchley", "lang": "en", "location": "null", "create_at": date("2013-01-05"), "description": "....", "followers_count": 17, "friends_count": 91, "statues_count": 635 }, "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("2015-11-23T16:14:21.000Z"), "id": 668945715737980928, "text": "Estudien y jodanse por lo suyo que nadie lo va hacer por ustedes , Dejen de estar pensando en la jodeera que hay tiempo para todo #Please", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "Please" }}, "user": { "id": 2369636068, "name": "Kevin Vēlez †", "screen_name": "kevodariel10", "lang": "en", "location": "Mayaguez P.R ⏩ Texas ", "create_at": date("2014-02-27"), "description": "North Central Texas College ⚾️ SC :kevin_dariel10 #LionsPride #KevinDurant", "followers_count": 520, "friends_count": 602, "statues_count": 17870 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gainesville, TX", "id": "342e8c0818874630", "name": "Gainesville", "place_type": "city", "bounding_box": rectangle("-97.174239,33.599665 -97.086339,33.669068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48097, "countyName": "Cooke", "cityID": 4827984, "cityName": "Gainesville" } }
+{ "create_at": datetime("2015-11-23T16:14:21.000Z"), "id": 668945715993960448, "text": "Our Motto: Get Torched, Get Wrecked and Get Shredded! By now, you’ve… https://t.co/3AcBYeN76a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.3389664,41.9682198"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 137557364, "name": "Tiffany Wetherbee", "screen_name": "KickboxingNA", "lang": "en", "location": "North Attleboro, Massachusetts", "create_at": date("2010-04-26"), "description": "Owner of Torched Kickboxing and Fitness Center 508-699-8400", "followers_count": 468, "friends_count": 206, "statues_count": 2933 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Attleboro, MA", "id": "0000321b41466bc8", "name": "North Attleboro", "place_type": "city", "bounding_box": rectangle("-71.381728,41.914734 -71.268541,42.014947") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25005, "countyName": "Bristol", "cityID": 2502690, "cityName": "Attleboro" } }
+{ "create_at": datetime("2015-11-23T16:14:21.000Z"), "id": 668945716140752896, "text": "Who doesn't love brand new sheets?!?! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 83720736, "name": "Courtney Kronenberge", "screen_name": "cakronenberger", "lang": "en", "location": "null", "create_at": date("2009-10-19"), "description": "Ü ❤️ Living and Loving ❤️ Ü", "followers_count": 317, "friends_count": 366, "statues_count": 4017 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spanish Fort, AL", "id": "28c3c441652d963f", "name": "Spanish Fort", "place_type": "city", "bounding_box": rectangle("-87.931342,30.6569 -87.842091,30.699659") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1003, "countyName": "Baldwin", "cityID": 171976, "cityName": "Spanish Fort" } }
+{ "create_at": datetime("2015-11-23T16:14:21.000Z"), "id": 668945716274819072, "text": "@Kevmath @jkuether3 @PH_POKER you have 0 chance of PH answering, poor staff, great turnout though!", "in_reply_to_status": 668905569219158016, "in_reply_to_user": 14944256, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14944256, 33668089, 363314545 }}, "user": { "id": 394277166, "name": "Kevin ODonnell", "screen_name": "KevinODonnell15", "lang": "en", "location": "Arizona", "create_at": date("2011-10-19"), "description": "Love my kids, poker and traveling", "followers_count": 352, "friends_count": 230, "statues_count": 2495 }, "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("2015-11-23T16:14:21.000Z"), "id": 668945716480348160, "text": "Today was a success! Love my beautiful girls�� @madisontaaaylor @mikaylaflynn @laurentaylor124 @thats00gaby https://t.co/5jUikxp0Xk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3128300822, 192404231, 1708522958, 2450888162 }}, "user": { "id": 1330504376, "name": "Janea Myers", "screen_name": "janea_y", "lang": "en", "location": "null", "create_at": date("2013-04-05"), "description": "Chasing after my creators heart❤", "followers_count": 196, "friends_count": 369, "statues_count": 1094 }, "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("2015-11-23T16:14:21.000Z"), "id": 668945716509859840, "text": "Well #iubb fans. We can watch like fisher win now", "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": 139439092, "name": "Mitch Wolfe", "screen_name": "KingSamShockley", "lang": "en", "location": "Indiana", "create_at": date("2010-05-02"), "description": "#colts #iubb I like to play in the rain.", "followers_count": 42, "friends_count": 252, "statues_count": 2425 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Angola, IN", "id": "001371f4585e4b70", "name": "Angola", "place_type": "city", "bounding_box": rectangle("-85.047873,41.585608 -84.981072,41.671792") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18151, "countyName": "Steuben", "cityID": 1801666, "cityName": "Angola" } }
+{ "create_at": datetime("2015-11-23T16:14:21.000Z"), "id": 668945716572631040, "text": "We're #hiring! Click to apply: OR Residency Program for Certified Surgical Technologists - https://t.co/k2JzIj7fec #Job #BoyntonBeach, FL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.0697937,26.5047866"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Job", "BoyntonBeach" }}, "user": { "id": 2615559954, "name": "Bethesda Health Jobs", "screen_name": "BH_Jobs", "lang": "en", "location": "null", "create_at": date("2014-07-10"), "description": "Whether you’re a nurse looking for a more challenging position or a clerical prof. looking for great benefits, you’ll find a rewarding career @ Bethesda Health.", "followers_count": 325, "friends_count": 672, "statues_count": 1774 }, "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("2015-11-23T16:14:21.000Z"), "id": 668945716694355968, "text": "@sophsophsmith1 sounds perfect to me", "in_reply_to_status": 668944907097260032, "in_reply_to_user": 3239226225, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3239226225 }}, "user": { "id": 14807206, "name": "Mandie", "screen_name": "Believeinme1991", "lang": "en", "location": "Chicago now, Nashville SOON!", "create_at": date("2008-05-16"), "description": "Mandie, 24, Music Lover, Chronic pain, POTS, Mito, My lungs SUCK & MUCH more! #Nashie #Clarebowen #GreysAnatomy #Swiftie #lovatic #5sosfam", "followers_count": 617, "friends_count": 382, "statues_count": 13070 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Campton Hills, IL", "id": "004e10e27b41078d", "name": "Campton Hills", "place_type": "city", "bounding_box": rectangle("-88.458185,41.906432 -88.374331,41.987508") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17089, "countyName": "Kane", "cityID": 1710906, "cityName": "Campton Hills" } }
+{ "create_at": datetime("2015-11-23T16:14:21.000Z"), "id": 668945716723609600, "text": "@slorozco17 wed > mon", "in_reply_to_status": 668944510513053696, "in_reply_to_user": 1312053120, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1312053120 }}, "user": { "id": 14217183, "name": "Bryan A. Monzon", "screen_name": "bryanmonzon", "lang": "en", "location": "San Diego, CA", "create_at": date("2008-03-25"), "description": "Build stuff on WordPress, coach triathletes and a health coach.", "followers_count": 594, "friends_count": 401, "statues_count": 6294 }, "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:21.000Z"), "id": 668945716874764288, "text": "@BJW___ I'll sleep on that one", "in_reply_to_status": 668920300378202112, "in_reply_to_user": 3008299046, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3008299046 }}, "user": { "id": 2898570468, "name": "Mia Joe ❣", "screen_name": "UniqueFagg", "lang": "en", "location": "Ypsilanti, MI | Pete", "create_at": date("2014-11-13"), "description": "Everybody's Replaceable. ✈️ JR ♥️", "followers_count": 4889, "friends_count": 4824, "statues_count": 37494 }, "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:21.000Z"), "id": 668945716920901633, "text": "This Friday Night in Miami we have exactly what you've been waiting on!!! #FloridaNupes ����… https://t.co/vJhBKpEPmC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.10410801,26.37144701"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FloridaNupes" }}, "user": { "id": 97774874, "name": "The Krowd Favorite™", "screen_name": "1JazzyJeff", "lang": "en", "location": "Mixtape . Club . Radio", "create_at": date("2009-12-18"), "description": "Jazzy Hendrix™\n\n.:: Crüe ::.\n\n.:: RiP Yomi ::.\n\n#LongLiveDiesel\n\nDJ | ΘΗ Nupe | #TeamSBE \n\n• Instagram: 1JazzyJeff •\n• Booking: 305-527-3653", "followers_count": 4785, "friends_count": 1404, "statues_count": 298124 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945717109526529, "text": "@ayoo_james19 why", "in_reply_to_status": 668945431162843136, "in_reply_to_user": 994430377, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 994430377 }}, "user": { "id": 246069206, "name": "Chris(tmas)", "screen_name": "FzProbst", "lang": "en", "location": "Las Vegas ✈ Reno", "create_at": date("2011-02-01"), "description": "18 | University of Nevada-Reno | Pats, Bolts and Sox | Snapchat and Instagram : patsprobst. Follow me", "followers_count": 1095, "friends_count": 184, "statues_count": 110077 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945717155799041, "text": "@realDonaldTrump @iliveamongyou @BostonGlobe watch the terrorist cheering https://t.co/x1G6hzJft4", "in_reply_to_status": 668918522114875393, "in_reply_to_user": 25073877, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25073877, 3321571577, 95431448 }}, "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": 1615, "friends_count": 1709, "statues_count": 23177 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945717226897408, "text": "#Delicous������ @ Soul Vegetarian Restaurant Tallahassee https://t.co/xxLmJ8ByD7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.281601,30.4320107"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Delicous" }}, "user": { "id": 452141483, "name": "Debo Sounds", "screen_name": "debosounds", "lang": "en", "location": "Tallahassee,Fl", "create_at": date("2012-01-01"), "description": "Traveling Live Sound Engineer, Re-Enforced Sounds, Live Recordings, DJ Services, Stage Plays, Business, Homes, & Car Audio Installs & Sales!", "followers_count": 214, "friends_count": 265, "statues_count": 304 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945717235482624, "text": "@chrischapla forgot your \" \"", "in_reply_to_status": 668945564743155712, "in_reply_to_user": 411104409, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 411104409 }}, "user": { "id": 26942833, "name": "Kevin Dwire", "screen_name": "kevindwire", "lang": "en", "location": "Fort Wayne", "create_at": date("2009-03-26"), "description": "Supressing my inner music junkie/hippie in order to do dad/business-runner duties.", "followers_count": 129, "friends_count": 277, "statues_count": 3367 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945717336129537, "text": "I'm at @Whataburger in Euless, TX https://t.co/SL5TfCXPtK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.10120893,32.88097768"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 353060311 }}, "user": { "id": 788270491, "name": "amy ;-/", "screen_name": "amyshaped", "lang": "en", "location": "texas", "create_at": date("2012-08-28"), "description": "fangirl. bad influence. photographer. lonestar state. car crash heart. half-doomed. i live in pun jail. dad jokes i got 'em in spades.", "followers_count": 1945, "friends_count": 486, "statues_count": 54784 }, "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:22.000Z"), "id": 668945717470371840, "text": "Hahah s/o to @sophiaremsik for reading and editing my 8 page science paper about the brain.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1216019095 }}, "user": { "id": 3657177195, "name": "emmaaa", "screen_name": "emma_wightkin", "lang": "en", "location": "naperthrill", "create_at": date("2015-09-14"), "description": "~vibin~", "followers_count": 85, "friends_count": 85, "statues_count": 465 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945717474500608, "text": "times 35 https://t.co/0qGKuatK64", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3015421715, "name": "hanna gonzales⚜", "screen_name": "__hannerxo", "lang": "en", "location": "504", "create_at": date("2015-02-03"), "description": "null", "followers_count": 957, "friends_count": 669, "statues_count": 13084 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945717508087808, "text": "<sad trombone>", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.10928109,40.04250907"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 75007709, "name": "Heath Matthias", "screen_name": "HMATTHIAS", "lang": "en", "location": "null", "create_at": date("2009-09-17"), "description": "null", "followers_count": 454, "friends_count": 273, "statues_count": 111718 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westfield, IN", "id": "e4f5e6a715f4c8cd", "name": "Westfield", "place_type": "city", "bounding_box": rectangle("-86.183588,39.999226 -86.089631,40.072257") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18057, "countyName": "Hamilton", "cityID": 1882700, "cityName": "Westfield" } }
+{ "create_at": datetime("2015-11-23T16:14:22.000Z"), "id": 668945717541621762, "text": "@allerandi06 that is ABSURD! NO ONE is an enemy of #apple #pie.", "in_reply_to_status": 668929799696445442, "in_reply_to_user": 24239477, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "apple", "pie" }}, "user_mentions": {{ 24239477 }}, "user": { "id": 22077191, "name": "Luca de Sanctis", "screen_name": "ZooeyBuddy", "lang": "en", "location": "?", "create_at": date("2009-02-26"), "description": "Designer", "followers_count": 126, "friends_count": 169, "statues_count": 892 }, "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:22.000Z"), "id": 668945717545803776, "text": "PeterGold you are a real HERO. Godspeed that you recover 100% https://t.co/NnDmokv3tp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 77252938, "name": "M C MatasChamberlain", "screen_name": "kraffin614", "lang": "en", "location": " Maryland", "create_at": date("2009-09-25"), "description": "CoolChick,wife,sister, aunt, friend, Nats/Hoos Fan,Travel Fan,Foodie Fan,Classic Movie Fan.MSHAI,RN,CDE,PLWD,Life Experiences over Amassing Trinkets anyday!", "followers_count": 48, "friends_count": 165, "statues_count": 737 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945717608615936, "text": "����- absolutely love how you curl your hair", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2723301972, "name": "maria", "screen_name": "Maria_B1227", "lang": "en", "location": "null", "create_at": date("2014-08-10"), "description": "moody hs", "followers_count": 393, "friends_count": 314, "statues_count": 5277 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945717994500098, "text": "“@lachelracy: We need more Chris Finch's in this world” this made my day", "in_reply_to_status": 668875258896642048, "in_reply_to_user": 377772809, "favorite_count": 0, "coordinate": point("-93.58423859,41.75972186"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 377772809 }}, "user": { "id": 499240194, "name": "Chris Finch", "screen_name": "chrisGOLDfinch3", "lang": "en", "location": "Nashville, TN", "create_at": date("2012-02-21"), "description": "Sophomore at Vanderbilt University that does a poor job of updating his social media.", "followers_count": 343, "friends_count": 290, "statues_count": 9899 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ankeny, IA", "id": "265f5cd93d66b1a1", "name": "Ankeny", "place_type": "city", "bounding_box": rectangle("-93.677966,41.688221 -93.545095,41.777677") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1902305, "cityName": "Ankeny" } }
+{ "create_at": datetime("2015-11-23T16:14:22.000Z"), "id": 668945718443282432, "text": "Fierce Full Charge Bookkeepers - Accountemps: (#FremontWalnutCreekPleasanton, CA) https://t.co/I0cOaOrsWx #Accounting #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.7304815,41.1526233"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FremontWalnutCreekPleasanton", "Accounting", "Job", "Jobs", "Hiring" }}, "user": { "id": 318670019, "name": "California Acct Jobs", "screen_name": "tmj_CA_ACCT", "lang": "en", "location": "California Non-Metro", "create_at": date("2011-06-16"), "description": "Follow this account for geo-targeted Accounting job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 395, "friends_count": 298, "statues_count": 301 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Iowa, USA", "id": "3cd4c18d3615bbc9", "name": "Iowa", "place_type": "admin", "bounding_box": rectangle("-96.639706,40.375437 -90.140061,43.50102") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19039, "countyName": "Clarke" } }
+{ "create_at": datetime("2015-11-23T16:14:22.000Z"), "id": 668945718455853056, "text": "We're #hiring! Click to apply: Regional Advisor Consultant - https://t.co/1N8Pm7g3qT #Sales #Denver, CO #Job #Jobs #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", "Sales", "Denver", "Job", "Jobs", "CareerArc" }}, "user": { "id": 21681521, "name": "TMJ - DEN Sales Jobs", "screen_name": "tmj_den_sales", "lang": "en", "location": "Denver, CO", "create_at": date("2009-02-23"), "description": "Follow this account for geo-targeted Sales job tweets in Denver, CO. Need help? Tweet us at @CareerArc!", "followers_count": 469, "friends_count": 310, "statues_count": 594 }, "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:22.000Z"), "id": 668945718711861248, "text": "@ck____1 so many of the events have transpired", "in_reply_to_status": 668945106553081856, "in_reply_to_user": 734048454, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 734048454 }}, "user": { "id": 15795459, "name": "daledo", "screen_name": "zinfandale_", "lang": "en", "location": "NYC | ocean city, nj", "create_at": date("2008-08-09"), "description": "Baritenor, Pianist • Slowly turning into Squidward since 1994 • #q #1995", "followers_count": 363, "friends_count": 311, "statues_count": 8421 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945718791512064, "text": "InstaMoon @ Sycamore Junior High School https://t.co/n7TK5J45BG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.3683333,39.2286111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24210409, "name": "Eric Routenberg", "screen_name": "EricRoutenberg", "lang": "en", "location": "Mason, OH", "create_at": date("2009-03-13"), "description": "Boss of Customer Grins No1 Most Loved Cincinnati Periscoper Tips & Tricks Extrovert Awesome dimples Founder Owner #GreenDrinksCincinnati https://t.co/MnAdhGoDuq", "followers_count": 9818, "friends_count": 10798, "statues_count": 123996 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945718812524544, "text": "@Mattforbes99 ❤️❤️❤️����", "in_reply_to_status": 668945276229545986, "in_reply_to_user": 512687732, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 512687732 }}, "user": { "id": 534778656, "name": "Blake Johnson", "screen_name": "blakejohnsonb", "lang": "en", "location": "919", "create_at": date("2012-03-23"), "description": "Whether you think you can, or you think you can't. You're right.", "followers_count": 407, "friends_count": 331, "statues_count": 2228 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsboro, NC", "id": "002eefd54973e7f7", "name": "Pittsboro", "place_type": "city", "bounding_box": rectangle("-79.192038,35.699485 -79.153955,35.750723") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37037, "countyName": "Chatham", "cityID": 3752660, "cityName": "Pittsboro" } }
+{ "create_at": datetime("2015-11-23T16:14:22.000Z"), "id": 668945718871093248, "text": "Step moms are 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": 2411826235, "name": "Ann.", "screen_name": "annabrady318", "lang": "en", "location": "null", "create_at": date("2014-03-25"), "description": "snap @ ohheyitsann", "followers_count": 332, "friends_count": 421, "statues_count": 2096 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945718879633409, "text": "monty got a fat bald guy here who is a bug tamer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2606747834, "name": "quacy mcleod", "screen_name": "kingquacy", "lang": "en", "location": "brooklyn", "create_at": date("2014-07-05"), "description": "null", "followers_count": 268, "friends_count": 1681, "statues_count": 9505 }, "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:22.000Z"), "id": 668945718917353473, "text": "Yass lord���� https://t.co/YHyU8neJqi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 520667778, "name": "⚜Javona⚜", "screen_name": "ItsJustJavona", "lang": "en", "location": "Myrtle Beach, SC", "create_at": date("2012-03-10"), "description": "✨Everything happens for a reason.✨ #USCU19", "followers_count": 619, "friends_count": 523, "statues_count": 18659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valley Falls, SC", "id": "83f69050c89a6a95", "name": "Valley Falls", "place_type": "city", "bounding_box": rectangle("-82.024291,34.989317 -81.937438,35.028922") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45083, "countyName": "Spartanburg", "cityID": 4573465, "cityName": "Valley Falls" } }
+{ "create_at": datetime("2015-11-23T16:14:22.000Z"), "id": 668945718971904000, "text": "@Looooo24 you playing in that league tonight?", "in_reply_to_status": -1, "in_reply_to_user": 3703131747, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3703131747 }}, "user": { "id": 868929355, "name": "AMorris™", "screen_name": "Born_legend0", "lang": "en", "location": "null", "create_at": date("2012-10-08"), "description": "-RIP DAD❤️ #Babyfacegang #BleedOrange", "followers_count": 1489, "friends_count": 1119, "statues_count": 25169 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jasper, AL", "id": "fa4a0db0e23833d3", "name": "Jasper", "place_type": "city", "bounding_box": rectangle("-87.313548,33.806298 -87.212496,33.910309") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1127, "countyName": "Walker", "cityID": 138416, "cityName": "Jasper" } }
+{ "create_at": datetime("2015-11-23T16:14:22.000Z"), "id": 668945719256940545, "text": "when ur suddenly musically popular https://t.co/6iSnli5PzB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2964150170, "name": "mags :)", "screen_name": "maggieedarrr", "lang": "en", "location": "HTX", "create_at": date("2015-01-07"), "description": "hmu", "followers_count": 358, "friends_count": 657, "statues_count": 2565 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sienna Plantation, TX", "id": "004b5a825b5fa14b", "name": "Sienna Plantation", "place_type": "city", "bounding_box": rectangle("-95.546917,29.473466 -95.486207,29.511206") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4867766, "cityName": "Sienna Plantation" } }
+{ "create_at": datetime("2015-11-23T16:14:22.000Z"), "id": 668945719613661184, "text": "@isaiahxporter ����", "in_reply_to_status": 668945578881978368, "in_reply_to_user": 1658061462, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1658061462 }}, "user": { "id": 1029197312, "name": "addy wap", "screen_name": "AdelisaHebibi", "lang": "en", "location": "it must've been 20 minutes", "create_at": date("2012-12-22"), "description": "❤️RIP mama❤️ | IG: adelisahebibi", "followers_count": 521, "friends_count": 201, "statues_count": 31657 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945719752040450, "text": "@Sam_Cookney That's less fun. Cava might make it better, if less precise!", "in_reply_to_status": 668944077224878081, "in_reply_to_user": 21288823, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21288823 }}, "user": { "id": 2199956191, "name": "Bronwen", "screen_name": "BronwenLiz", "lang": "en", "location": "Brooklyn, NY, USA", "create_at": date("2013-11-17"), "description": "null", "followers_count": 87, "friends_count": 159, "statues_count": 1436 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945719840124929, "text": "Easy RT @BadRedBeezy: Write me a poem or a song.", "in_reply_to_status": 668945038676770817, "in_reply_to_user": 180687406, "favorite_count": 0, "coordinate": point("-75.24482832,39.90254554"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 180687406 }}, "user": { "id": 843574214, "name": "[aв]", "screen_name": "MrTable4One", "lang": "en", "location": "The Attic Future Was Counting Money In.", "create_at": date("2012-09-24"), "description": "This page contains subject matter and language that may be disturbing to some viewers, viewer discretion is advised.", "followers_count": 437, "friends_count": 437, "statues_count": 41614 }, "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:22.000Z"), "id": 668945720087588864, "text": "2 PM EST game?!?! Sweet, thought I was going to miss their 2nd game.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2272587080, "name": "christopher", "screen_name": "Bluesunday12", "lang": "en", "location": "null", "create_at": date("2014-01-01"), "description": "26 Years Old, Gemini, Entrepreneur, Proud Father! #ColtsNation #JRNation #IUBB #Reds", "followers_count": 608, "friends_count": 726, "statues_count": 4106 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945720104341505, "text": "There's this tune I found that makes me think of you some home and I play it on repeatttt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 585729407, "name": "vanilla swirl", "screen_name": "megan_sebastian", "lang": "en", "location": "null", "create_at": date("2012-05-20"), "description": "we in there\n\nlike swim wear", "followers_count": 651, "friends_count": 354, "statues_count": 42654 }, "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:22.000Z"), "id": 668945720108449792, "text": "Want to work at RemX Specialty Staffing? We're #hiring in #Pasadena, CA! Click for details: https://t.co/OWTuJsV6Ol #Purchasing #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.1386005,34.1427587"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Pasadena", "Purchasing", "Job", "Jobs" }}, "user": { "id": 149184111, "name": "TMJ-CA Purch Jobs", "screen_name": "tmj_ca_purch", "lang": "en", "location": "California", "create_at": date("2010-05-28"), "description": "Follow this account for geo-targeted Purchasing/Procurement job tweets in California Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 92, "friends_count": 71, "statues_count": 85 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945720418762752, "text": "Join the CVS Health team! See our latest #Retail #job opening here: https://t.co/5J8bfINM9c #Trotwood, OH #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.2924071,39.7977816"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "Trotwood", "Hiring" }}, "user": { "id": 563319506, "name": "CVS Health Jobs", "screen_name": "CVSHealthJobs", "lang": "en", "location": "null", "create_at": date("2012-04-25"), "description": "null", "followers_count": 423, "friends_count": 3, "statues_count": 53169 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Trotwood, OH", "id": "43f1e6b6e43df0ea", "name": "Trotwood", "place_type": "city", "bounding_box": rectangle("-84.330503,39.74629 -84.25346,39.833568") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39113, "countyName": "Montgomery", "cityID": 3977504, "cityName": "Trotwood" } }
+{ "create_at": datetime("2015-11-23T16:14:22.000Z"), "id": 668945720548835328, "text": "@megtwelvee us leaving the view Saturday night... https://t.co/ADOfGK1IaD", "in_reply_to_status": -1, "in_reply_to_user": 779661540, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 779661540 }}, "user": { "id": 548780084, "name": "mad dog", "screen_name": "_mhamilton", "lang": "en", "location": "Scottsdale // Flagstaff", "create_at": date("2012-04-08"), "description": "what you must understand about me is that i'm a deeply unhappy person", "followers_count": 225, "friends_count": 168, "statues_count": 3533 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945720687403008, "text": "I did not know I was supposed to be at 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": 2526391705, "name": "December 8th ♐️❤️", "screen_name": "lovekendra1_", "lang": "en", "location": "704 ", "create_at": date("2014-05-26"), "description": "null", "followers_count": 432, "friends_count": 298, "statues_count": 13210 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945720821620736, "text": "I hurt my tooth putting the cross in my mouth (snapstory for details)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 279273612, "name": "Cory", "screen_name": "CoryChass", "lang": "en", "location": "Fallston, Maryland", "create_at": date("2011-04-08"), "description": "19. Critically acclaimed. Mediocre posture. Leviticus 18:22. Snapchat ⇨ CoryChass", "followers_count": 366, "friends_count": 449, "statues_count": 1588 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945720825647104, "text": "Me in a bathroom @ Indiana University-Purdue University Indianapolis (IUPUI) https://t.co/Y8Q7Y5p599", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.17454172,39.77374609"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 18403584, "name": "Taylor Pennington", "screen_name": "radtastictaylor", "lang": "en", "location": "iupui", "create_at": date("2008-12-26"), "description": "speedway gas station icee connoisseur", "followers_count": 247, "friends_count": 210, "statues_count": 8253 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945720846757890, "text": "holy holy https://t.co/yCYj8RxZcL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2313940176, "name": "sidneigh marie", "screen_name": "sidneighscobie", "lang": "en", "location": "zebra country ", "create_at": date("2014-01-27"), "description": "snapchat/insta: sidneigh.scobie | '17", "followers_count": 1104, "friends_count": 534, "statues_count": 15873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rochester, IN", "id": "cf7036581a451292", "name": "Rochester", "place_type": "city", "bounding_box": rectangle("-86.24189,41.038581 -86.159562,41.080211") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18049, "countyName": "Fulton", "cityID": 1865214, "cityName": "Rochester" } }
+{ "create_at": datetime("2015-11-23T16:14:22.000Z"), "id": 668945720964022272, "text": "This #Sales #job might be a great fit for you: Event Specialist Part Time Sales - https://t.co/tkBUHqS5Im #Flagstaff, AZ #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.639249,35.2013516"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "job", "Flagstaff", "Hiring" }}, "user": { "id": 28720800, "name": "Advantage Careers", "screen_name": "asmcareers", "lang": "en", "location": "North America", "create_at": date("2009-04-03"), "description": "Advantage Sales and Marketing has a career for you! \nCome Join Our Team Today!", "followers_count": 1980, "friends_count": 1250, "statues_count": 28543 }, "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("2015-11-23T16:14:22.000Z"), "id": 668945720964210688, "text": "so happy 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": 544671897, "name": "stephie.", "screen_name": "LovedForever_", "lang": "en", "location": "null", "create_at": date("2012-04-03"), "description": "blessed.", "followers_count": 1483, "friends_count": 1014, "statues_count": 57309 }, "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.200639,42.517441") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2674900, "cityName": "Southfield" } }
+{ "create_at": datetime("2015-11-23T16:14:23.000Z"), "id": 668945721769398272, "text": "@machinegunkelly Your music is amazing. You are one of my favorite artist to listen to on a daily basis. ✊�� https://t.co/Ej99aGqZzM", "in_reply_to_status": -1, "in_reply_to_user": 17861062, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17861062 }}, "user": { "id": 2264986428, "name": "Jordan Hershberger", "screen_name": "Undead_Hunterxx", "lang": "en", "location": "null", "create_at": date("2013-12-27"), "description": "|| |||18||| //Senior\\\\ [Mississippi] Est. 1997.", "followers_count": 568, "friends_count": 1151, "statues_count": 1356 }, "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("2015-11-23T16:14:23.000Z"), "id": 668945722327322626, "text": "@albaaam12 yeah probably...", "in_reply_to_status": 668936497077362694, "in_reply_to_user": 2717328131, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2717328131 }}, "user": { "id": 240500290, "name": "Pete Franco", "screen_name": "P_Francs", "lang": "en", "location": "Worcester, MA", "create_at": date("2011-01-19"), "description": "In life there are pushers and there are pullers. -Coach B #StateChampion #Sarcasm Bridgewater State Football '18", "followers_count": 576, "friends_count": 309, "statues_count": 14232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bridgewater, MA", "id": "006b91cb41e4a9cc", "name": "Bridgewater", "place_type": "city", "bounding_box": rectangle("-71.035483,41.929851 -70.898031,42.010968") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25023, "countyName": "Plymouth", "cityID": 2508050, "cityName": "Bridgewater" } }
+{ "create_at": datetime("2015-11-23T16:14:23.000Z"), "id": 668945722528649216, "text": "Shelby knows me ������ https://t.co/UmT6nqUh78", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 730423830, "name": "adri.", "screen_name": "adriana_mariiie", "lang": "en", "location": "#RIPMichael #RIPVincent ❤️", "create_at": date("2012-08-01"), "description": "don't break bread with people you didn't starve with. #FreeGucci", "followers_count": 1473, "friends_count": 1393, "statues_count": 35154 }, "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("2015-11-23T16:14:23.000Z"), "id": 668945722751000576, "text": "State probably could've had Bryant Crawford if they wanted him", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.6781475,35.785931"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 474499869, "name": "Alec Lower", "screen_name": "AlecLower", "lang": "en", "location": "null", "create_at": date("2012-01-25"), "description": "NC State sports writer for @backingthepack and @ACAllAmericans. wannabe funnyman #NCSU18 #Basspack. http://t.co/NrqKpeIN5l http://t.co/2OK9t9rZ2z", "followers_count": 386, "friends_count": 898, "statues_count": 5085 }, "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("2015-11-23T16:14:23.000Z"), "id": 668945722755178496, "text": "he makes my heart happy ��❣ https://t.co/unNzxaFg6C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 987463483, "name": "Alyssa DiMaria", "screen_name": "lysdimaria", "lang": "en", "location": "south florida", "create_at": date("2012-12-03"), "description": "it's a good life homie, @Mr_McCulloch", "followers_count": 2079, "friends_count": 132, "statues_count": 26848 }, "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("2015-11-23T16:14:23.000Z"), "id": 668945722876784641, "text": "Dr Marc Lamont Hill & Daniel Black discussing Blacks's new book \"The Coming\" at Davage Auditorium,… https://t.co/QsgMwgiC10", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.4129028,33.7482605"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20586638, "name": "Susan Ross", "screen_name": "photogriot", "lang": "en", "location": "null", "create_at": date("2009-02-11"), "description": "PhotoGriot &cultural historian", "followers_count": 1797, "friends_count": 1742, "statues_count": 10004 }, "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("2015-11-23T16:14:23.000Z"), "id": 668945723354820608, "text": "Seriously fuck today", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2828790895, "name": "Last Page Podcast", "screen_name": "LastPage_Comics", "lang": "en", "location": "null", "create_at": date("2014-09-23"), "description": "The Last Page is a monthly podcast hosted by Jessie & Rene from CBFC, reviewing comics & discussing comic news/films provided by The Empty Rant podcast network!", "followers_count": 153, "friends_count": 337, "statues_count": 1678 }, "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("2015-11-23T16:14:23.000Z"), "id": 668945723539390464, "text": "When people take pics of me while I'm working <<����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1445528310, "name": "Alison", "screen_name": "alisonjohnsonn1", "lang": "en", "location": "Anchorage, AK", "create_at": date("2013-05-20"), "description": "whats so simple in the moonlight, by morning never is", "followers_count": 189, "friends_count": 225, "statues_count": 5632 }, "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:23.000Z"), "id": 668945723577208832, "text": "all christmas everything", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1592282118, "name": "BigEl", "screen_name": "ellietaylor99", "lang": "en", "location": "DHS '17", "create_at": date("2013-07-13"), "description": "I had a better crossover than you when I was 3 - @jjhlane | snap: elliet43", "followers_count": 524, "friends_count": 472, "statues_count": 1114 }, "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("2015-11-23T16:14:23.000Z"), "id": 668945723757563904, "text": "ass ass ass ass ass https://t.co/cZlcCvJZHe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1483340388, "name": "lex ➰", "screen_name": "_alexiskiara", "lang": "en", "location": "criz t's chipotle ", "create_at": date("2013-06-04"), "description": "different is good, good is different", "followers_count": 1366, "friends_count": 1288, "statues_count": 22238 }, "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("2015-11-23T16:14:23.000Z"), "id": 668945723828736000, "text": "I found my perfect #DDEspresso beverage & want to win free Dunkin' espresso for a year. #Sweepstakes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DDEspresso", "Sweepstakes" }}, "user": { "id": 33991305, "name": "Salda not Salsa", "screen_name": "jsalda", "lang": "en", "location": "Oklahoma City, OK", "create_at": date("2009-04-21"), "description": "Designer; former thinker, it started to hurt too much.", "followers_count": 152, "friends_count": 124, "statues_count": 849 }, "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("2015-11-23T16:14:23.000Z"), "id": 668945723891650560, "text": "@FVCKBree_ I'm up here Wya", "in_reply_to_status": 668935783768690688, "in_reply_to_user": 578555819, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 578555819 }}, "user": { "id": 301642455, "name": "Flex Daddy of #FTH", "screen_name": "dinnerwithyamom", "lang": "en", "location": "New Orleans✈Dallas✈️KANSAS", "create_at": date("2011-05-19"), "description": ".", "followers_count": 2312, "friends_count": 1391, "statues_count": 42308 }, "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("2015-11-23T16:14:23.000Z"), "id": 668945723921068032, "text": "No pill head tho..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1504610659, "name": "G.O.A.T", "screen_name": "Bigford__", "lang": "en", "location": "Beaumont, Tx", "create_at": date("2013-06-11"), "description": "Less is more it's plenty of us... #Rubbabandklan #LamarU19: sc-cford300", "followers_count": 162, "friends_count": 192, "statues_count": 1441 }, "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("2015-11-23T16:14:23.000Z"), "id": 668945724147527680, "text": "Downtown Rochester! @ Rochester, Michigan https://t.co/Hru8OOZumF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.13388889,42.68055556"), "retweet_count": 0, "lang": "de", "is_retweet": false, "user": { "id": 2355903170, "name": "Nicholas Castano", "screen_name": "NickBCastano", "lang": "en", "location": "null", "create_at": date("2014-02-21"), "description": "Validation Engineer for LG Electronics. Michigan State. Comics. Pro Wrestling. Gamer. Engaged to a beautiful woman.", "followers_count": 78, "friends_count": 383, "statues_count": 911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rochester, MI", "id": "92b61f92af3c7d38", "name": "Rochester", "place_type": "city", "bounding_box": rectangle("-83.149048,42.673538 -83.093781,42.704651") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2669020, "cityName": "Rochester" } }
+{ "create_at": datetime("2015-11-23T16:14:23.000Z"), "id": 668945724252536832, "text": "@BarbaraNiven @charcram @hallmarkmovie if you are in it, I am positive I will!! But I will give my UNbiased (yeah right !!) review!!❤️����", "in_reply_to_status": 668930296943763456, "in_reply_to_user": 18015386, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18015386, 2605918945, 241176008 }}, "user": { "id": 3300803895, "name": "JoJoRN", "screen_name": "RNBCHQT", "lang": "en", "location": "Huntington Beach, CA", "create_at": date("2015-05-27"), "description": "Nurse & single mom of the best daughter ever! Love animals, beach, kayaking, SUPing, long walks, Starbucks, my friends near and far, musicals & country music.", "followers_count": 121, "friends_count": 84, "statues_count": 4586 }, "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("2015-11-23T16:14:23.000Z"), "id": 668945724516634624, "text": "So tired with everything!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3511299734, "name": "Jason", "screen_name": "jasongoicochea", "lang": "en", "location": "Salina, KS", "create_at": date("2015-09-09"), "description": "SBVC✈️KWU ⚽️ SC: goicochea12", "followers_count": 169, "friends_count": 219, "statues_count": 1155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salina, KS", "id": "e7bae49f1ac7f22e", "name": "Salina", "place_type": "city", "bounding_box": rectangle("-97.676631,38.768801 -97.557719,38.885242") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20169, "countyName": "Saline", "cityID": 2062700, "cityName": "Salina" } }
+{ "create_at": datetime("2015-11-23T16:14:23.000Z"), "id": 668945724600487936, "text": "#Nursing in #StGeorge, UT: Travel Nurse - RN - Emergency Room Registered Nurse - 58075 at Fastaff Travel Nursing https://t.co/eIiE4yD1dh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.5780556,37.0952778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "StGeorge" }}, "user": { "id": 22009950, "name": "TMJ-UT Nursing Jobs", "screen_name": "tmj_ut_nursing", "lang": "en", "location": "Utah", "create_at": date("2009-02-26"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Utah Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 308, "friends_count": 292, "statues_count": 16 }, "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("2015-11-23T16:14:23.000Z"), "id": 668945724739035138, "text": "Packed house at #Newburgh city Council meeting https://t.co/ICZ1EMPpDi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Newburgh" }}, "user": { "id": 305816067, "name": "Austin DuBois", "screen_name": "AustinEsq", "lang": "en", "location": "Newburgh, NY", "create_at": date("2011-05-26"), "description": "Attorney in NY's Hudson Valley, specializing in estate/wealth/elder/tax/business planning. And other non-sequitur commentary.", "followers_count": 347, "friends_count": 511, "statues_count": 1553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newburgh, NY", "id": "0047e81d7db6c89a", "name": "Newburgh", "place_type": "city", "bounding_box": rectangle("-74.116963,41.487948 -74.004603,41.519289") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36071, "countyName": "Orange", "cityID": 3650034, "cityName": "Newburgh" } }
+{ "create_at": datetime("2015-11-23T16:14:23.000Z"), "id": 668945724818726912, "text": "@maddsshae crying bc tina", "in_reply_to_status": 668881680984317952, "in_reply_to_user": 3013822488, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3013822488 }}, "user": { "id": 3013822488, "name": "ㅤㅤㅤ", "screen_name": "maddsshae", "lang": "en", "location": "null", "create_at": date("2015-02-08"), "description": "walled lake western", "followers_count": 498, "friends_count": 391, "statues_count": 1509 }, "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("2015-11-23T16:14:23.000Z"), "id": 668945724869099521, "text": "Central Park walks @ New York, New York https://t.co/CDoKKTEFBx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0064,40.7142"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 446741652, "name": "Emily Mitchell", "screen_name": "Thatfashiongirl", "lang": "en", "location": "❁ 17", "create_at": date("2011-12-25"), "description": "yes im that girl who makes YouTube videos", "followers_count": 27862, "friends_count": 639, "statues_count": 11241 }, "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("2015-11-23T16:14:23.000Z"), "id": 668945724969582592, "text": "@Ali_B_ @JayyGood_23 @Loueezus pub at 12 with our woes", "in_reply_to_status": 668945388884242432, "in_reply_to_user": 14730335, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14730335, 301303657, 120974505 }}, "user": { "id": 3029619389, "name": "Ignacio", "screen_name": "thunderbird_i", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-02-10"), "description": "UCI • ΦΚΨ", "followers_count": 466, "friends_count": 591, "statues_count": 5646 }, "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("2015-11-23T16:14:23.000Z"), "id": 668945725074612224, "text": "You were like coming up for fresh air. Its like I was drowning, and you saved me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 337204983, "name": "Cera Marano♛", "screen_name": "ceramarano", "lang": "en", "location": "Camden/Columbus", "create_at": date("2011-07-17"), "description": "Be the change you wish to see in the world. @codyilgenfritz ❣/ Columbus State cheerleader", "followers_count": 1342, "friends_count": 493, "statues_count": 22856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kingsland, GA", "id": "1397b2e09bd96ad9", "name": "Kingsland", "place_type": "city", "bounding_box": rectangle("-81.719358,30.752932 -81.593076,30.831355") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13039, "countyName": "Camden", "cityID": 1343640, "cityName": "Kingsland" } }
+{ "create_at": datetime("2015-11-23T16:14:23.000Z"), "id": 668945725082894336, "text": "������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 221113773, "name": "¡Zorro![KingRam♛s]", "screen_name": "NEJTOR", "lang": "en", "location": "San Antonio, TX", "create_at": date("2010-11-29"), "description": "21. DOMINICANO. RAP [únicoIG: NestorJrRamos] http://es.favstar.fm/users/Nejtor", "followers_count": 4019, "friends_count": 2496, "statues_count": 84634 }, "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:23.000Z"), "id": 668945725103951872, "text": "rip #STONE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "STONE" }}, "user": { "id": 330547646, "name": "#MAKEITHAPPEN", "screen_name": "___woozy___", "lang": "en", "location": "NICETOWN ", "create_at": date("2011-07-06"), "description": "c/o 2016 MLB Simon Gratz MVP", "followers_count": 1857, "friends_count": 1435, "statues_count": 26557 }, "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:23.000Z"), "id": 668945725267558404, "text": "@SamanthaJoMirto keep your head up girl. I'm here for you", "in_reply_to_status": 668941934380101632, "in_reply_to_user": 631566791, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 631566791 }}, "user": { "id": 451961014, "name": "jeppyserenks", "screen_name": "jeff_serenks", "lang": "en", "location": "null", "create_at": date("2011-12-31"), "description": "I sling Meds and popcorn depending on what day of the week it is", "followers_count": 116, "friends_count": 589, "statues_count": 4178 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945725431021568, "text": "This is what I call \"Perfect Fall Weather\" Crystal clear and in the 70''s #perfectfallweather… https://t.co/AuHx9BPak8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.72713718,33.57510057"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "perfectfallweather" }}, "user": { "id": 808197403, "name": "Donna Puzifera", "screen_name": "DforDesignOC", "lang": "en", "location": "Aliso Viejo, CA", "create_at": date("2012-09-06"), "description": "D for Design - Orange County Interior Designer showing a little of my work and my life here on Twitter.", "followers_count": 37, "friends_count": 81, "statues_count": 35 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945725464690688, "text": "@T0T_G0T_IT hey boo!", "in_reply_to_status": 668945318378086400, "in_reply_to_user": 417417725, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 417417725 }}, "user": { "id": 232975463, "name": "tyler❁", "screen_name": "tyshamone", "lang": "en", "location": "Atlanta, GA", "create_at": date("2011-01-01"), "description": "She is more precious than rubies: and all the things thou canst desire are not to be compared unto her. proverbs 3:15| georgia state 17 ❁| IG: tyshamone_", "followers_count": 1575, "friends_count": 1152, "statues_count": 29836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairburn, GA", "id": "0006d55513b0c159", "name": "Fairburn", "place_type": "city", "bounding_box": rectangle("-84.626791,33.509481 -84.531197,33.59282") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1328380, "cityName": "Fairburn" } }
+{ "create_at": datetime("2015-11-23T16:14:24.000Z"), "id": 668945725552766976, "text": "Low quality pic with a high quality person #seam @ Gwynedd Mercy University https://t.co/JVMlQkPntp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.236,40.198"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "seam" }}, "user": { "id": 366776213, "name": "Madison", "screen_name": "MAD4maddiee", "lang": "en", "location": "Gwynedd Mercy University", "create_at": date("2011-09-02"), "description": "Please appreciate my jokes", "followers_count": 518, "friends_count": 1141, "statues_count": 19279 }, "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": 42091, "countyName": "Montgomery", "cityID": 4273264, "cityName": "Spring House" } }
+{ "create_at": datetime("2015-11-23T16:14:24.000Z"), "id": 668945725846364160, "text": "Cavs game about to be over by halftime", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 365232822, "name": "Chase", "screen_name": "ChaseSchertz", "lang": "en", "location": "Wall Street ", "create_at": date("2011-08-30"), "description": "null", "followers_count": 1154, "friends_count": 951, "statues_count": 34321 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallmadge, OH", "id": "cdb98b233892e8af", "name": "Tallmadge", "place_type": "city", "bounding_box": rectangle("-81.465754,41.060128 -81.373636,41.135788") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3976106, "cityName": "Tallmadge" } }
+{ "create_at": datetime("2015-11-23T16:14:24.000Z"), "id": 668945725863006208, "text": "When you can't trust a man cause your father is a coward ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3270829069, "name": "Miranda", "screen_name": "mperalta_11", "lang": "en", "location": "null", "create_at": date("2015-07-07"), "description": "The grind never stops.", "followers_count": 311, "friends_count": 199, "statues_count": 2994 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945726068641792, "text": "@dolcifantasie__ Thursday's and Sunday's", "in_reply_to_status": 668945572951408640, "in_reply_to_user": 251669044, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 251669044 }}, "user": { "id": 2455450530, "name": "Grace Li", "screen_name": "_Graciiieee", "lang": "en", "location": "null", "create_at": date("2014-04-20"), "description": "IG: _Graciiieee New Twitter", "followers_count": 70, "friends_count": 81, "statues_count": 694 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945726211158016, "text": "I don't understand how someone could be starving yet winning weight lifting championships.. Hmm.. anyone?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 392531132, "name": "GitanjaliGulveSehgal", "screen_name": "GitanjaliGulve", "lang": "en", "location": "Cupertino, California", "create_at": date("2011-10-16"), "description": "IRL: Founder+CEO Early Stage Hearables Startup @OhmPye Hindu 49y 5ft4 Self Repr Ind Amer dating Ital Amer man offline, mom of @rashmallow", "followers_count": 250, "friends_count": 232, "statues_count": 21964 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945726248882176, "text": "@adoreashley22 thinks for the follow", "in_reply_to_status": -1, "in_reply_to_user": 2454217933, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2454217933 }}, "user": { "id": 294161571, "name": "Joyce Vlaminck", "screen_name": "Jolt2555J", "lang": "en", "location": "Marshall, MN. (USA)", "create_at": date("2011-05-06"), "description": "I am a huge One Direction fan\n2015 to 2115\n also love Elvis Presley music.\n\nI love going to Rocklahoma every year in May\n\nmovies--Hunger games & Harry Porter", "followers_count": 7187, "friends_count": 7059, "statues_count": 35029 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marshall, MN", "id": "4a13437a59c88958", "name": "Marshall", "place_type": "city", "bounding_box": rectangle("-95.838221,44.420507 -95.745738,44.484299") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27083, "countyName": "Lyon", "cityID": 2740688, "cityName": "Marshall" } }
+{ "create_at": datetime("2015-11-23T16:14:24.000Z"), "id": 668945726257385472, "text": "I'm at Coalition in New York, NY w/ @ifanzito https://t.co/cortS0m0cZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.99190127,40.74615292"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1029805285 }}, "user": { "id": 284564746, "name": "Kio Gimenez", "screen_name": "kio_gimenez", "lang": "en", "location": "null", "create_at": date("2011-04-19"), "description": "Alegre, lutador, cantor (shower), dançarino, simpático. Personalidade forte, portanto CUIDADO rsrsrsr", "followers_count": 38, "friends_count": 33, "statues_count": 3573 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945726450200576, "text": "You save that epic in your brain for when you're ready. @scottmccloud JUST published his at last this year! You don't have to rush.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 22413809 }}, "user": { "id": 54343756, "name": "Lee T Blauersouth", "screen_name": "AmmoniteInk", "lang": "en", "location": "St Paul", "create_at": date("2009-07-06"), "description": "The arty 1/2 of Studio Interrobang. Lover of comics, cartoons, opera, tattoos, & nature documentaries. Sometimes I post art, or rant, or complain about writing.", "followers_count": 462, "friends_count": 420, "statues_count": 18304 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945726630596608, "text": "Best #BBQ in #Memphis #central @ Central BBQ https://t.co/b5fBhwgJ2G", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.98706583,35.1258456"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BBQ", "Memphis", "central" }}, "user": { "id": 197252078, "name": "Kevin Meister", "screen_name": "TheMeisterMan", "lang": "en", "location": "working/building playgrounds", "create_at": date("2010-09-30"), "description": "Account Executive @ Cleo, FIJI, UW Badger, connoisseur of all things fine. Active trader, experimental cook, movie buff, travel junkie", "followers_count": 482, "friends_count": 512, "statues_count": 6039 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945726735564800, "text": "@doohenitclassy basically kidnapping is what it is", "in_reply_to_status": 668945247548940289, "in_reply_to_user": 1062113324, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1062113324 }}, "user": { "id": 236005956, "name": "soup bone", "screen_name": "magiikaarp", "lang": "en", "location": "null", "create_at": date("2011-01-09"), "description": "I'm the white fetty wap #Fuck169 #RIPgoat", "followers_count": 463, "friends_count": 252, "statues_count": 27257 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945726928502784, "text": "@ItIsRalphTho lol I need some 14's to run in b", "in_reply_to_status": 668945508405280768, "in_reply_to_user": 25554145, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25554145 }}, "user": { "id": 295350247, "name": "Too $hort", "screen_name": "GodsLastSon_", "lang": "en", "location": "null", "create_at": date("2011-05-08"), "description": "I Know The Price, Know The Risk, Know The Wrongs and The Rights, Still My Blood Flows Ice It's Just My Life.", "followers_count": 935, "friends_count": 865, "statues_count": 48761 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945727033184256, "text": "St. Mary JV ended the half on a 15-4 run over the last 2:20. Larry and Diggs lead the Lady Cards in scoring with 13 apiece.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 381140899, "name": "Parsons Sun Sports", "screen_name": "PKSunSports", "lang": "en", "location": "Parsons, Kansas", "create_at": date("2011-09-27"), "description": "Coverage of Labette and SEK sports from the Parsons Sun sports editor, @SeanEFrye", "followers_count": 1065, "friends_count": 664, "statues_count": 5214 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945727129853956, "text": "Fun ending to an engagement session today! @ Fayetteville, North Carolina https://t.co/B7T1JPATiS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.9176,35.0667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15843342, "name": "Brent Schmidt", "screen_name": "brentschmidt", "lang": "en", "location": "Fayetteville, NC", "create_at": date("2008-08-13"), "description": "Husband. Father. Solider. Photographer.", "followers_count": 198, "friends_count": 276, "statues_count": 9311 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945727251456000, "text": "@JKNorton91564 @alexmeiser97 @haleybard idk I didn't know what to say so I tagged someone else", "in_reply_to_status": 668945147812556800, "in_reply_to_user": 379370009, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 379370009, 1459966710, 336909595 }}, "user": { "id": 777319290, "name": "carli", "screen_name": "CCover99", "lang": "en", "location": "rip Julian 5.23.15", "create_at": date("2012-08-23"), "description": "somethin about the chase", "followers_count": 1355, "friends_count": 954, "statues_count": 66241 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945727335198721, "text": "@therealslimsyd https://t.co/UwlVXML4Vw", "in_reply_to_status": 668945576143163392, "in_reply_to_user": 329502680, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 329502680 }}, "user": { "id": 33362764, "name": "Kylie Austin", "screen_name": "Kyliethesmiley", "lang": "en", "location": "HTX // Wichita Falls", "create_at": date("2009-04-19"), "description": "I've probably serenaded you at a red light before || Midwestern State #15⚽️", "followers_count": 1415, "friends_count": 628, "statues_count": 62863 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945727373090817, "text": "Here's some more �� @_fabtab https://t.co/fxnOlypsXr", "in_reply_to_status": 668945610834370562, "in_reply_to_user": 2161620536, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 348492274 }}, "user": { "id": 2161620536, "name": "caitlyn", "screen_name": "cait_han", "lang": "en", "location": "910 [704]", "create_at": date("2013-10-28"), "description": "|wingate university| •ΑΟΠ• {tyler❤️} #SupportOurTroops", "followers_count": 1131, "friends_count": 1034, "statues_count": 8307 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wingate, NC", "id": "771c65a8182567f1", "name": "Wingate", "place_type": "city", "bounding_box": rectangle("-80.469694,34.970002 -80.417632,35.001812") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37179, "countyName": "Union", "cityID": 3774760, "cityName": "Wingate" } }
+{ "create_at": datetime("2015-11-23T16:14:24.000Z"), "id": 668945727381512192, "text": "I'm at ComputerManLand in Marietta, GA https://t.co/I9ddoLFCTV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.48217424,33.90688641"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18137474, "name": "TheComputerMan", "screen_name": "TheComputerMan", "lang": "en", "location": "33.690458,-84.404892", "create_at": date("2008-12-15"), "description": "I am here to SAVE you!", "followers_count": 289, "friends_count": 710, "statues_count": 8033 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945727679148032, "text": "@loveshirae no ones cryin you're just being hella boring", "in_reply_to_status": 668945616374919168, "in_reply_to_user": 2225775445, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2225775445 }}, "user": { "id": 175558145, "name": "Dopeman", "screen_name": "donovan_khv", "lang": "en", "location": "Yay Area", "create_at": date("2010-08-06"), "description": "Class of '15 .. ,I DONT FUCK WITH NONE OF YOU NIGGAS an if it ain't about no money you gettin no attention #KHILL", "followers_count": 386, "friends_count": 516, "statues_count": 13724 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945727742083072, "text": "Still on plan A pussy niggas on plan B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26838447, "name": "B", "screen_name": "BPolk_", "lang": "en", "location": "Austin, TX", "create_at": date("2009-03-26"), "description": "iDJ #GVibes | Co-Host @MavenReport | Hear Me: https://soundcloud.com/officialdjb/g-vibes-5", "followers_count": 407, "friends_count": 284, "statues_count": 27561 }, "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:24.000Z"), "id": 668945727817543680, "text": "@JeremyM6090 ??", "in_reply_to_status": 668945688647098368, "in_reply_to_user": 705471996, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 705471996 }}, "user": { "id": 3810880454, "name": "ジェシーアート", "screen_name": "jessydesuu", "lang": "en", "location": "Vallejo, CA", "create_at": date("2015-10-06"), "description": "i'm excited when it comes to anime, art, and architecture. also a tea enthusiast with nice taste in music. I teach Fine Art. DM ME FOR ART.", "followers_count": 250, "friends_count": 249, "statues_count": 2022 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945728052572160, "text": "People from age 12 to age 72 came to see me today. Why? To become a… https://t.co/gBHKAYIbc7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.7142105,29.5972004"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 604087108, "name": "Zach Voss", "screen_name": "Voss_Zach", "lang": "en", "location": "LA", "create_at": date("2012-06-09"), "description": "Inspiring change through movement and performance. Every human being has the right to perform at their highest level.", "followers_count": 664, "friends_count": 748, "statues_count": 3087 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houma, LA", "id": "2a4c12f527197dab", "name": "Houma", "place_type": "city", "bounding_box": rectangle("-90.779993,29.544349 -90.649261,29.61995") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22109, "countyName": "Terrebonne", "cityID": 2236255, "cityName": "Houma" } }
+{ "create_at": datetime("2015-11-23T16:14:24.000Z"), "id": 668945728115347456, "text": "I have little baby teeth", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2984579713, "name": "BTSTU", "screen_name": "_mothercreep", "lang": "en", "location": "Hawthorne, CA", "create_at": date("2015-01-15"), "description": "crybaby", "followers_count": 98, "friends_count": 78, "statues_count": 5212 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945728140525569, "text": "@VWilson2010 I've never tweeted at you before lmao but I heard you like getting people unstuck", "in_reply_to_status": -1, "in_reply_to_user": 447706086, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 447706086 }}, "user": { "id": 1508048450, "name": "Gabriel Maldonado Jr", "screen_name": "Kokedout", "lang": "en", "location": "Shafter,CA", "create_at": date("2013-06-11"), "description": "You're gonna have to carry that weight", "followers_count": 282, "friends_count": 311, "statues_count": 9041 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945728149037056, "text": "2 more classes ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2372958362, "name": "flavio", "screen_name": "jstanfield00", "lang": "en", "location": "330", "create_at": date("2014-03-04"), "description": "#ALLinCLE // @_jamiehuzzard ❤️", "followers_count": 527, "friends_count": 353, "statues_count": 8946 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945728249733120, "text": "@KenzzieeRaee https://t.co/rfNQmVyAke", "in_reply_to_status": -1, "in_reply_to_user": 291940620, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 291940620 }}, "user": { "id": 230839893, "name": "Bailey Cope", "screen_name": "babyecope", "lang": "en", "location": "Memphis || Chattanooga ", "create_at": date("2010-12-26"), "description": "Experience: that most brutal of teachers. But you learn, my God do you learn.", "followers_count": 871, "friends_count": 426, "statues_count": 16612 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945728262156289, "text": "when thanksgiving finna be lit cause you have 3 houses to go to ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 367386266, "name": "Mamacita❣", "screen_name": "cy_breeezy", "lang": "en", "location": ".", "create_at": date("2011-09-03"), "description": "class of 2016 | chi-town | currently in bummy ass abq", "followers_count": 124, "friends_count": 98, "statues_count": 2715 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945728467668993, "text": "who wants to go get their nails done with me wednesday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1645999772, "name": "ur fav", "screen_name": "atayakellogg", "lang": "en", "location": "north high", "create_at": date("2013-08-04"), "description": "I like kitkats and girls wby?", "followers_count": 2027, "friends_count": 991, "statues_count": 17715 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945728472002561, "text": "@dylanmiller57g1 @MeekHayden I don't need to ��", "in_reply_to_status": 668944840932114433, "in_reply_to_user": 3145231027, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3145231027, 549770296 }}, "user": { "id": 942391386, "name": "Holly Christmas", "screen_name": "holly_jones22", "lang": "en", "location": "null", "create_at": date("2012-11-11"), "description": "tis the season", "followers_count": 689, "friends_count": 728, "statues_count": 5265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Concord, OH", "id": "2112fba34d5d99cb", "name": "New Concord", "place_type": "city", "bounding_box": rectangle("-81.761264,39.980842 -81.699015,40.006252") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39119, "countyName": "Muskingum", "cityID": 3954446, "cityName": "New Concord" } }
+{ "create_at": datetime("2015-11-23T16:14:24.000Z"), "id": 668945728723484672, "text": "First Day! Success ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 352811127, "name": "Juan Martinez Jr", "screen_name": "JuanMartinez956", "lang": "en", "location": "Rio Grande City, TX", "create_at": date("2011-08-10"), "description": "Just Go With It... ✌️", "followers_count": 62, "friends_count": 73, "statues_count": 2947 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945728731938816, "text": "Finally Up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 232392139, "name": "SlimFine ✌", "screen_name": "XxDraay", "lang": "en", "location": "null", "create_at": date("2010-12-30"), "description": "null", "followers_count": 747, "friends_count": 1470, "statues_count": 15062 }, "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("2015-11-23T16:14:24.000Z"), "id": 668945729206034433, "text": "Welcome my great niece to the world #thankful #EUATweetsForTurkeys, yes my nephew is a goof �� https://t.co/eHAI75Nlgo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thankful", "EUATweetsForTurkeys" }}, "user": { "id": 564032399, "name": "Bob Vajgrt", "screen_name": "RobertV_EUA", "lang": "en", "location": "Milwaukee, Wisconsin, USA", "create_at": date("2012-04-26"), "description": "Architect, Leader, Pilot, Photographer, Artist, Runner, love Family & Friends - Thought Leader for advancing learning environments and leaders for tomorrow @EUA", "followers_count": 356, "friends_count": 434, "statues_count": 2972 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenfield, WI", "id": "7f42154616cee8fa", "name": "Greenfield", "place_type": "city", "bounding_box": rectangle("-88.069295,42.930385 -87.948504,42.988186") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5531175, "cityName": "Greenfield" } }
+{ "create_at": datetime("2015-11-23T16:14:24.000Z"), "id": 668945729302327296, "text": "@blakeeldredge thankyou brother .. And I'll try my best", "in_reply_to_status": 668945476792639489, "in_reply_to_user": 539233302, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 539233302 }}, "user": { "id": 1093755186, "name": "Dylan", "screen_name": "DylanDitz", "lang": "en", "location": "C/O '17", "create_at": date("2013-01-15"), "description": "SC: dylanditz22 | Yeen tryna scrap doe", "followers_count": 612, "friends_count": 294, "statues_count": 2610 }, "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("2015-11-23T16:14:25.000Z"), "id": 668945729730252802, "text": "I love when people ask me about @The_Chatterjee at tournaments like we're friends 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": {{ 1648844659 }}, "user": { "id": 2817029016, "name": "Jake Wallack", "screen_name": "jakeewwall", "lang": "en", "location": "null", "create_at": date("2014-09-18"), "description": "Maddie's husband", "followers_count": 239, "friends_count": 153, "statues_count": 458 }, "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("2015-11-23T16:14:25.000Z"), "id": 668945729881161728, "text": "breaking out the classics https://t.co/x9bKD5UckW", "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": "hello from the twitter bio", "followers_count": 837, "friends_count": 691, "statues_count": 17734 }, "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("2015-11-23T16:14:25.000Z"), "id": 668945729969238017, "text": "3 https://t.co/7xCfroRX21", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2215263800, "name": "dësus❦", "screen_name": "agbcabeyo", "lang": "en", "location": "null", "create_at": date("2013-11-25"), "description": "preachers preach to different crowds.", "followers_count": 11516, "friends_count": 541, "statues_count": 130089 }, "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("2015-11-23T16:14:25.000Z"), "id": 668945730174910469, "text": "INGLORIOUS BASTARDS IS ON NETFLIX. JOB WELL DONE BOYS. #ScalpinNazis @netflix", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ScalpinNazis" }}, "user_mentions": {{ 16573941 }}, "user": { "id": 378377703, "name": "Stephanie Phillips", "screen_name": "steph_phillips6", "lang": "en", "location": "Strongsville||Kent", "create_at": date("2011-09-22"), "description": "Don't lose your happiness on the pursuit for more..... SHS '14 & HDFS major KSU'18; Psych minor #MetroLifeguard *IG; stephphillips_ --in love with a US Marine--", "followers_count": 598, "friends_count": 454, "statues_count": 21461 }, "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("2015-11-23T16:14:25.000Z"), "id": 668945730694959105, "text": "How the fuck did this metamorphosis happen?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 610142588, "name": "Ang", "screen_name": "Ang_Slapnicker4", "lang": "en", "location": "null", "create_at": date("2012-06-16"), "description": "teen drinking is very bad", "followers_count": 456, "friends_count": 359, "statues_count": 28301 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conneaut, OH", "id": "4a85a33b72070cc0", "name": "Conneaut", "place_type": "city", "bounding_box": rectangle("-80.627791,41.910055 -80.529345,41.972781") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39007, "countyName": "Ashtabula", "cityID": 3918350, "cityName": "Conneaut" } }
+{ "create_at": datetime("2015-11-23T16:14:25.000Z"), "id": 668945730703265792, "text": "@buttermuffin710 that's not a choice", "in_reply_to_status": 668944927208796160, "in_reply_to_user": 2197234104, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2197234104 }}, "user": { "id": 2544947671, "name": "Jack", "screen_name": "jackiee_boy", "lang": "en", "location": "On my way to USD", "create_at": date("2014-06-03"), "description": "I love this place", "followers_count": 251, "friends_count": 232, "statues_count": 3996 }, "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:25.000Z"), "id": 668945730707587072, "text": "Sheesh y'all are ALL growing up �� https://t.co/D74TzSXwQg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 73550542, "name": "Ƭaiwo•Ade", "screen_name": "MemoirsofTAI", "lang": "en", "location": "The Hill", "create_at": date("2009-09-11"), "description": "chronicles of a modern-day griot | J E S U S", "followers_count": 1063, "friends_count": 241, "statues_count": 56527 }, "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:25.000Z"), "id": 668945730896310272, "text": "I'm at Mangham La https://t.co/EkxmJVbGbY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-91.77613161,32.31446431"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 92890677, "name": "Lee Jones", "screen_name": "LeeJ07", "lang": "en", "location": "Monroe,LA", "create_at": date("2009-11-26"), "description": "Handsome southern charmer who loves to travel(At least, that's what I tell everyone off the record. I may have a few secrets!!).", "followers_count": 449, "friends_count": 2004, "statues_count": 20347 }, "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": 22083, "countyName": "Richland", "cityID": 2248260, "cityName": "Mangham" } }
+{ "create_at": datetime("2015-11-23T16:14:25.000Z"), "id": 668945731030491136, "text": "@Cello5150 ok imma give her a heads up. She gonna wanna no wht stores u like. Dm me. And i let her no", "in_reply_to_status": 668945209154256896, "in_reply_to_user": 214331711, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 214331711 }}, "user": { "id": 2392430020, "name": "Lovely Ebony", "screen_name": "LovelyEbony28", "lang": "en", "location": "Maryland, USA", "create_at": date("2014-03-07"), "description": "If u aint Talking dollars. You aint making since", "followers_count": 351, "friends_count": 313, "statues_count": 15604 }, "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:25.000Z"), "id": 668945731084881920, "text": "Can't wait to watch you ball out tomorrow������ https://t.co/fT6q0vJgQ6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1653860324, "name": "caroline casey", "screen_name": "CaroolineeCasey", "lang": "en", "location": "null", "create_at": date("2013-08-07"), "description": "H-TOWN", "followers_count": 504, "friends_count": 582, "statues_count": 1205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hernando, MS", "id": "0006bec76806fe3b", "name": "Hernando", "place_type": "city", "bounding_box": rectangle("-90.0249,34.806843 -89.962927,34.890142") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28033, "countyName": "DeSoto", "cityID": 2831780, "cityName": "Hernando" } }
+{ "create_at": datetime("2015-11-23T16:14:25.000Z"), "id": 668945731152121856, "text": "@insidethehall how is anyone surprised. Talent or not the coach doesn't prepare.", "in_reply_to_status": 668945553288527872, "in_reply_to_user": 21956202, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21956202 }}, "user": { "id": 36773051, "name": "Joshua Caddick", "screen_name": "jkc4033", "lang": "en", "location": "null", "create_at": date("2009-04-30"), "description": "null", "followers_count": 94, "friends_count": 458, "statues_count": 2319 }, "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("2015-11-23T16:14:25.000Z"), "id": 668945731324133376, "text": "Kickin it old skool with #SnowCap this amazing #SativaStrain with light up your Monday. Need energy?… https://t.co/JG7xO9xcCJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.3040695,34.1016159"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SnowCap", "SativaStrain" }}, "user": { "id": 53129965, "name": "GreenhouseHC", "screen_name": "GreenhouseHC", "lang": "en", "location": "Hollywood CA", "create_at": date("2009-07-02"), "description": "All patients must be 18+ y/o, a CA resident with ID & have a current Dr recommendation.\nMON-SUN 10A-8P\ninfo@GreenHouseHerbalCenter.org", "followers_count": 4146, "friends_count": 2987, "statues_count": 2730 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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:25.000Z"), "id": 668945731416399872, "text": "When you work hella OT and almost double your check right before Black Friday ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 197086731, "name": "Jene' ♏", "screen_name": "Jenes_WORLD", "lang": "en", "location": "Newark, DE", "create_at": date("2010-09-30"), "description": "I'm just the girl from the bar who told you her name was Michelle", "followers_count": 588, "friends_count": 444, "statues_count": 35817 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bear, DE", "id": "92291a71f0bfba69", "name": "Bear", "place_type": "city", "bounding_box": rectangle("-75.716794,39.599414 -75.645751,39.638355") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle", "cityID": 1004130, "cityName": "Bear" } }
+{ "create_at": datetime("2015-11-23T16:14:25.000Z"), "id": 668945731433201664, "text": "Jess told to post �������� https://t.co/cC8wGRHFjo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2200382646, "name": "Al", "screen_name": "almariewhitey", "lang": "en", "location": "null", "create_at": date("2013-11-17"), "description": "namaste♒️ RIP stipey ❤️", "followers_count": 510, "friends_count": 144, "statues_count": 6682 }, "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:25.000Z"), "id": 668945731630309377, "text": "Mce Papi Champu ���� https://t.co/rQoOKdo4DS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 349196052, "name": "Sujayden", "screen_name": "TheGoldenJa", "lang": "en", "location": "#goldentaughtyou", "create_at": date("2011-08-05"), "description": "The grass is greener on this side. 5'10", "followers_count": 1472, "friends_count": 934, "statues_count": 36701 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elsmere, DE", "id": "955ba211f6e98e8e", "name": "Elsmere", "place_type": "city", "bounding_box": rectangle("-75.607323,39.729659 -75.580887,39.746405") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle", "cityID": 1024540, "cityName": "Elsmere" } }
+{ "create_at": datetime("2015-11-23T16:14:25.000Z"), "id": 668945732045549568, "text": "I'm one math test away from going home", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 545388992, "name": "Cameron Cummings", "screen_name": "Cambalambam", "lang": "en", "location": "765 ✈423", "create_at": date("2012-04-04"), "description": "she made broken look beautiful\nand strong look invincible.\nshe walked with the universe\non her shoulders and made it\nlook like a pair of wings. \netsu '19.", "followers_count": 993, "friends_count": 914, "statues_count": 26071 }, "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.259078 -82.300523,36.430258") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47179, "countyName": "Washington", "cityID": 4738320, "cityName": "Johnson City" } }
+{ "create_at": datetime("2015-11-23T16:14:25.000Z"), "id": 668945732230119424, "text": "HENRY WENT AS REID AWH AWH AWH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 633137129, "name": "kida", "screen_name": "gwynmxnroe", "lang": "en", "location": "@ hhs or with alley", "create_at": date("2012-07-11"), "description": "warrior queen mermaid w magic hair that glows when i sing also a jedi with knowledge of witchcraft //", "followers_count": 926, "friends_count": 631, "statues_count": 25823 }, "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:25.000Z"), "id": 668945732418691072, "text": "You Ain't The Only One Tryna Be The Only One !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.3150839,33.7196092"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 286454166, "name": "Esha❤️", "screen_name": "ILuvMy_KTmont", "lang": "en", "location": "null", "create_at": date("2011-04-22"), "description": "Long Live Mondo Free Beon", "followers_count": 1262, "friends_count": 915, "statues_count": 16247 }, "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("2015-11-23T16:14:25.000Z"), "id": 668945732511113216, "text": "All smiles on a #manicMonday #nofilter I love the sun shining behind me... Means He's got my back.… https://t.co/SpWdVDGdQU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.714,34.4258"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "manicMonday", "nofilter" }}, "user": { "id": 386614853, "name": "Evanne Robinson", "screen_name": "EvanneRobinson", "lang": "en", "location": "null", "create_at": date("2011-10-07"), "description": "null", "followers_count": 112, "friends_count": 175, "statues_count": 559 }, "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("2015-11-23T16:14:25.000Z"), "id": 668945732611780608, "text": "I miss Sterling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 420031756, "name": "coralz❁", "screen_name": "Corali_Diaz", "lang": "en", "location": "☻rlando", "create_at": date("2011-11-23"), "description": "can your girl get a quesadilla", "followers_count": 223, "friends_count": 122, "statues_count": 2248 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pine Castle, FL", "id": "c69bcfa65fe69f62", "name": "Pine Castle", "place_type": "city", "bounding_box": rectangle("-81.387393,28.450839 -81.359349,28.483734") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1256500, "cityName": "Pine Castle" } }
+{ "create_at": datetime("2015-11-23T16:14:25.000Z"), "id": 668945732657913857, "text": "@lakings d-lines Mcnabb-doughty muzzin-martinez erhoff-mcbain Quick in net #gokingsgo #lakings #LosAngeles #socal #angelenos #lakvsfla #la", "in_reply_to_status": -1, "in_reply_to_user": 19013887, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "gokingsgo", "lakings", "LosAngeles", "socal", "angelenos", "lakvsfla", "la" }}, "user_mentions": {{ 19013887 }}, "user": { "id": 775025400, "name": "South Florida Sports", "screen_name": "southfloridasp", "lang": "en", "location": "South Florida", "create_at": date("2012-08-22"), "description": "South Florida Sports Coverage - covering Miami, Fort Lauderdale and West Palm Beach sports since 1991", "followers_count": 601, "friends_count": 1915, "statues_count": 5487 }, "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("2015-11-23T16:14:25.000Z"), "id": 668945733064777728, "text": "Tasting #NEW menu @Panorama120wine #Wine&Food yum", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "NEW", "Wine" }}, "user_mentions": {{ 1072082366 }}, "user": { "id": 24914924, "name": "Theresa Marley", "screen_name": "terrymarley", "lang": "en", "location": "Philadelphia, United States", "create_at": date("2009-03-17"), "description": "Philly District Attorney's Office. Organizer. Voter. Momma. Tweets are my OWN!", "followers_count": 854, "friends_count": 558, "statues_count": 9377 }, "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:25.000Z"), "id": 668945733173813256, "text": "@raedonnn @jollybenjamins ����❤️", "in_reply_to_status": -1, "in_reply_to_user": 287307294, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 287307294, 490907536 }}, "user": { "id": 2238446881, "name": "Kyle Ortiz", "screen_name": "jollykyles", "lang": "en", "location": "Gainesville", "create_at": date("2013-12-09"), "description": "PHS '16 | Track and Field [Pole Vaulter]", "followers_count": 276, "friends_count": 174, "statues_count": 1189 }, "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("2015-11-23T16:14:25.000Z"), "id": 668945733177884673, "text": "Donal Trump. The GOP will wait til it's toooo late for you to run as IND then they will take you down. Run now as IND.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 634387197, "name": "ReadMe4Facts", "screen_name": "ReadMe4Facts", "lang": "en", "location": "null", "create_at": date("2012-07-12"), "description": "If you are looking for facts you have found it. I will also tell you what's on myPLATE (Political Lies, Assumptions, Trickeries, and Exposures).", "followers_count": 244, "friends_count": 1099, "statues_count": 423 }, "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:25.000Z"), "id": 668945733228212227, "text": "Asian girls everywhere. #UCLA @ University of California, Los Angeles https://t.co/BHSQqDVwTl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.44409722,34.07222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "UCLA" }}, "user": { "id": 318783513, "name": "Sarahi ૐAlexandra", "screen_name": "Sarahi_Hndez", "lang": "en", "location": "Across The Universe ", "create_at": date("2011-06-16"), "description": "uh-huh, honey", "followers_count": 249, "friends_count": 196, "statues_count": 8513 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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:25.000Z"), "id": 668945733358387200, "text": "I LOOOOVE HOW WOODFIELD LOOKS NOW INSIDE!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 41855441, "name": "Alexis Bishop", "screen_name": "alexiscoral", "lang": "en", "location": "Hanover Park, IL", "create_at": date("2009-05-22"), "description": "22 years old. Bulls. Reality tv. Kardashians. Disney Princesses.", "followers_count": 195, "friends_count": 147, "statues_count": 8312 }, "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("2015-11-23T16:14:25.000Z"), "id": 668945733370949636, "text": "Good win for #Wake vs. a ranked team in Maui. Hopefully the Deacs can make some noise in the ACC.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Wake" }}, "user": { "id": 50536891, "name": "Travis Rawe", "screen_name": "TRAWE4", "lang": "en", "location": "Winston-Salem, NC", "create_at": date("2009-06-24"), "description": "My work, is your play! Wake Forest University #CampusRecreation #IntramuralSports", "followers_count": 345, "friends_count": 384, "statues_count": 3866 }, "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("2015-11-23T16:14:25.000Z"), "id": 668945733379366912, "text": "@Screamin_FLOCKA @k_birchall did you give her a nigga pass", "in_reply_to_status": 668945105974378496, "in_reply_to_user": 3020867428, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3020867428, 83474813 }}, "user": { "id": 113136587, "name": "I Troll w the Trojan", "screen_name": "Aka2Rubbas", "lang": "en", "location": "otw to heaven to raise hell", "create_at": date("2010-02-10"), "description": "hi my name is David, its nice to eat ya.", "followers_count": 1175, "friends_count": 102, "statues_count": 75319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fuquay-Varina, NC", "id": "000b2b6f4a781bba", "name": "Fuquay-Varina", "place_type": "city", "bounding_box": rectangle("-78.844741,35.560069 -78.697863,35.670874") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3725300, "cityName": "Fuquay-Varina" } }
+{ "create_at": datetime("2015-11-23T16:14:25.000Z"), "id": 668945733714886656, "text": "@demantejarnel Yo my mom is black and I'm still scared for this", "in_reply_to_status": 668929749943762944, "in_reply_to_user": 2696813968, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2696813968 }}, "user": { "id": 3302710483, "name": "Saint Deezy", "screen_name": "Trystdeezy", "lang": "en", "location": "Chilltown | RVA | VCU'19", "create_at": date("2015-07-31"), "description": "Bad Habits.", "followers_count": 174, "friends_count": 172, "statues_count": 995 }, "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("2015-11-23T16:14:25.000Z"), "id": 668945733727338496, "text": "Inside Hoover Dam #AngelBaby #AlwaysAmazing #VacationMode @ Hoover Dam https://t.co/neQ9XNhPE4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-114.73777778,36.01555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AngelBaby", "AlwaysAmazing", "VacationMode" }}, "user": { "id": 1121281340, "name": "Angel Baby", "screen_name": "AngelBbyDJ", "lang": "en", "location": "null", "create_at": date("2013-01-25"), "description": "Listen to me on 100.3fm Latino Mix Mon-Fri 3p-7p you can also check out my Chisme de la Semana segment every Sunday on the Univision AZ 10pm newscast.", "followers_count": 773, "friends_count": 423, "statues_count": 4600 }, "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": 4, "stateName": "Arizona", "countyID": 4015, "countyName": "Mohave" } }
+{ "create_at": datetime("2015-11-23T16:14:26.000Z"), "id": 668945733895131136, "text": "@Has_Bean_Taken a Christmas surprise ������", "in_reply_to_status": 668939689785733120, "in_reply_to_user": 1511975450, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1511975450 }}, "user": { "id": 115776632, "name": "MLË", "screen_name": "doctorpaprika", "lang": "en", "location": "Ganymede", "create_at": date("2010-02-19"), "description": "†®¡é$ ßü† mî§ūπdèr$†åⁿdz ♈✴*⭐✳★*❇✨", "followers_count": 119, "friends_count": 219, "statues_count": 8421 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pacheco, CA", "id": "f8a097c20b34c748", "name": "Pacheco", "place_type": "city", "bounding_box": rectangle("-122.079747,37.976001 -122.057535,37.999454") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 654764, "cityName": "Pacheco" } }
+{ "create_at": datetime("2015-11-23T16:14:26.000Z"), "id": 668945734134341633, "text": "@Kodetrix just got it", "in_reply_to_status": 668945471751200768, "in_reply_to_user": 3043360700, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3043360700 }}, "user": { "id": 183709692, "name": "Albus Severus Potter", "screen_name": "Iam_UT", "lang": "en", "location": "El Dorado", "create_at": date("2010-08-27"), "description": "YOU MAY BEAT ME DOWN, BUT LIKE A DICK, I RISE http://rantsofsilence.wordpress.com", "followers_count": 3860, "friends_count": 1595, "statues_count": 174720 }, "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("2015-11-23T16:14:26.000Z"), "id": 668945734201270272, "text": "If it dont shatter it dont matter", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2307306000, "name": "Tony", "screen_name": "Toniii77777", "lang": "en", "location": "Wichita, KS", "create_at": date("2014-01-23"), "description": "IG: lmAntonio316 NHS, prepared fo the worst. RIPmanitaz ,from the 316", "followers_count": 641, "friends_count": 642, "statues_count": 3266 }, "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("2015-11-23T16:14:26.000Z"), "id": 668945734310334465, "text": "Not going to Black Friday . Be too many people for me �� rather pay full price then to deal w 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": 626173608, "name": "12/1✈️✨", "screen_name": "TiaaGuwop_", "lang": "en", "location": "Kentrell❤️", "create_at": date("2012-07-03"), "description": "Sittin back coutin rolls , laughing at broke hoes✈️", "followers_count": 5334, "friends_count": 1761, "statues_count": 50835 }, "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:26.000Z"), "id": 668945734385795072, "text": "This Mike Tirico interview w/ Chandler Jones is 100x better than a Gruden GRINDER sesh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 271689432, "name": "Michael Katz", "screen_name": "MichaelLKatz", "lang": "en", "location": "Twin Falls, Idaho", "create_at": date("2011-03-24"), "description": "Boise State football, CSI & prep sports reporter for the Times-News. Formerly at OC Register. Proud USC & Redlands alum. IG: michaellkatz mkatz@magicvalley.com", "followers_count": 1015, "friends_count": 1284, "statues_count": 12126 }, "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("2015-11-23T16:14:26.000Z"), "id": 668945734448914432, "text": "My baby's really having a baby�� @Nysdaddy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 503424613 }}, "user": { "id": 235027463, "name": "sparks.", "screen_name": "xo_sparks", "lang": "en", "location": "null", "create_at": date("2011-01-06"), "description": "null", "followers_count": 1820, "friends_count": 979, "statues_count": 21191 }, "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("2015-11-23T16:14:26.000Z"), "id": 668945734499180544, "text": "@1Skip_ ya mama", "in_reply_to_status": 668945408022831104, "in_reply_to_user": 327600773, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 327600773 }}, "user": { "id": 54621438, "name": "k a y s i e", "screen_name": "cantyouSIE_", "lang": "en", "location": "georgia southern u. ", "create_at": date("2009-07-07"), "description": "✨it aint that deep fam✨", "followers_count": 1726, "friends_count": 845, "statues_count": 109994 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McDonough, GA", "id": "1a9ded63d1438aca", "name": "McDonough", "place_type": "city", "bounding_box": rectangle("-84.202757,33.401349 -84.099542,33.4835") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13151, "countyName": "Henry", "cityID": 1348624, "cityName": "McDonough" } }
+{ "create_at": datetime("2015-11-23T16:14:26.000Z"), "id": 668945734796840961, "text": "@JedediahBila So cute...", "in_reply_to_status": -1, "in_reply_to_user": 60680268, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 60680268 }}, "user": { "id": 2458671007, "name": "Supreme Ruler & Love", "screen_name": "KingofKingONE", "lang": "en", "location": "null", "create_at": date("2014-04-22"), "description": "And Jesus answered him, The first of all the commandments is, Hear, O Israel; The Lord our God is one Lord:", "followers_count": 2234, "friends_count": 2437, "statues_count": 147117 }, "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("2015-11-23T16:14:26.000Z"), "id": 668945734910107648, "text": "SWEET @JaredLeto �� @ShannonLeto�� + @tomofromearth �� DREAMS My Angels❤������#MARSHUGS #MARSKISSES #MARSLOVE + #MARSWISHES,Always,Forever+Beyond", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MARSHUGS", "MARSKISSES", "MARSLOVE", "MARSWISHES" }}, "user_mentions": {{ 27711339, 24341373, 17662117 }}, "user": { "id": 373916683, "name": "#DoOrDieDreamOutLoud", "screen_name": "WakeToMyDream", "lang": "en", "location": "Trenton, NJ", "create_at": date("2011-09-15"), "description": "ECHELON since 2001,Singer,Writer,Artist, DREAMER+ECHELON! Lover of ALL people! No Matter WHAT, I'll get My Dream!Find Me @MyLifeMyECHELON + @RTDONATEHELP_TY too", "followers_count": 1641, "friends_count": 2061, "statues_count": 25019 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florence, NJ", "id": "014b31a469fbfa23", "name": "Florence", "place_type": "city", "bounding_box": rectangle("-74.828572,40.072057 -74.755997,40.126223") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34005, "countyName": "Burlington", "cityID": 3423820, "cityName": "Florence" } }
+{ "create_at": datetime("2015-11-23T16:14:26.000Z"), "id": 668945735195471874, "text": "Anyone notice how weird this chick's voice is on Jeopardy tonight ?!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 406745433, "name": "Sharon Needles", "screen_name": "SHARON_NEEDLES", "lang": "en", "location": "Pittsburgh, Transylvania", "create_at": date("2011-11-06"), "description": "An example of current social anxieties... aka a punk rock sex clown.", "followers_count": 208389, "friends_count": 385, "statues_count": 9049 }, "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("2015-11-23T16:14:26.000Z"), "id": 668945735606476802, "text": "@rianwatt BP: The Flamethrower", "in_reply_to_status": 668944347161825280, "in_reply_to_user": 386843237, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 386843237 }}, "user": { "id": 517292690, "name": "nyquil mustache", "screen_name": "BeersNTrumpets", "lang": "en", "location": "Jamaica Plain, Boston", "create_at": date("2012-03-06"), "description": "midwestern dad // toiling away in the #content factory // formerly CHI and SEA // baseball words at http://wrigleyville.baseballprospectus.com", "followers_count": 305, "friends_count": 388, "statues_count": 16643 }, "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("2015-11-23T16:14:26.000Z"), "id": 668945735631548418, "text": "Kenneff think he knows me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1918536402, "name": "LaniBandz➰", "screen_name": "alanikiaraa", "lang": "en", "location": "N O A H S G I R L", "create_at": date("2013-09-29"), "description": "unbothered ✨", "followers_count": 1363, "friends_count": 723, "statues_count": 29820 }, "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:26.000Z"), "id": 668945735820423169, "text": "like please take a number and PISS OFF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 371528299, "name": "taylor", "screen_name": "Oasying", "lang": "en", "location": "carthage '19", "create_at": date("2011-09-10"), "description": "lover of Jesus, puppies, & sweet tea • @finally_erik :) • IG: Oasying", "followers_count": 558, "friends_count": 69, "statues_count": 33737 }, "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:26.000Z"), "id": 668945735962898432, "text": "I'm just hurt, & it's my fault ...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 853369765, "name": "dec 12", "screen_name": "OrginalQueenJ", "lang": "en", "location": "H o u s t o n, Tx", "create_at": date("2012-09-29"), "description": "another mixed girl with curly big hair. 19. College Girl. •Karl #RIP•", "followers_count": 1352, "friends_count": 673, "statues_count": 30145 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:26.000Z"), "id": 668945736382455808, "text": "I swear when I go away to college, I'm not going to want to come 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": 2155291446, "name": "oui❣", "screen_name": "Natalie13G", "lang": "en", "location": "Chicago, IL", "create_at": date("2013-10-25"), "description": "Nothing's impossible. And all you lame niggas showed me what not to do — J. Cole | Spanish in my blood | In love with my ambition | #K6G4 | IG: @ngarxia", "followers_count": 291, "friends_count": 232, "statues_count": 2615 }, "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:26.000Z"), "id": 668945736445202432, "text": "When your SAT scores come in tomorrow and you terrified asf.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3386827659, "name": "Sc:Pookiemosleyy", "screen_name": "_pookiemosley", "lang": "en", "location": "minding my fucking business ", "create_at": date("2015-07-21"), "description": "#14 #PookieBTW", "followers_count": 350, "friends_count": 451, "statues_count": 2600 }, "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("2015-11-23T16:14:26.000Z"), "id": 668945736546066432, "text": "is fantasy basketball a game?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3060929561, "name": "Ti'Anna Lewis", "screen_name": "zeprincess5", "lang": "en", "location": "null", "create_at": date("2015-02-24"), "description": "null", "followers_count": 188, "friends_count": 191, "statues_count": 817 }, "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("2015-11-23T16:14:26.000Z"), "id": 668945736982073345, "text": "��FREE XMAS PARTY����A SPECIAL CHRISTMAS GIFT FOR YOU!!!����FREE OSCAR G ON EON GUEST LIST TILL 1… https://t.co/OwUSGARkjk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.1929092,25.7845402"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 466006755, "name": "VixenJ", "screen_name": "Vixenj0601", "lang": "en", "location": "Miami Fl", "create_at": date("2012-01-16"), "description": "EON Radio host / IG: Vixenj0601 #nofilter follow me on periscope", "followers_count": 605, "friends_count": 1320, "statues_count": 7534 }, "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("2015-11-23T16:14:26.000Z"), "id": 668945737078587392, "text": "the idea of a one night stand literally scares 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": 38570702, "name": "noodz", "screen_name": "wicvh", "lang": "en", "location": "Los Angeles", "create_at": date("2009-05-07"), "description": "bookings: cyim@icmpartners.com || love: wicvhwicvh@gmail.com || TSNMI", "followers_count": 17992, "friends_count": 208, "statues_count": 17783 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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:26.000Z"), "id": 668945737078726658, "text": "Someone hmu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 602281090, "name": "Von 1️⃣8️⃣", "screen_name": "Jinxeus", "lang": "en", "location": "Brockton, MA", "create_at": date("2012-06-07"), "description": "I feel like the Tom Sawyer for real niggas. Welcome to the No Chill Zone. Turn on notifications so you won't miss me roasting @kdr_xo", "followers_count": 2345, "friends_count": 2137, "statues_count": 12711 }, "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("2015-11-23T16:14:26.000Z"), "id": 668945737091186688, "text": "Can't wait for Cody to 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": 377727132, "name": "B2K", "screen_name": "BrittaniStamps", "lang": "en", "location": "null", "create_at": date("2011-09-21"), "description": "crw", "followers_count": 436, "friends_count": 274, "statues_count": 5355 }, "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("2015-11-23T16:14:26.000Z"), "id": 668945737397456896, "text": "Mom asked me what I want from the store but I can't think of anything, any suggestions?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 437906955, "name": "✨LongLiveJuwop✏", "screen_name": "KalilaShante_", "lang": "en", "location": "Alexandria, VA", "create_at": date("2011-12-15"), "description": "living life ✌️.", "followers_count": 2302, "friends_count": 2149, "statues_count": 56942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hybla Valley, VA", "id": "b0eea3e446844aec", "name": "Hybla Valley", "place_type": "city", "bounding_box": rectangle("-77.096062,38.730221 -77.057234,38.765894") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5139304, "cityName": "Hybla Valley" } }
+{ "create_at": datetime("2015-11-23T16:14:26.000Z"), "id": 668945737548304384, "text": "@Candace_28 L O L... https://t.co/u4H1TRdZOB", "in_reply_to_status": -1, "in_reply_to_user": 633956773, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 633956773 }}, "user": { "id": 226006251, "name": "Mar", "screen_name": "MarLeeMassey", "lang": "en", "location": "null", "create_at": date("2010-12-12"), "description": "Philthy, MS. Est. 1993 \ninstagram: Mar_Massey \nsnapchat: Mar_Massey", "followers_count": 1839, "friends_count": 1963, "statues_count": 18336 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marion, MS", "id": "005be2a32bfc1700", "name": "Marion", "place_type": "city", "bounding_box": rectangle("-88.662754,32.405028 -88.640414,32.443004") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28075, "countyName": "Lauderdale", "cityID": 2845160, "cityName": "Marion" } }
+{ "create_at": datetime("2015-11-23T16:14:26.000Z"), "id": 668945737686863873, "text": "I'm at The @HomeDepot in Parkville, MD https://t.co/XKWWPOZumC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.54382408,39.39555378"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14791918 }}, "user": { "id": 354045422, "name": "Larry Sampson", "screen_name": "larrysam1", "lang": "en", "location": "Baltimore", "create_at": date("2011-08-12"), "description": "null", "followers_count": 77, "friends_count": 167, "statues_count": 7857 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945738240360448, "text": "I'm a baller so idc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 730955682, "name": "Andrew ⛅", "screen_name": "AjPuente_12", "lang": "en", "location": "Austin 3:16", "create_at": date("2012-08-01"), "description": ":-)", "followers_count": 671, "friends_count": 1044, "statues_count": 44183 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945738345287680, "text": "If my mom makes me move schools my senior year Ima cry ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325274357, "name": "Ashton", "screen_name": "Ashhtonnnnn", "lang": "en", "location": "Highlands, TX", "create_at": date("2011-06-27"), "description": "Every new day is another chance to change your life.", "followers_count": 581, "friends_count": 250, "statues_count": 7224 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highlands, TX", "id": "da714a0bda9efd55", "name": "Highlands", "place_type": "city", "bounding_box": rectangle("-95.08943,29.792163 -95.02753,29.846673") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4833836, "cityName": "Highlands" } }
+{ "create_at": datetime("2015-11-23T16:14:27.000Z"), "id": 668945738437554176, "text": "@Luvlymonica_ girl I heard!!! ����", "in_reply_to_status": 668945651003035649, "in_reply_to_user": 777679171, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 777679171 }}, "user": { "id": 2264784942, "name": "✨Astrid✨", "screen_name": "Astriddo_", "lang": "en", "location": "Paradise ", "create_at": date("2013-12-27"), "description": "I hate sweet tea, but I love Jesus Christ", "followers_count": 655, "friends_count": 380, "statues_count": 14275 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945738852753409, "text": "Accident cleared in #Arlington on 360 SB before Abram St, stop and go traffic in the area back to 183, delay of 16 mins #DFWTraffic", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.06232,32.73044"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Arlington", "DFWTraffic" }}, "user": { "id": 41679815, "name": "Total Traffic DFW", "screen_name": "TotalTrafficDFW", "lang": "en", "location": "Dallas/Fort Worth", "create_at": date("2009-05-21"), "description": "Total Traffic Network for Dallas/Fort Worth-Call us to report traffic delays: 214-866-8888", "followers_count": 5397, "friends_count": 142, "statues_count": 121564 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945738894733312, "text": "I HOPE WE LOSE THIS ONE TOO SO WE CAN GET RID OF OUR DRUNK COACH https://t.co/dYm7IAE0Rr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3002656093, "name": "Logan Beegle", "screen_name": "L_Beeg11", "lang": "en", "location": "null", "create_at": date("2015-01-29"), "description": "IUFB", "followers_count": 489, "friends_count": 462, "statues_count": 1133 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945738894848000, "text": "@KingSolar_ alright lmk", "in_reply_to_status": 668945704417497088, "in_reply_to_user": 2391476281, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2391476281 }}, "user": { "id": 2258686712, "name": "NeckBeard", "screen_name": "NeckBeardNick_", "lang": "en", "location": "Lakeland, FL", "create_at": date("2013-12-23"), "description": "Fat Nigga With a Neck Beard", "followers_count": 503, "friends_count": 377, "statues_count": 16574 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945739012251648, "text": "kinky to the extreme ���� if you're satisfied it's worth the pain, right? https://t.co/l8s24bkMFZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 283589766, "name": "A. Nic † 10.30", "screen_name": "babygirl_ashiee", "lang": "en", "location": "Summerville, SC ", "create_at": date("2011-04-17"), "description": "11/22/15 ❤️", "followers_count": 650, "friends_count": 651, "statues_count": 22058 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945739230277633, "text": "People that talk about bullshit blows 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": 262471942, "name": "Briiiii™", "screen_name": "prettyassbriii", "lang": "en", "location": "New W a r leans™", "create_at": date("2011-03-07"), "description": "RestGramaw RestTroy❤️ IG | ohhbriiii", "followers_count": 540, "friends_count": 830, "statues_count": 41824 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945739322695680, "text": "I wouldn't have half the convos I do a day if I didn't start them #sad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sad" }}, "user": { "id": 432674243, "name": "Fiona", "screen_name": "FionaGormally", "lang": "en", "location": "Yonkers, NY", "create_at": date("2011-12-09"), "description": "I've been selling craic since like the 5th grade Instagram: fionagormally", "followers_count": 668, "friends_count": 849, "statues_count": 8785 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945739490439168, "text": "aaaaaaaand this is the reason why i dont fuck with many girls to have as friends LOL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1545743072, "name": "mariah❦", "screen_name": "ayemariah1", "lang": "en", "location": "null", "create_at": date("2013-06-25"), "description": "know your worth", "followers_count": 630, "friends_count": 312, "statues_count": 30977 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945739838562304, "text": "16| @ your bitch/mans", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 42499705, "name": "Diammmm✨", "screen_name": "ForTheLoveOfDi", "lang": "en", "location": "null", "create_at": date("2009-05-25"), "description": "null", "followers_count": 960, "friends_count": 628, "statues_count": 26230 }, "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:27.000Z"), "id": 668945739930714113, "text": "BEAUTIFUL ���� @ladygaga https://t.co/4UGFBfeHRF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14230524 }}, "user": { "id": 3250133095, "name": "Christian Brooke", "screen_name": "badkidchris93", "lang": "en", "location": "null", "create_at": date("2015-06-19"), "description": "Take pride in yourself. Let your creativity flow. Love your body, mind and spirit. Don't be afraid to let our your inner freak.", "followers_count": 78, "friends_count": 327, "statues_count": 601 }, "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:27.000Z"), "id": 668945740077531137, "text": "@m3brittany hell yeah an that Ferrari FF ������", "in_reply_to_status": 668945343120195585, "in_reply_to_user": 857495317, "favorite_count": 0, "coordinate": point("-122.85424113,44.55454068"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 857495317 }}, "user": { "id": 56609882, "name": "JEFF⚒MONAHAN", "screen_name": "MasterJeffPDX", "lang": "en", "location": "Sharp as a home made shank", "create_at": date("2009-07-13"), "description": "23 | SKYLAR | STRAIGHT EDGE | 42-42-564", "followers_count": 3013, "friends_count": 903, "statues_count": 47431 }, "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": 41043, "countyName": "Linn" } }
+{ "create_at": datetime("2015-11-23T16:14:27.000Z"), "id": 668945740107022336, "text": "Its like we talk here and there knowing he's in love with someone else. Sigh*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 284908761, "name": "raisa sarwar", "screen_name": "ruff_rawrs", "lang": "en", "location": "null", "create_at": date("2011-04-19"), "description": "While we are postponing, life speeds by.- Seneca", "followers_count": 368, "friends_count": 348, "statues_count": 26131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballenger Creek, MD", "id": "be0e62c690c5acbc", "name": "Ballenger Creek", "place_type": "city", "bounding_box": rectangle("-77.467661,39.341113 -77.388726,39.403823") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24021, "countyName": "Frederick", "cityID": 2403800, "cityName": "Ballenger Creek" } }
+{ "create_at": datetime("2015-11-23T16:14:27.000Z"), "id": 668945740278829057, "text": "@cyph718 I'll take captain for one thousannnnnnnnnnd", "in_reply_to_status": 668945435554414592, "in_reply_to_user": 153045480, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 153045480 }}, "user": { "id": 35366599, "name": "Kimberley Reller", "screen_name": "kimreller", "lang": "en", "location": "Orlando, FL area", "create_at": date("2009-04-25"), "description": "Wife to Hans (@hansreller) and mom to Brad (@epicface42) and Jake. My family and friends are my life. #FutureTeacher. |-/", "followers_count": 293, "friends_count": 1581, "statues_count": 4883 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Mary, FL", "id": "119a13e394cab98e", "name": "Lake Mary", "place_type": "city", "bounding_box": rectangle("-81.373433,28.726142 -81.291154,28.788831") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12117, "countyName": "Seminole", "cityID": 1238425, "cityName": "Lake Mary" } }
+{ "create_at": datetime("2015-11-23T16:14:27.000Z"), "id": 668945740572598272, "text": "GRAYSONDKXKSKBKM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ro", "is_retweet": false, "user": { "id": 3078679912, "name": "Reindeer Rach☄pinned", "screen_name": "EthansSmile", "lang": "en", "location": "spirtually with may", "create_at": date("2015-03-07"), "description": "Ethan And Grayson make my jingle bells ring", "followers_count": 1631, "friends_count": 788, "statues_count": 28351 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945740782247936, "text": "Well u gonna share with us, interested in how it came out https://t.co/zFea8AisNv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3054541210, "name": "James Tucker", "screen_name": "gueetin66", "lang": "en", "location": "New York, USA", "create_at": date("2015-02-22"), "description": "Cross country truck driver /paid tourist, @Dale Jr fan, #49ers, NYY, #Harley Davidson rider. NY /Florida /Tennessee", "followers_count": 46, "friends_count": 163, "statues_count": 612 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fultonville, NY", "id": "00665f9052579959", "name": "Fultonville", "place_type": "city", "bounding_box": rectangle("-74.380771,42.940461 -74.342949,42.949869") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36057, "countyName": "Montgomery", "cityID": 3627859, "cityName": "Fultonville" } }
+{ "create_at": datetime("2015-11-23T16:14:27.000Z"), "id": 668945740790546432, "text": "Accident cleared in #FortWorth on 35W SB at Sycamore School Rd, stop and go traffic in the area back to 20, delay of 5 mins #DFWTraffic", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.32146,32.63515"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FortWorth", "DFWTraffic" }}, "user": { "id": 41679815, "name": "Total Traffic DFW", "screen_name": "TotalTrafficDFW", "lang": "en", "location": "Dallas/Fort Worth", "create_at": date("2009-05-21"), "description": "Total Traffic Network for Dallas/Fort Worth-Call us to report traffic delays: 214-866-8888", "followers_count": 5397, "friends_count": 142, "statues_count": 121565 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945740920557568, "text": "Eu pedi 30 dólares P meu pai r ele mandou 100 kkkaksks TO até feliz agora", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 3203509221, "name": "Iozzi", "screen_name": "jadiozzi", "lang": "pt", "location": "Mississippi, USA", "create_at": date("2015-04-24"), "description": "babe gurl you're so damn fine", "followers_count": 673, "friends_count": 286, "statues_count": 19989 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945740954144768, "text": "@alladinsane69 @UpperDeckSports yes they are. Here comes some photos", "in_reply_to_status": 668945586519851008, "in_reply_to_user": 20995442, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20995442, 43032657 }}, "user": { "id": 1023051320, "name": "James Yellis", "screen_name": "ssgcav", "lang": "en", "location": "Norco, Ca", "create_at": date("2012-12-19"), "description": "James, SSG (retired), Biggest Patriots fan ever! Football card collector, and love photos from around the world. i support our Military 100%", "followers_count": 2350, "friends_count": 2585, "statues_count": 45262 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945740962508800, "text": "\" You did what you thought was best for yourself when it actually also turned out to be exactly what was best for me \" - Me 01/03/15", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 129064351, "name": "Lizzzzyyyyyyyy ☔️", "screen_name": "out2findbliss", "lang": "en", "location": "The world, but mainly Oklahoma", "create_at": date("2010-04-02"), "description": "I'm a lover of life & a hater of negativity. Feel free to join my life as long as you make me better. If you don't plan on doing that, dont get attached.", "followers_count": 139, "friends_count": 267, "statues_count": 3270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Guymon, OK", "id": "85b427f4f597ddc2", "name": "Guymon", "place_type": "city", "bounding_box": rectangle("-101.507239,36.667631 -101.434428,36.714181") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40139, "countyName": "Texas", "cityID": 4031750, "cityName": "Guymon" } }
+{ "create_at": datetime("2015-11-23T16:14:27.000Z"), "id": 668945740975226881, "text": "Kaylin �� https://t.co/0bndv4cyvq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 44691362, "name": "Meagan Malloy", "screen_name": "jazminnnnnn_", "lang": "en", "location": "ATL", "create_at": date("2009-06-04"), "description": "@ilikedarkskins. #TEAMCOZY", "followers_count": 441, "friends_count": 260, "statues_count": 17167 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945741109444608, "text": "Still trying to figure out if I should go to school tmrw 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": 3346959819, "name": "lugene", "screen_name": "lugeneex", "lang": "en", "location": "null", "create_at": date("2015-06-26"), "description": "null", "followers_count": 137, "friends_count": 119, "statues_count": 1995 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945741356900354, "text": "@Honest_Opinion\nThat is a long list worthy of indictment, but alas,the GOP have Billions.\"they can buy votes&justice https://t.co/AR5ZgnKOsd", "in_reply_to_status": -1, "in_reply_to_user": 62451416, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 62451416 }}, "user": { "id": 3251967993, "name": "Sophia Mahmud.MD.", "screen_name": "sophia_mahmud", "lang": "en", "location": "Englewood, NJ", "create_at": date("2015-05-13"), "description": "A Consultant & Director Cardiology trned Actor. Ambiguous Ethnicity. British Accent. Rep SAG, IMDb '.Trained Drama. Classics. Improv Comedy &Camera inten.&com.", "followers_count": 90, "friends_count": 105, "statues_count": 1054 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945741512093696, "text": "One last charge in Auburn and then we're on our way home. Some burgers at Red Robin first! #teslatrip", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "teslatrip" }}, "user": { "id": 4145801, "name": "Tim Dorr", "screen_name": "timdorr", "lang": "en", "location": "Atlanta, GA", "create_at": date("2007-04-11"), "description": "Currently rebillionizing with @ShowcaseIDX. Cofounder of @SalesLoft, @asmallorange, @IgnitionAlley, and @armyofbees. Former EIR at @ATDC. Married to @beccadorr.", "followers_count": 2370, "friends_count": 319, "statues_count": 16766 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945741520494592, "text": "@AlexisZaryk like there's 0 hope of getting any higher than that. I'm not doing the study guide. Every time I look at it my mind goes blank��", "in_reply_to_status": 668945382668410881, "in_reply_to_user": 1216939244, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1216939244 }}, "user": { "id": 3127664039, "name": "taylor", "screen_name": "t_hamm13", "lang": "en", "location": "null", "create_at": date("2015-03-29"), "description": "ʷʰʸ ᵗʰᵉ fᵘͨᵏ ᵘ ˡʸʸʸ'ⁿ", "followers_count": 354, "friends_count": 352, "statues_count": 7352 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945741524660224, "text": "@Spotdadot @itsmalinotmolly ��", "in_reply_to_status": 668945316507357184, "in_reply_to_user": 623405015, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 623405015, 482484344 }}, "user": { "id": 3396407501, "name": "Adherent", "screen_name": "kaynicole_x", "lang": "en", "location": "null", "create_at": date("2015-07-30"), "description": "@grownboitrap❤️8202014✨", "followers_count": 246, "friends_count": 201, "statues_count": 2194 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945741902123013, "text": "So this just happened!!! #LGBTChamps #lgbt #equality #TransMilitary #OpenTransService @ The Burke… https://t.co/TofpWITJaw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.0227271,38.89448202"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LGBTChamps", "lgbt", "equality", "TransMilitary", "OpenTransService" }}, "user": { "id": 14895754, "name": "Fiona Dawson", "screen_name": "fionajdawson", "lang": "en", "location": "Washington, DC", "create_at": date("2008-05-24"), "description": "Director/Producer @TransMilitary | #gender #transgender #cisgender #equality #LGBT #bisexual http://www.facebook.com/fionajdawson", "followers_count": 1492, "friends_count": 1491, "statues_count": 4056 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945741931417600, "text": "@lurkin21 buy me this and I'll wear it everyday forever and I'll love Christmas again. https://t.co/atwzaZGmm4", "in_reply_to_status": -1, "in_reply_to_user": 517177027, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 517177027 }}, "user": { "id": 128730166, "name": "PnYiLcEk", "screen_name": "NickPyle15", "lang": "en", "location": "null", "create_at": date("2010-04-01"), "description": "null", "followers_count": 518, "friends_count": 1316, "statues_count": 9983 }, "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("2015-11-23T16:14:27.000Z"), "id": 668945742023671808, "text": "Ribéry ������ https://t.co/LGoE6hlwBF", "in_reply_to_status": 668945586943492096, "in_reply_to_user": 3276698054, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user": { "id": 3276698054, "name": "عويد.", "screen_name": "3lawAl3waid", "lang": "en", "location": "null", "create_at": date("2015-07-11"), "description": "وفخري لآل هاشم انتمي. بأمانة موسى ابن جعفر. #HalaMadrid", "followers_count": 1492, "friends_count": 411, "statues_count": 34731 }, "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("2015-11-23T16:14:28.000Z"), "id": 668945742376132608, "text": "Take this quiz to find out if you actually know me.\n\nhttps://t.co/uvrgyrnrXs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 726540672, "name": "Cait", "screen_name": "cmdavis_4", "lang": "en", "location": "Ohio", "create_at": date("2012-07-30"), "description": "Be a classy woman with a little bit of hood and a lot of God in you. insta:caitmua snapchat:caitlinmerris96", "followers_count": 493, "friends_count": 267, "statues_count": 42231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ashland, OH", "id": "fefd06f07572907a", "name": "Ashland", "place_type": "city", "bounding_box": rectangle("-82.352896,40.835537 -82.275563,40.893194") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39005, "countyName": "Ashland", "cityID": 3902568, "cityName": "Ashland" } }
+{ "create_at": datetime("2015-11-23T16:14:28.000Z"), "id": 668945742459895809, "text": "U know it's real when u willingly spend your money 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": 261734782, "name": "Devon McMillan", "screen_name": "_devonmcmillan", "lang": "en", "location": "Houston, TX", "create_at": date("2011-03-06"), "description": "I like queso and ΔΓ", "followers_count": 502, "friends_count": 356, "statues_count": 4799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:28.000Z"), "id": 668945742564724736, "text": "@ashleynicokelly @JosephKellyJr beautiful picture! ❤️", "in_reply_to_status": 668899313825611776, "in_reply_to_user": 256869647, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 256869647, 252293151 }}, "user": { "id": 361915149, "name": "Ashley Heather", "screen_name": "Ashnicole1111", "lang": "en", "location": "Brentwood, CA", "create_at": date("2011-08-25"), "description": "Baseball is my one true love!⚾#ForTheLoveOfTheGame #CardinalNation #DubNation #MiamiHurricanes #TheU #GirlsWhoLoveSports", "followers_count": 566, "friends_count": 1053, "statues_count": 10639 }, "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("2015-11-23T16:14:28.000Z"), "id": 668945742611005440, "text": "@maud_Lin a cmnt is important 2 (if available) however even sharing content one disagrees w means ur sharing an idea so why share at all?", "in_reply_to_status": 668944853179326464, "in_reply_to_user": 223434041, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 223434041 }}, "user": { "id": 36753620, "name": "triste", "screen_name": "sadgirlgang", "lang": "en", "location": "did u follow me on accident", "create_at": date("2009-04-30"), "description": "rgv", "followers_count": 482, "friends_count": 502, "statues_count": 59632 }, "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("2015-11-23T16:14:28.000Z"), "id": 668945742640357376, "text": "\"I rarely watch the basketball team. I get really nervous\" -Nobel Prize winner Aziz Sancar #CHTC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CHTC" }}, "user": { "id": 1968176760, "name": "Stephanie Lamm", "screen_name": "slamm_5", "lang": "en", "location": "lost on public transportation", "create_at": date("2013-10-17"), "description": "Assistant City Editor @DailyTarHeel. Multimedia. Data. Policy. @unc 2017.", "followers_count": 708, "friends_count": 1435, "statues_count": 2488 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chapel Hill, NC", "id": "23ebe811ea93fd41", "name": "Chapel Hill", "place_type": "city", "bounding_box": rectangle("-79.083153,35.869376 -78.968887,35.969507") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37135, "countyName": "Orange", "cityID": 3711800, "cityName": "Chapel Hill" } }
+{ "create_at": datetime("2015-11-23T16:14:28.000Z"), "id": 668945742795374592, "text": "i went to go try on dresses and i felt like a princess in them����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2212001148, "name": "diana lmao", "screen_name": "dianamendeslmao", "lang": "en", "location": "Bay Area", "create_at": date("2013-11-23"), "description": "lmao alright", "followers_count": 1278, "friends_count": 343, "statues_count": 31967 }, "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("2015-11-23T16:14:28.000Z"), "id": 668945742816382976, "text": "You make me never wanna love again", "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": "Franklyn", "screen_name": "lil__nene", "lang": "en", "location": "Denver, CO", "create_at": date("2012-11-24"), "description": "Just a youngin with big dreams", "followers_count": 943, "friends_count": 907, "statues_count": 19665 }, "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:28.000Z"), "id": 668945743063846912, "text": "I want some Buffalo Wild Wings", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 842531490, "name": "ryan Thompson", "screen_name": "rthompson0812", "lang": "en", "location": "null", "create_at": date("2012-09-23"), "description": "I am Better than @caseysnowman23 HaHa", "followers_count": 19, "friends_count": 238, "statues_count": 244 }, "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("2015-11-23T16:14:28.000Z"), "id": 668945743219134464, "text": "Shavin Javan ���� #mavrick #mavrickartistsagency #axlemgmt #axle #modeling #fashion #model #portfolio… https://t.co/tcwra43Fgf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.78369091,33.54179133"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "mavrick", "mavrickartistsagency", "axlemgmt", "axle", "modeling", "fashion", "model", "portfolio" }}, "user": { "id": 2868663642, "name": "Javan Maegla Joslin", "screen_name": "Javan_Joslin", "lang": "en", "location": "Venice, California ", "create_at": date("2014-10-21"), "description": "Born in Nashiville, TN. I'm Ambidextrous . Heroes get remembered but legends never die, follow your heart kid, you'll never go wrong. JavanMaegla@gmail.com", "followers_count": 181, "friends_count": 959, "statues_count": 282 }, "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("2015-11-23T16:14:28.000Z"), "id": 668945743311323136, "text": "������ yeah ok https://t.co/hk79Snn1DZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 83974431, "name": "ig: brvndo.__", "screen_name": "kvng_shxt", "lang": "en", "location": "follows you", "create_at": date("2009-10-20"), "description": "run wit or run from me..", "followers_count": 1014, "friends_count": 935, "statues_count": 42645 }, "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:28.000Z"), "id": 668945743516934144, "text": "I got da anti twitter nigha shot before i confirmed my account back in 09 ima be regular regardless ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 158056426, "name": "The Enforcer", "screen_name": "Mr_AssHol", "lang": "en", "location": "SUITLAND nigga", "create_at": date("2010-06-21"), "description": "#SmokeAnStrokeGang #SmokeAnStrokeManagement\r\n#SuitlandSide #6Ward #TrueStoner #TeamSmokeAndStroke", "followers_count": 1717, "friends_count": 1430, "statues_count": 265527 }, "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("2015-11-23T16:14:28.000Z"), "id": 668945743781212160, "text": "im sarah. https://t.co/1xxHCdki12", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 2326063499, "name": "dabbinnn'", "screen_name": "sarah2lit", "lang": "en", "location": "null", "create_at": date("2014-02-05"), "description": "null", "followers_count": 519, "friends_count": 350, "statues_count": 151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waldorf, MD", "id": "4ddec4770a850303", "name": "Waldorf", "place_type": "city", "bounding_box": rectangle("-76.985159,38.608878 -76.841929,38.660379") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24017, "countyName": "Charles", "cityID": 2481175, "cityName": "Waldorf" } }
+{ "create_at": datetime("2015-11-23T16:14:28.000Z"), "id": 668945743974019072, "text": "���� https://t.co/FyoW4yipAP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 719584164, "name": "Flawzay Monay♡", "screen_name": "flawzaybrock", "lang": "en", "location": "null", "create_at": date("2012-07-26"), "description": "ig; monayyy____", "followers_count": 319, "friends_count": 377, "statues_count": 8643 }, "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("2015-11-23T16:14:28.000Z"), "id": 668945744028565504, "text": "@JasonHousto3922 \nThanks. We hot to keep working", "in_reply_to_status": 668945461802172417, "in_reply_to_user": 2274732510, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2274732510 }}, "user": { "id": 368105151, "name": "Coach Patton", "screen_name": "coachpatton40", "lang": "en", "location": "Waco, Tx", "create_at": date("2011-09-04"), "description": "Head Basketball Coach Midway High @midwayhoops", "followers_count": 400, "friends_count": 278, "statues_count": 3532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hewitt, TX", "id": "725760d442f62757", "name": "Hewitt", "place_type": "city", "bounding_box": rectangle("-97.218507,31.421915 -97.158685,31.488825") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4833428, "cityName": "Hewitt" } }
+{ "create_at": datetime("2015-11-23T16:14:28.000Z"), "id": 668945744053870593, "text": "Yes, fellow T passengers...I'm aware of my raccoon eyes #boxing #Monday #sweatbabysweat", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "boxing", "Monday", "sweatbabysweat" }}, "user": { "id": 26758586, "name": "Karina Becerra", "screen_name": "kbecerra", "lang": "en", "location": "Boston/San Anto/Mexico", "create_at": date("2009-03-26"), "description": "SideStepper & Wanderlust Vagabond | A Veces en #Español | Opinions My Own | #Tech #Marketing @comugrad Grad | Ball & Chained by a Mensch", "followers_count": 507, "friends_count": 1295, "statues_count": 1982 }, "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("2015-11-23T16:14:28.000Z"), "id": 668945744653516800, "text": "I use to not like soto, well I still don't like his bitchass, But he's one of my good homies now��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3189347119, "name": "Angel", "screen_name": "Angel68924605", "lang": "en", "location": "null", "create_at": date("2015-05-08"), "description": "null", "followers_count": 165, "friends_count": 132, "statues_count": 2632 }, "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("2015-11-23T16:14:28.000Z"), "id": 668945744934674433, "text": "Work got a little windy today... @ WLKY https://t.co/P3ExjD7xaw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.71030577,38.26382652"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 83270097, "name": "Jon Maynard", "screen_name": "theheadofhouse", "lang": "en", "location": "Louisville, KY", "create_at": date("2009-10-17"), "description": "Activist, artist, pilgrim.", "followers_count": 366, "friends_count": 712, "statues_count": 3710 }, "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:28.000Z"), "id": 668945744972353536, "text": "#NF @CA_DENCE tyvm 4the #follo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "sv", "is_retweet": false, "hashtags": {{ "NF", "follo" }}, "user_mentions": {{ 578291551 }}, "user": { "id": 2846987224, "name": "Ivory Tower Project", "screen_name": "ITPMark", "lang": "en", "location": "Queens, NY", "create_at": date("2014-10-27"), "description": "Independent recording artists: Ivory Tower Project is a new breed of Old School rockers-Original Classic Pop/Rock inspired by the great artists of the 70s+80s.", "followers_count": 1642, "friends_count": 1710, "statues_count": 4971 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Farmingdale, NY", "id": "c55c9fed765e35a1", "name": "South Farmingdale", "place_type": "city", "bounding_box": rectangle("-73.468517,40.704497 -73.429787,40.730502") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3669001, "cityName": "South Farmingdale" } }
+{ "create_at": datetime("2015-11-23T16:14:28.000Z"), "id": 668945745110634497, "text": "Tell us if you see us on TV���� #NYR @ Madison Square Garden https://t.co/hYVIu72tLq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.99278645,40.75052792"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NYR" }}, "user": { "id": 391752964, "name": "marisa danowski✌", "screen_name": "marisadanowskki", "lang": "en", "location": "null", "create_at": date("2011-10-15"), "description": "40.92N 72.66S Long Island⛅️", "followers_count": 350, "friends_count": 248, "statues_count": 5842 }, "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("2015-11-23T16:14:28.000Z"), "id": 668945745173610496, "text": "Elm n longbeach \nOut with my bros\nsb700 #�� #oygDbo @ Nails On Compton https://t.co/qE9Pvu94cC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.20864444,33.90031556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "oygDbo" }}, "user": { "id": 998425398, "name": "Wheel¢hairGangstaBSM", "screen_name": "mudiamondsDEBSM", "lang": "en", "location": "BUFFALO NY", "create_at": date("2012-12-08"), "description": "the #wheelchairgangsta ceo #dynestyent #bricksquadmonopoly affiliate i wanna make it #RG #muzikkzone (bmi artist) http://datpiff.com/Dj-Streetz-101…", "followers_count": 7659, "friends_count": 1903, "statues_count": 48909 }, "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("2015-11-23T16:14:28.000Z"), "id": 668945745207259136, "text": "mariano mas vale que subas el video de lali en la trastienda #LaliEnLaTrastienda", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "LaliEnLaTrastienda" }}, "user": { "id": 3008598335, "name": "Anto✨", "screen_name": "Esposbeautiful", "lang": "es", "location": "soy la peke de mi hermana lia❤", "create_at": date("2015-01-31"), "description": "Mariana Esposito∞Demetria Devonne Lovato\nM&M", "followers_count": 209, "friends_count": 97, "statues_count": 10801 }, "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("2015-11-23T16:14:28.000Z"), "id": 668945745504944128, "text": "Beyond irritated", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 324994749, "name": "DenverStrange", "screen_name": "DenverStrange", "lang": "en", "location": "405", "create_at": date("2011-06-27"), "description": "fearfully and wonderfully made •snapchat:denver.strange• fit coach• {Rose State}", "followers_count": 887, "friends_count": 1077, "statues_count": 15983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Guthrie, OK", "id": "0c6aa4d2eb567955", "name": "Guthrie", "place_type": "city", "bounding_box": rectangle("-97.469458,35.852965 -97.385124,35.898632") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40083, "countyName": "Logan", "cityID": 4031700, "cityName": "Guthrie" } }
+{ "create_at": datetime("2015-11-23T16:14:28.000Z"), "id": 668945745932918785, "text": "praying for everyone at citrus����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2932833797, "name": "samantha", "screen_name": "samduncannn", "lang": "en", "location": "tampa, florida", "create_at": date("2014-12-19"), "description": "ahs lacrosse", "followers_count": 209, "friends_count": 405, "statues_count": 1767 }, "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("2015-11-23T16:14:29.000Z"), "id": 668945746507538433, "text": "I have 0 motivation 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": 19891302, "name": "brooke", "screen_name": "akilliex", "lang": "en", "location": "null", "create_at": date("2009-02-01"), "description": "1 word: ruthless. RIP Cody", "followers_count": 372, "friends_count": 194, "statues_count": 23362 }, "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("2015-11-23T16:14:29.000Z"), "id": 668945746566213632, "text": "I'm just out here Tryna 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": 2909950095, "name": "Adaaa☯", "screen_name": "Amlrx3", "lang": "en", "location": "null", "create_at": date("2014-12-07"), "description": "Dubington", "followers_count": 509, "friends_count": 242, "statues_count": 16486 }, "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("2015-11-23T16:14:29.000Z"), "id": 668945746713042945, "text": "23-14. Tweets delayed for video.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1550505343, "name": "Herald and Chronicle", "screen_name": "OctoraroPublish", "lang": "en", "location": "Rising Sun, MD", "create_at": date("2013-06-27"), "description": "Local newspapers serving Cecil and southern Lancaster counties. Covering local news and sports.", "followers_count": 1062, "friends_count": 232, "statues_count": 5146 }, "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.965622,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("2015-11-23T16:14:29.000Z"), "id": 668945746880765952, "text": "sushi date with my cattle cattle price tag��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 340043976, "name": "pajbecker", "screen_name": "paigebecker50", "lang": "en", "location": "Kaplan, LA", "create_at": date("2011-07-21"), "description": "care free living.", "followers_count": 424, "friends_count": 225, "statues_count": 8914 }, "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("2015-11-23T16:14:29.000Z"), "id": 668945746893406208, "text": "Worst play call ever #iu #nopresswason", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "iu", "nopresswason" }}, "user": { "id": 1397221453, "name": "Derrick DeMoss", "screen_name": "DeMossD", "lang": "en", "location": "null", "create_at": date("2013-05-02"), "description": "null", "followers_count": 256, "friends_count": 243, "statues_count": 680 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sullivan, IN", "id": "b0d7cc29eb358470", "name": "Sullivan", "place_type": "city", "bounding_box": rectangle("-87.426353,39.081203 -87.381674,39.125504") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18153, "countyName": "Sullivan", "cityID": 1874006, "cityName": "Sullivan" } }
+{ "create_at": datetime("2015-11-23T16:14:29.000Z"), "id": 668945747212038146, "text": "This bitch❣ so lucky to have reconnected with her���� #redknightlight#medievaltimes https://t.co/HB2p9fDIzr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 550303853, "name": "Big AL", "screen_name": "AlliMerrill", "lang": "en", "location": "⭐️♋️", "create_at": date("2012-04-10"), "description": "⚫️", "followers_count": 79, "friends_count": 220, "statues_count": 1490 }, "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("2015-11-23T16:14:29.000Z"), "id": 668945747392536576, "text": "Ok... Who is available Dec 2nd & 3rd to cuddle/drink hot chocolate/watch the Santa Clause movies with me ?¿ thx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160656131, "name": "Mackenzie Fraim", "screen_name": "mackenziefraim", "lang": "en", "location": "Georgia someday", "create_at": date("2010-06-28"), "description": "GV // Marketing Major // Jadelynn Brooke Campus Rep // LYM Campus Crew", "followers_count": 1520, "friends_count": 695, "statues_count": 36206 }, "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("2015-11-23T16:14:29.000Z"), "id": 668945747421896704, "text": "Heaven help us. @backbayboy", "in_reply_to_status": 668938836899192834, "in_reply_to_user": 71800820, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 71800820 }}, "user": { "id": 73333278, "name": "Joyce Jeffries", "screen_name": "MsRock4Ever", "lang": "en", "location": "Louisville, KY", "create_at": date("2009-09-11"), "description": "Humanitarian, Social Activist, Yellow Dog Democrat; Equal Rights for ALL! LOVE President Obama, also music, books, Native America, & British Isles,", "followers_count": 15534, "friends_count": 17009, "statues_count": 284118 }, "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:29.000Z"), "id": 668945747585429504, "text": "@juliannacacc thanks chick love you too!", "in_reply_to_status": 668945669416214528, "in_reply_to_user": 355723808, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 355723808 }}, "user": { "id": 1708288640, "name": "Sean Brÿant", "screen_name": "_seanyb23", "lang": "en", "location": "42.3806° N, 71.2350° W", "create_at": date("2013-08-28"), "description": "Some things are better left untweeted - #JVWisdom", "followers_count": 575, "friends_count": 555, "statues_count": 5299 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Belmont, MA", "id": "f1d5d884f1dd220c", "name": "Belmont", "place_type": "city", "bounding_box": rectangle("-71.210571,42.375133 -71.151386,42.415861") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2505105, "cityName": "Belmont" } }
+{ "create_at": datetime("2015-11-23T16:14:29.000Z"), "id": 668945747778396160, "text": "I ❣ Animals.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 31287001, "name": "✨BuZzzz☺️", "screen_name": "Urbalicious_Yum", "lang": "en", "location": "P. Shermon, 42 Wallaby Way", "create_at": date("2009-04-14"), "description": "✏️ #ATL #TeamLeo♌️Honest* Happy* Quirky*Creative*Passionate*Optimistic* Introvert*West Indian by blood❤️ ⚠️Lady Lover⚠️", "followers_count": 440, "friends_count": 287, "statues_count": 12842 }, "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("2015-11-23T16:14:29.000Z"), "id": 668945747795165184, "text": "@chaz1944 undercover racist", "in_reply_to_status": 668940236219621376, "in_reply_to_user": 163813257, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 163813257 }}, "user": { "id": 570859259, "name": "roosevelt barfield", "screen_name": "colbarroosevelt", "lang": "en", "location": "null", "create_at": date("2012-05-04"), "description": "Concerned Citizen and Registered Voter.", "followers_count": 216, "friends_count": 186, "statues_count": 26968 }, "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("2015-11-23T16:14:29.000Z"), "id": 668945747988103168, "text": "@C_bucksSauce good one.", "in_reply_to_status": 668943080247181312, "in_reply_to_user": 162125890, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 162125890 }}, "user": { "id": 177545554, "name": "Evan Burgmeier", "screen_name": "EvanFratmeier", "lang": "en", "location": "Pensacola, FL", "create_at": date("2010-08-12"), "description": "The Dalai Lama told me that on my death bed I will receive total consciousness. So i got that going for me, which is nice", "followers_count": 368, "friends_count": 624, "statues_count": 2814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brent, FL", "id": "aab15d455af4dcc9", "name": "Brent", "place_type": "city", "bounding_box": rectangle("-87.285468,30.437965 -87.218996,30.504789") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12033, "countyName": "Escambia", "cityID": 1208300, "cityName": "Brent" } }
+{ "create_at": datetime("2015-11-23T16:14:29.000Z"), "id": 668945748340449280, "text": "So happy to be home in jersey thanks to my lovely sissy ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.717952,41.040879"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 497173065, "name": "Melissa Thomas", "screen_name": "melissthomas345", "lang": "en", "location": "Tampa, FL", "create_at": date("2012-02-19"), "description": "null", "followers_count": 252, "friends_count": 234, "statues_count": 5569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Andover, NJ", "id": "07d9d5b701882004", "name": "Andover, NJ", "place_type": "poi", "bounding_box": rectangle("-74.71795209999999,41.040878899999996 -74.717952,41.040879") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34027, "countyName": "Morris" } }
+{ "create_at": datetime("2015-11-23T16:14:29.000Z"), "id": 668945748440977408, "text": "mlrt us asf�������� @_xqueene", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.2676809,31.6320619"), "retweet_count": 0, "lang": "lt", "is_retweet": false, "user_mentions": {{ 2354026758 }}, "user": { "id": 2338847402, "name": "kadaja.", "screen_name": "kadajaa4", "lang": "en", "location": "teague, tx. ", "create_at": date("2014-02-11"), "description": "its 5 o'clock somewhere", "followers_count": 419, "friends_count": 215, "statues_count": 11090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Teague, TX", "id": "45c219ba999e5dea", "name": "Teague", "place_type": "city", "bounding_box": rectangle("-96.304086,31.59946 -96.255958,31.64951") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48161, "countyName": "Freestone", "cityID": 4872020, "cityName": "Teague" } }
+{ "create_at": datetime("2015-11-23T16:14:29.000Z"), "id": 668945748688400385, "text": "babes���� https://t.co/y5pOeFI5BH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3316567805, "name": "Alex Johnson", "screen_name": "alliej177", "lang": "en", "location": "kansas city MO", "create_at": date("2015-06-09"), "description": "Just be kind. and a die hard Royals fan.", "followers_count": 158, "friends_count": 199, "statues_count": 1438 }, "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("2015-11-23T16:14:29.000Z"), "id": 668945748860387328, "text": "Was going through the drive through @ Chik-Fil-a & saw babe looking all cute & shit �� I miss you so much! https://t.co/fWsKPTowrQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2711251495, "name": "Stoney McStonerson", "screen_name": "waddabug", "lang": "en", "location": "253 // Eastside Bremerton, WA ", "create_at": date("2014-08-05"), "description": "San Francisco 49ers ❤ | 3-7 | class of '17 // go ahead & lurk bitch~ chicana pride :.", "followers_count": 327, "friends_count": 176, "statues_count": 16144 }, "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("2015-11-23T16:14:29.000Z"), "id": 668945749233819649, "text": "Ew downgrade and a half bro", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 598855637, "name": "Emily O'Connell", "screen_name": "emxoxo329", "lang": "en", "location": "null", "create_at": date("2012-06-03"), "description": "snapchat - emily.oconnell", "followers_count": 579, "friends_count": 388, "statues_count": 12002 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hamburg, NY", "id": "0032b71e19053652", "name": "Hamburg", "place_type": "city", "bounding_box": rectangle("-78.916774,42.696345 -78.783331,42.789724") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3631643, "cityName": "Hamburg" } }
+{ "create_at": datetime("2015-11-23T16:14:29.000Z"), "id": 668945749443534848, "text": "So happy I passed my drug test ❤️������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1173750139, "name": "Leah Hack️", "screen_name": "thatbitchleahhh", "lang": "en", "location": "null", "create_at": date("2013-02-12"), "description": "who are you to judge? Just a heartless girl in this judgmental world #freechrismcgee", "followers_count": 681, "friends_count": 80, "statues_count": 33919 }, "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("2015-11-23T16:14:29.000Z"), "id": 668945749481267200, "text": "@Spags28 It's on our wishlist, but it's not looking like it right now", "in_reply_to_status": 668902511588335616, "in_reply_to_user": 417572250, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 417572250 }}, "user": { "id": 2422993615, "name": "The Open Bottle", "screen_name": "theopenbottleco", "lang": "en", "location": "Tinley Park, IL", "create_at": date("2014-04-01"), "description": "Bring home your beer, open it in store, or enjoy a glass of fresh beer in our tap room. We are your new favorite craft beer shop. (708) 263-0449", "followers_count": 829, "friends_count": 454, "statues_count": 1456 }, "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("2015-11-23T16:14:29.000Z"), "id": 668945749707632640, "text": "@mirandafinkral lol glad you clarified that Mirr ��", "in_reply_to_status": 668891226918092801, "in_reply_to_user": 346770995, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 346770995 }}, "user": { "id": 62166586, "name": "➶Lexx☀️☕️⚡️", "screen_name": "lexxdarlinn", "lang": "en", "location": "Probably in my hammock", "create_at": date("2009-08-01"), "description": "↟ I'm actually a sunflower. ↡", "followers_count": 535, "friends_count": 394, "statues_count": 13854 }, "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("2015-11-23T16:14:29.000Z"), "id": 668945749787455488, "text": "Today was honestly the worst 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": 367428352, "name": "Sergio Flores", "screen_name": "SergioSevillan", "lang": "en", "location": "Woodland Park, NJ", "create_at": date("2011-09-03"), "description": "null", "followers_count": 296, "friends_count": 266, "statues_count": 332 }, "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("2015-11-23T16:14:29.000Z"), "id": 668945749909102593, "text": "@Shanah_Cachet bet !!!! Lol", "in_reply_to_status": 668945421935472642, "in_reply_to_user": 143963022, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 143963022 }}, "user": { "id": 53630929, "name": "patriots undefeated", "screen_name": "jarel22", "lang": "en", "location": "null", "create_at": date("2009-07-04"), "description": "Cleveland", "followers_count": 1618, "friends_count": 1475, "statues_count": 137639 }, "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("2015-11-23T16:14:29.000Z"), "id": 668945750223491073, "text": "@TheOfficePosts @joaniesearle u + me", "in_reply_to_status": 668865228839387136, "in_reply_to_user": 1154072965, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1154072965, 536752256 }}, "user": { "id": 525824109, "name": "camillionaire", "screen_name": "CamilleDemman", "lang": "en", "location": "San Francisco, CA", "create_at": date("2012-03-15"), "description": "Meryl Streep is my spirit animal", "followers_count": 308, "friends_count": 322, "statues_count": 3798 }, "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:29.000Z"), "id": 668945750231920640, "text": "Someone take me to see this �� https://t.co/gGyNsZSysf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1102209787, "name": "ℋǟℵℕäɧ ₩ǎყ", "screen_name": "hannahway_", "lang": "en", "location": "null", "create_at": date("2013-01-18"), "description": "McMurry Unversity Soccer⚽️❤️", "followers_count": 745, "friends_count": 693, "statues_count": 4410 }, "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("2015-11-23T16:14:29.000Z"), "id": 668945750529875969, "text": "@Dziva_ lmfao he was like do you know why whips make that cracking sound?? It's because it's breaking the sound barrier ������", "in_reply_to_status": 668945110940405760, "in_reply_to_user": 885830450, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 885830450 }}, "user": { "id": 70049484, "name": "#SAMO©⚡", "screen_name": "_YoungPicasso_", "lang": "en", "location": "Surulere / D[M]V", "create_at": date("2009-08-29"), "description": "null", "followers_count": 2292, "friends_count": 1779, "statues_count": 186179 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945750596919296, "text": "@danimarieserban sorry. Should I do that next time I apparently tweet about her?", "in_reply_to_status": 668945362900643840, "in_reply_to_user": 99642440, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 99642440 }}, "user": { "id": 1012153122, "name": "NATE MARTIN", "screen_name": "Nathan34298550", "lang": "en", "location": "New Philadelphia, OH", "create_at": date("2012-12-14"), "description": "I enjoy throwing baseballs at high velocities. Don't talk about it, be about it. Senior NPHS. Snapchat: nathan_martin", "followers_count": 320, "friends_count": 284, "statues_count": 2044 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945750718619648, "text": "Finally making it to The Metropolitan Opera for the season premier of La Boheme! So pumped!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 101584786, "name": "Jonathan Pevarnek", "screen_name": "jpevarnek", "lang": "en", "location": "New York, NY", "create_at": date("2010-01-03"), "description": "Software Engineer for @GoogleIdeas", "followers_count": 82, "friends_count": 107, "statues_count": 360 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945750827646976, "text": "Still the jammie https://t.co/Knrq5HkFIw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70533195, "name": "Dan O'Leary", "screen_name": "danohleary", "lang": "en", "location": "Norwalk, CT", "create_at": date("2009-08-31"), "description": "I'm the one they call for a good time", "followers_count": 363, "friends_count": 202, "statues_count": 46056 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945751012192258, "text": "Besides Maryland & MSU the Big Ten is gonna suck this year for basketball.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 613911812, "name": "Jarrett", "screen_name": "BananaBrice", "lang": "en", "location": "Ortonville, MI", "create_at": date("2012-06-20"), "description": "Family, Friends, & Scott Mescudi", "followers_count": 508, "friends_count": 189, "statues_count": 29271 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ortonville, MI", "id": "3d5baee92992e621", "name": "Ortonville", "place_type": "city", "bounding_box": rectangle("-83.471992,42.811046 -83.414527,42.876021") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2661220, "cityName": "Ortonville" } }
+{ "create_at": datetime("2015-11-23T16:14:30.000Z"), "id": 668945751116873729, "text": "Single baby https://t.co/8Xadwrug1V", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 366897412, "name": "Cash Money", "screen_name": "Cash2_Cool", "lang": "en", "location": "DTX ", "create_at": date("2011-09-02"), "description": "#NavarroCollege'17 Making Dreams Become Reality", "followers_count": 1437, "friends_count": 798, "statues_count": 121363 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945751305789440, "text": "Selena is my wife,I cut my steak with a knife, I like turtles ��������������������������������������������������������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3043385855, "name": "Sid nasty 8️⃣", "screen_name": "_sidd8", "lang": "en", "location": "DeLand, FL", "create_at": date("2015-02-17"), "description": "the official twitter on Saeed El-Zayat™ snapchat //saeedelzayat8", "followers_count": 342, "friends_count": 433, "statues_count": 2862 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945751456661505, "text": "Someone bring food to Copeland Mills School of the Arts I'll be your BFF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 543072861, "name": "Hunter", "screen_name": "Hunteer_Morgann", "lang": "en", "location": "in bed", "create_at": date("2012-04-01"), "description": "HHS'16 • CMSA Dance Magic • Miss Hickory 2016", "followers_count": 541, "friends_count": 526, "statues_count": 14068 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945751737679872, "text": "I want you , and you want me ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2839579873, "name": "marwheezyyy ", "screen_name": "MarzYabish", "lang": "en", "location": "null", "create_at": date("2014-10-03"), "description": "null", "followers_count": 255, "friends_count": 644, "statues_count": 185 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945751821692928, "text": "@_alana___ happy birthday alana ! ����", "in_reply_to_status": -1, "in_reply_to_user": 451782797, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 451782797 }}, "user": { "id": 3388688350, "name": "Niyah ❤", "screen_name": "___NiyNiy", "lang": "en", "location": "null", "create_at": date("2015-07-22"), "description": "on my own wave ! sc : nikilla15", "followers_count": 314, "friends_count": 330, "statues_count": 2003 }, "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:30.000Z"), "id": 668945751918125058, "text": "don't trust it at all", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1145980616, "name": "g", "screen_name": "gvercetti", "lang": "en", "location": "null", "create_at": date("2013-02-03"), "description": "sunkissed punjabi - ig: g.vercetti", "followers_count": 2142, "friends_count": 754, "statues_count": 56224 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945752194969600, "text": "@Renene__ ������", "in_reply_to_status": 668906494667137024, "in_reply_to_user": 63576035, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 63576035 }}, "user": { "id": 27983804, "name": "Jasmine Hendrix", "screen_name": "juicyJAYCouture", "lang": "en", "location": "Tampa, FL", "create_at": date("2009-03-31"), "description": "AkilahJanaeBlount | 21 | YourFavoriteFlight Attendant | IG - JaszCouture_ | Black&Italian | USF", "followers_count": 3280, "friends_count": 851, "statues_count": 84382 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945752291430401, "text": "Chillin with chene sis my sis lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3020264355, "name": "LilBrown.", "screen_name": "TaliyahD_", "lang": "en", "location": "Detroit, MI", "create_at": date("2015-02-05"), "description": "Lil Cool ass Chick from the Westside", "followers_count": 258, "friends_count": 238, "statues_count": 4294 }, "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:30.000Z"), "id": 668945752312295424, "text": "@rsolis1128 @XXL FORREAL. Gonna cop one of them hoes. ����", "in_reply_to_status": 668945483780390912, "in_reply_to_user": 537709886, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 537709886, 14629315 }}, "user": { "id": 167299374, "name": "Joseph Villarreal", "screen_name": "JuicyJoe_225", "lang": "en", "location": "null", "create_at": date("2010-07-15"), "description": "Crunch life. UTRGV life.", "followers_count": 544, "friends_count": 296, "statues_count": 12323 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945752467505152, "text": "@msheffield00 ohhhhh yeah huh ��", "in_reply_to_status": 668945463924539392, "in_reply_to_user": 2275030716, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2275030716 }}, "user": { "id": 2367953534, "name": "Medina .", "screen_name": "alyssamedina55", "lang": "en", "location": "null", "create_at": date("2014-03-01"), "description": "Philippians 4:13 // Check out mine & Shorty's Youtube video, link below...like & subscribe !", "followers_count": 537, "friends_count": 319, "statues_count": 18590 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945752668815360, "text": "I wanna leave the house and hang out with someone with hot chocolate", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 434479026, "name": "diego the sushi god", "screen_name": "uglyassdiego", "lang": "en", "location": "323 512", "create_at": date("2011-12-11"), "description": "I'm unattractive, but you're still lurking", "followers_count": 312, "friends_count": 256, "statues_count": 13920 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945752949981184, "text": "@GovJVentura - Amen brother... We as a species are still territorial war monkeys fighting over common ground...", "in_reply_to_status": 668944626435276800, "in_reply_to_user": 2218177956, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2218177956 }}, "user": { "id": 1855713859, "name": "Cozy Cottage Home ", "screen_name": "CCHomeFurn", "lang": "en", "location": "We Ship Worldwide!", "create_at": date("2013-09-11"), "description": "Eclectic Fun Home Décor", "followers_count": 575, "friends_count": 895, "statues_count": 747 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Island Lake, IL", "id": "2358ff8216a2895b", "name": "Island Lake", "place_type": "city", "bounding_box": rectangle("-88.249433,42.254043 -88.167046,42.306868") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17111, "countyName": "McHenry", "cityID": 1737894, "cityName": "Island Lake" } }
+{ "create_at": datetime("2015-11-23T16:14:30.000Z"), "id": 668945753071468544, "text": "My neighbor is fuckin his hyna and i can not only hear them but the bed is shakin 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": 1582141069, "name": "Totinski", "screen_name": "Anteguadalupe", "lang": "en", "location": "null", "create_at": date("2013-07-09"), "description": "@Blyssfull is amazing", "followers_count": 496, "friends_count": 505, "statues_count": 28316 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945753092571136, "text": "@Kalliex3 appears to be the case this year. Oh well, I'm happy to be going honestly.", "in_reply_to_status": 668943045996466177, "in_reply_to_user": 27218038, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27218038 }}, "user": { "id": 96918563, "name": "blaked potato.", "screen_name": "BlakeRAWRZ", "lang": "en", "location": "Flavor Town, USA ", "create_at": date("2009-12-14"), "description": "[24] [Open Minded] [Anxious] [Butt Enthusiast]", "followers_count": 300, "friends_count": 229, "statues_count": 37655 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temperance, MI", "id": "422fb2c25d040f2f", "name": "Temperance", "place_type": "city", "bounding_box": rectangle("-83.58944,41.729339 -83.528417,41.822604") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26115, "countyName": "Monroe", "cityID": 2679240, "cityName": "Temperance" } }
+{ "create_at": datetime("2015-11-23T16:14:30.000Z"), "id": 668945753193058304, "text": "He karate kicked her������ https://t.co/Obzop0qOLm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3094718522, "name": "Hitman", "screen_name": "bmoreboykc", "lang": "en", "location": "Pembroke Pines, FL", "create_at": date("2015-03-17"), "description": "Baltimore, MD| sc: kcwavegame23", "followers_count": 80, "friends_count": 58, "statues_count": 947 }, "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:30.000Z"), "id": 668945753197400068, "text": "#weave #twerk #happyclients https://t.co/JCGjoHelW7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.41150013,40.8502681"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "weave", "twerk", "happyclients" }}, "user": { "id": 426321789, "name": "Rockin' Pretty Stylz", "screen_name": "RockPrettyStylz", "lang": "en", "location": "Huntington, NY", "create_at": date("2011-12-01"), "description": "Skilled and licensed in the newest extension and straightening techniques. New trends and classic methods in coloring, cutting, treatment, and bonding.", "followers_count": 8, "friends_count": 99, "statues_count": 3 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945753608478720, "text": "Cannot wait for ABC Family's 25 Days of Christmas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2212957865, "name": "Ericka", "screen_name": "ErickaJayBeeee", "lang": "en", "location": "Jax, FL", "create_at": date("2013-12-07"), "description": "Senior | Worry about your character, not your reputation. Your character is who you are. Your reputation is who people think you are. Sc: erickajaybee |", "followers_count": 581, "friends_count": 353, "statues_count": 13907 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakleaf Plantation, FL", "id": "014f51c9371e663a", "name": "Oakleaf Plantation", "place_type": "city", "bounding_box": rectangle("-81.865314,30.138094 -81.777908,30.189141") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12019, "countyName": "Clay", "cityID": 1250630, "cityName": "Oakleaf Plantation" } }
+{ "create_at": datetime("2015-11-23T16:14:30.000Z"), "id": 668945753704947712, "text": "Rs! �� https://t.co/ZOa9nKezW4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 465584911, "name": "Sage❣", "screen_name": "Sageb____", "lang": "en", "location": "null", "create_at": date("2012-01-16"), "description": "SOUR!", "followers_count": 314, "friends_count": 242, "statues_count": 6954 }, "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:30.000Z"), "id": 668945753826529280, "text": "@ImaniOlander Im sorry my phone died ��������", "in_reply_to_status": 668945550893383680, "in_reply_to_user": 2825161789, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2825161789 }}, "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": 1038, "friends_count": 1783, "statues_count": 5536 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945753830723584, "text": "@ElnegroTrias @dannyviLe Son gob q defienden la industria nacional.No digo q este bien...", "in_reply_to_status": 668945402771697665, "in_reply_to_user": 158804385, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 158804385, 255476951 }}, "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": 87426 }, "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:30.000Z"), "id": 668945753906130945, "text": "Constantly redecorating. https://t.co/wcLqKDiioi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 334396065, "name": "Queen Ä", "screen_name": "BanditsNThieves", "lang": "en", "location": "Norman, OK", "create_at": date("2011-07-12"), "description": "OKHC • Photography • Vinyls • Travel • ⛺️", "followers_count": 253, "friends_count": 147, "statues_count": 5955 }, "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:30.000Z"), "id": 668945754036289536, "text": "same. https://t.co/ypG1zUoGPS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1126253370, "name": "zy❣", "screen_name": "zyonnicole", "lang": "en", "location": "the gutter ", "create_at": date("2013-01-27"), "description": "student athlete.3️⃣ all star⭐️ all state 2014 (: God first‼️ #blessed // 1027 // future pediatric nurse practitioner ' #UA20 #rolltide❤️", "followers_count": 948, "friends_count": 819, "statues_count": 56463 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945754107584512, "text": "@Hadezipoo @StimulusPact Hadez AX coming in hot. jk you suk", "in_reply_to_status": 668945234437517312, "in_reply_to_user": 2337634416, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2337634416, 1834501549 }}, "user": { "id": 792882901, "name": "Pixar AX | Max", "screen_name": "MrChanese", "lang": "en", "location": "Jew Land", "create_at": date("2012-08-30"), "description": "15. Lead Editor of @AxisKnifing. Joined 1/5/15. Unranked CSGO fag. Squad: Laprah | Lux | Yebca | Void | Sitrah | Neko | Skeet. DM for editing prices. Smile More", "followers_count": 1385, "friends_count": 100, "statues_count": 32311 }, "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:30.000Z"), "id": 668945754141016064, "text": "It's going to take me like 3 hours to copy all the pictures of my last trip to the computer��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 939866779, "name": "➴", "screen_name": "joaquinferriol", "lang": "es", "location": "Washington, USA", "create_at": date("2012-11-10"), "description": "Instagram ⌲ @joaquinferriol", "followers_count": 268, "friends_count": 184, "statues_count": 1735 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945754166272000, "text": "I love you @lindseytalleyyy ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 437655715 }}, "user": { "id": 369254510, "name": "Carolyn Campbell", "screen_name": "ccampbelll16", "lang": "en", "location": "null", "create_at": date("2011-09-06"), "description": "Bridgewater College '19 | BCSB⚾️", "followers_count": 1213, "friends_count": 618, "statues_count": 26643 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945754216599552, "text": "Our Planca Roasted#Salmon finished with Raw Jungle Honey from our friends followthehoney! We've… https://t.co/Zl4fbmC67h", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.1208725,42.3718681"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 479934094, "name": "PARK Cambridge", "screen_name": "PARKCambridge", "lang": "en", "location": "Harvard Square / Cambridge, MA", "create_at": date("2012-01-31"), "description": "PARK is the neighborhood restaurant and bar of today: a vibrant destination to linger over lively conversation, playful plates and thoughtful cocktails.", "followers_count": 1782, "friends_count": 709, "statues_count": 2021 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945754287919106, "text": "Crean is such a dumbass. Horrible use of timeouts. Terrible play drawn up. Getting out rebounded like crazy never helps either. 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": 2758804049, "name": "E. Tanner Robison", "screen_name": "tanner_robison", "lang": "en", "location": "New Albany ➡ Terre Haute", "create_at": date("2014-09-01"), "description": "I enjoy Hip-Hop, Baseball, and Katie", "followers_count": 323, "friends_count": 279, "statues_count": 6554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Terre Haute, IN", "id": "8a61588aff8b0577", "name": "Terre Haute", "place_type": "city", "bounding_box": rectangle("-87.460551,39.394564 -87.303557,39.520714") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18167, "countyName": "Vigo", "cityID": 1875428, "cityName": "Terre Haute" } }
+{ "create_at": datetime("2015-11-23T16:14:30.000Z"), "id": 668945754556239873, "text": "Fuck school", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3511299734, "name": "Jason", "screen_name": "jasongoicochea", "lang": "en", "location": "Salina, KS", "create_at": date("2015-09-09"), "description": "SBVC✈️KWU ⚽️ SC: goicochea12", "followers_count": 169, "friends_count": 219, "statues_count": 1156 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salina, KS", "id": "e7bae49f1ac7f22e", "name": "Salina", "place_type": "city", "bounding_box": rectangle("-97.676631,38.768801 -97.557719,38.885242") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20169, "countyName": "Saline", "cityID": 2062700, "cityName": "Salina" } }
+{ "create_at": datetime("2015-11-23T16:14:30.000Z"), "id": 668945754573004807, "text": "I ALMOST LOST MY SHIT #MTVSTARS #VideoMTV2015 5 Seconds Of Summer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MTVSTARS", "VideoMTV2015" }}, "user": { "id": 217480023, "name": "ana :-) TL", "screen_name": "STRlPPEDMGC", "lang": "en-gb", "location": "rowyso Dallas & slfl Dallas ", "create_at": date("2010-11-19"), "description": "the blood in my veins is made up of mistake", "followers_count": 16861, "friends_count": 4931, "statues_count": 116541 }, "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("2015-11-23T16:14:30.000Z"), "id": 668945754598350848, "text": "Phone gets no love", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2156066698, "name": "Trae", "screen_name": "__tsc", "lang": "en", "location": " way up", "create_at": date("2013-10-27"), "description": "null", "followers_count": 750, "friends_count": 543, "statues_count": 15646 }, "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("2015-11-23T16:14:31.000Z"), "id": 668945754833051648, "text": "Canada's Oil Reversal Marks a Turning Point for Climate Politics \nhttps://t.co/1wits6LRx9 via @Esquire\n#KeystoneXL #Energy #environment", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "KeystoneXL", "Energy", "environment" }}, "user_mentions": {{ 20455625 }}, "user": { "id": 1935100699, "name": "Joe Gerard", "screen_name": "OklaVoter", "lang": "en", "location": "Tulsa", "create_at": date("2013-10-04"), "description": "Believe in the American Dream. Not the Republican Scheme.", "followers_count": 258, "friends_count": 536, "statues_count": 6396 }, "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("2015-11-23T16:14:31.000Z"), "id": 668945754992549888, "text": "First one down������ https://t.co/NuOZWMdHIf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 392389484, "name": "Olivia Beck", "screen_name": "livbeck93", "lang": "en", "location": "(219) & (574)", "create_at": date("2011-10-16"), "description": "just your average sports-loving, biology nerd senior at saint mary's", "followers_count": 256, "friends_count": 331, "statues_count": 10750 }, "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("2015-11-23T16:14:31.000Z"), "id": 668945755101597697, "text": "I need to stop clowning myself man ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2765379648, "name": "shittt.", "screen_name": "thefamouslexs", "lang": "en", "location": "Moss Bluff, LA", "create_at": date("2014-08-24"), "description": "insta: thefamouslexs | snap: lexsfoster | rip sitó&bob |❤️", "followers_count": 972, "friends_count": 911, "statues_count": 6630 }, "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("2015-11-23T16:14:31.000Z"), "id": 668945755227430913, "text": "or nah https://t.co/qpDVUzXgiB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 408739667, "name": "Kayla Everslage", "screen_name": "Kayla_Everslage", "lang": "en", "location": "502 // 859", "create_at": date("2011-11-09"), "description": "|| UK'18 Delta Zeta || vincentlococo ✌️", "followers_count": 640, "friends_count": 139, "statues_count": 15859 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsboro Farm, KY", "id": "00728f4d0894d30c", "name": "Brownsboro Farm", "place_type": "city", "bounding_box": rectangle("-85.600403,38.298858 -85.572238,38.315005") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2110162, "cityName": "Brownsboro Farm" } }
+{ "create_at": datetime("2015-11-23T16:14:31.000Z"), "id": 668945755458007040, "text": "i have people so confused by my tweets", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1392473923, "name": "heather geronsin", "screen_name": "HEATHERGERONSiN", "lang": "en", "location": "null", "create_at": date("2013-04-30"), "description": "null", "followers_count": 71, "friends_count": 130, "statues_count": 1915 }, "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("2015-11-23T16:14:31.000Z"), "id": 668945755718184960, "text": "U just got to be mature bout how shit goes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 383507139, "name": "Zee", "screen_name": "AintSpanishHoe", "lang": "en", "location": "null", "create_at": date("2011-10-01"), "description": "20 | if speaking was silver then Silence is gold | SC : AintSpanishHoe", "followers_count": 528, "friends_count": 863, "statues_count": 17072 }, "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("2015-11-23T16:14:31.000Z"), "id": 668945756066177024, "text": "@JasonLindberg86 Large sized adult coats are most in-need but we won't turn away a good winter coat of any size.", "in_reply_to_status": 668916023832150017, "in_reply_to_user": 4106231114, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4106231114 }}, "user": { "id": 93559677, "name": "Fan HQ", "screen_name": "FanHQ", "lang": "en", "location": "Mntka, Eden Prairie, St. Cloud", "create_at": date("2009-11-29"), "description": "Fan HQ is your headquarters for officially licensed sports apparel, authentic memorabilia and autograph appearances. If it came from HQ - IT'S AUTHENTIQ!", "followers_count": 3024, "friends_count": 1228, "statues_count": 9617 }, "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("2015-11-23T16:14:31.000Z"), "id": 668945756129255425, "text": "I can barely keep my eyes open and I still have to go pick rachel up from work 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": 258419430, "name": "brittneyyyyyy", "screen_name": "bdoobzy", "lang": "en", "location": "ma", "create_at": date("2011-02-27"), "description": "bay path '16 • cosmetology • I find comfort in weakness because it's always there for me", "followers_count": 1010, "friends_count": 856, "statues_count": 44655 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spencer, MA", "id": "1dac4f191436694e", "name": "Spencer", "place_type": "city", "bounding_box": rectangle("-72.019832,42.213742 -71.946048,42.275719") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25027, "countyName": "Worcester", "cityID": 2566070, "cityName": "Spencer" } }
+{ "create_at": datetime("2015-11-23T16:14:31.000Z"), "id": 668945756171190273, "text": "��✈️ vamo noix (@ Dallas/Fort Worth International Airport (DFW) in Grapevine, TX) https://t.co/5G6SJpvfSH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.04086304,32.89803818"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 183825271, "name": "Mary Person⚽️❤️", "screen_name": "MaryPerson10", "lang": "en", "location": "United States", "create_at": date("2010-08-27"), "description": "Brasileira,de uma fé inabalavel.6 sentido aguçado.Jogadora de futebol.Apaixonada pela minha familia.\nInstagram: @maryperson10", "followers_count": 715, "friends_count": 182, "statues_count": 10862 }, "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("2015-11-23T16:14:31.000Z"), "id": 668945756355620864, "text": "Stay in ur lane ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 140264306, "name": "Kat", "screen_name": "katnguyen_", "lang": "en", "location": "Dulles HS '17", "create_at": date("2010-05-04"), "description": "@BlakeTakushi", "followers_count": 868, "friends_count": 554, "statues_count": 24178 }, "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("2015-11-23T16:14:31.000Z"), "id": 668945756548521984, "text": "Accident cleared in #Tulsa on 71st St at US 169 #traffic https://t.co/aSW1zAo2gS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.85956,36.06094"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Tulsa", "traffic" }}, "user": { "id": 249883532, "name": "TTN Tulsa", "screen_name": "TotalTrafficTUL", "lang": "en", "location": "Tulsa, OK", "create_at": date("2011-02-09"), "description": "null", "followers_count": 1089, "friends_count": 41, "statues_count": 24392 }, "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("2015-11-23T16:14:31.000Z"), "id": 668945756720635905, "text": "#jeopardy #themostannoyingwomanintheworld", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "jeopardy", "themostannoyingwomanintheworld" }}, "user": { "id": 538579408, "name": "OSURob3000", "screen_name": "OSUROB3000", "lang": "en", "location": "null", "create_at": date("2012-03-27"), "description": "Buckeyes!", "followers_count": 119, "friends_count": 956, "statues_count": 1611 }, "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("2015-11-23T16:14:31.000Z"), "id": 668945756808724481, "text": "@thesixthnewkid that's good. No one has to me at all", "in_reply_to_status": 668602260981735425, "in_reply_to_user": 56511752, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 56511752 }}, "user": { "id": 3199864498, "name": "Amy Butterbaugh", "screen_name": "emmettsbrat2", "lang": "en", "location": "Florida, USA", "create_at": date("2015-04-23"), "description": "A huge Kellan Lutz fan. Love to read", "followers_count": 49, "friends_count": 120, "statues_count": 67 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mango, FL", "id": "a22b1167a3d2c56f", "name": "Mango", "place_type": "city", "bounding_box": rectangle("-82.32727,27.972843 -82.277706,28.012429") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1242850, "cityName": "Mango" } }
+{ "create_at": datetime("2015-11-23T16:14:31.000Z"), "id": 668945756921794560, "text": "Hez rt https://t.co/cxpo8HC8LU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.370603,29.752067"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 2544795223, "name": "DONRIKOCZAE", "screen_name": "MR8DONRIKOCZAE", "lang": "en", "location": "LYFEINCORPORATED", "create_at": date("2014-06-03"), "description": "http://KICKSTARTER.COM/PROOFOFLYFE", "followers_count": 70, "friends_count": 250, "statues_count": 24199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tellepsen Family Downtown YMCA", "id": "07d9d37d71487002", "name": "Tellepsen Family Downtown YMCA", "place_type": "poi", "bounding_box": rectangle("-95.3706031,29.7520669 -95.370603,29.752067") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2015-11-23T16:14:31.000Z"), "id": 668945757202853888, "text": "Could use a training partner to make mo gainzzz https://t.co/MyoZyU3IBu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 416661103, "name": "Marvin Mendoza", "screen_name": "Adan_Brings_It", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-11-19"), "description": "Living the dream, one rep at a time ✨♑", "followers_count": 44, "friends_count": 43, "statues_count": 740 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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:31.000Z"), "id": 668945757223845889, "text": "@firewolf032005 https://t.co/BTrr04ndS4", "in_reply_to_status": -1, "in_reply_to_user": 2827282908, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2827282908 }}, "user": { "id": 3268734937, "name": "Delia Kreftmeyer", "screen_name": "kreftsdm", "lang": "en", "location": "El Paso, TX", "create_at": date("2015-07-04"), "description": "Texas Works Lead Advisor for the Texas Health and Human Services Commission", "followers_count": 89, "friends_count": 336, "statues_count": 546 }, "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("2015-11-23T16:14:31.000Z"), "id": 668945757676810240, "text": "I luv things about her but I don't luv her", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2269836366, "name": "Zoë", "screen_name": "zoelastname", "lang": "en", "location": "null", "create_at": date("2013-12-30"), "description": "HTX | ΦΜ", "followers_count": 400, "friends_count": 291, "statues_count": 3419 }, "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("2015-11-23T16:14:31.000Z"), "id": 668945758511435776, "text": "LMFAO MY FRIENDS ARE SO FUNNY OMG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 500170313, "name": "yulisa", "screen_name": "Yulisa_x3", "lang": "en", "location": "null", "create_at": date("2012-02-22"), "description": "keep your eye on the prize", "followers_count": 1187, "friends_count": 772, "statues_count": 37390 }, "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("2015-11-23T16:14:31.000Z"), "id": 668945758515675137, "text": "help me study for my permit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2200203374, "name": "alo", "screen_name": "__AlondraIsabel", "lang": "en", "location": "null", "create_at": date("2013-11-17"), "description": "cool calm & collected", "followers_count": 573, "friends_count": 444, "statues_count": 14147 }, "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:32.000Z"), "id": 668945758998122497, "text": "Fucking white people ....... https://t.co/x5ZyVOGz11", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2239727394, "name": "Brayden Squires", "screen_name": "BraydenSquires5", "lang": "en", "location": "Tennessee, USA", "create_at": date("2013-12-10"), "description": "null", "followers_count": 416, "friends_count": 290, "statues_count": 1884 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Smyrna, TN", "id": "f9eb63ab4f7dd5db", "name": "Smyrna", "place_type": "city", "bounding_box": rectangle("-86.624429,35.879451 -86.458783,36.029213") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47149, "countyName": "Rutherford", "cityID": 4769420, "cityName": "Smyrna" } }
+{ "create_at": datetime("2015-11-23T16:14:32.000Z"), "id": 668945759056695296, "text": "@LivnLrgChicago you make it easy to be nice! Your beauty is such a joy to gaze upon!", "in_reply_to_status": 668914686222536704, "in_reply_to_user": 2492090557, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2492090557 }}, "user": { "id": 2595562142, "name": "Bill Mayse", "screen_name": "BillMayse", "lang": "en", "location": "Houston, TX", "create_at": date("2014-06-29"), "description": "null", "followers_count": 85, "friends_count": 499, "statues_count": 186 }, "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("2015-11-23T16:14:32.000Z"), "id": 668945759065079809, "text": "That was a nice nap☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 473378637, "name": "Mando", "screen_name": "YoMando23", "lang": "en", "location": "null", "create_at": date("2012-01-24"), "description": "I can be your sugar daddy", "followers_count": 444, "friends_count": 332, "statues_count": 7289 }, "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("2015-11-23T16:14:32.000Z"), "id": 668945759094439936, "text": "I need to buy some Chapstick", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 139004753, "name": "glocktalk", "screen_name": "FurtherMoor", "lang": "en", "location": "Western Ave", "create_at": date("2010-05-01"), "description": "wats hatnin #TXST (Nigerian)", "followers_count": 1004, "friends_count": 311, "statues_count": 143774 }, "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("2015-11-23T16:14:32.000Z"), "id": 668945759119757312, "text": "It me https://t.co/Zj04wmYFNy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1202539458, "name": "Shän☾", "screen_name": "BlakeRobonson", "lang": "en", "location": "null", "create_at": date("2013-02-20"), "description": "Iconic", "followers_count": 200, "friends_count": 473, "statues_count": 967 }, "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("2015-11-23T16:14:32.000Z"), "id": 668945759186714624, "text": "@dirtysocksmgc WAS THIS OUTSIDE THE RIVIERA FOR 5SOS", "in_reply_to_status": 668945595843801089, "in_reply_to_user": 621666518, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 621666518 }}, "user": { "id": 329497063, "name": "emily", "screen_name": "kingIaurent", "lang": "en", "location": "null", "create_at": date("2011-07-04"), "description": "in the grand scheme of things", "followers_count": 20563, "friends_count": 293, "statues_count": 184054 }, "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:32.000Z"), "id": 668945759262277632, "text": "Few things compare to bubble baths and naps", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1406771192, "name": "Courtney", "screen_name": "courtneyrainer", "lang": "en", "location": "Coeur d'alene, ID", "create_at": date("2013-05-05"), "description": "pretty heart • pretty mind • pretty soul", "followers_count": 270, "friends_count": 183, "statues_count": 1191 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coeur d'Alene, ID", "id": "da6f5e66f73b8f7f", "name": "Coeur d'Alene", "place_type": "city", "bounding_box": rectangle("-116.855401,47.65878 -116.732088,47.744794") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16055, "countyName": "Kootenai", "cityID": 1616750, "cityName": "Coeur d'Alene" } }
+{ "create_at": datetime("2015-11-23T16:14:32.000Z"), "id": 668945759291576320, "text": "some you mo fo's called me an idiot when I Said trade Michael and Turbin .. to you I say Talk some more shit why dont ya Rawls Royce baby!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 245218524, "name": "CHRIS BARLOW", "screen_name": "BARLOW253", "lang": "en", "location": "tacoma wa", "create_at": date("2011-01-30"), "description": "#253 #balleronabudget seattle sports fan.. outspoken loud .. not giving a shit", "followers_count": 501, "friends_count": 814, "statues_count": 11083 }, "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("2015-11-23T16:14:32.000Z"), "id": 668945759333695488, "text": "Where is Mecca McFadden?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 240501091, "name": "Myles W.B. Davis", "screen_name": "oldirtyginger", "lang": "en", "location": "MPLS MKE ATL", "create_at": date("2011-01-19"), "description": "Knowledge is the key? Then show me the lock. Ain't a hood nigga but a nigga from the hood. The Morehouse College.", "followers_count": 1590, "friends_count": 1246, "statues_count": 36805 }, "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("2015-11-23T16:14:32.000Z"), "id": 668945759543435264, "text": "i don't believe in Cupid, that shit stupid", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 589597216, "name": "Chri$", "screen_name": "__PoloManChris", "lang": "en", "location": "Columbus, GA", "create_at": date("2012-05-24"), "description": "The Finesse Kid ‼️‼️‼️", "followers_count": 616, "friends_count": 228, "statues_count": 41571 }, "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:32.000Z"), "id": 668945759895724032, "text": "you're so obsessed �� stop", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 162940395, "name": "Rαchhh ❁", "screen_name": "Rachael_Haleyyy", "lang": "en", "location": "East Tennessee", "create_at": date("2010-07-04"), "description": "Instagram: @rachael_haleyyy | ❁ all smiles // YL", "followers_count": 1458, "friends_count": 1055, "statues_count": 8528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elizabethton, TN", "id": "00bad99d6b05a213", "name": "Elizabethton", "place_type": "city", "bounding_box": rectangle("-82.319523,36.285879 -82.154114,36.368033") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47019, "countyName": "Carter", "cityID": 4723500, "cityName": "Elizabethton" } }
+{ "create_at": datetime("2015-11-23T16:14:32.000Z"), "id": 668945760076111872, "text": "@ToriProkop that's unfortunately the way of the world. 1/2 the information and not enough empathy. Been in those shoes many times. Sorry!", "in_reply_to_status": 668921901004230656, "in_reply_to_user": 19139863, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19139863 }}, "user": { "id": 449367126, "name": "Kevin Brigandi", "screen_name": "KevinBrigandi", "lang": "en", "location": "Connecticut", "create_at": date("2011-12-28"), "description": "Principal at Terracon Consultants, Inc.", "followers_count": 61, "friends_count": 235, "statues_count": 470 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gahanna, OH", "id": "c97807ac2cd60207", "name": "Gahanna", "place_type": "city", "bounding_box": rectangle("-82.905845,39.987076 -82.802554,40.05651") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3929106, "cityName": "Gahanna" } }
+{ "create_at": datetime("2015-11-23T16:14:32.000Z"), "id": 668945760239689728, "text": "@aniyahadley OMGGGGG we just had this conversation ���������� https://t.co/7f5yXnCLW2", "in_reply_to_status": -1, "in_reply_to_user": 1592954720, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1592954720 }}, "user": { "id": 2440470920, "name": "Lex❤", "screen_name": "alexis_victorya", "lang": "en", "location": "null", "create_at": date("2014-04-12"), "description": "❤⚽️❤", "followers_count": 502, "friends_count": 766, "statues_count": 4264 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Springdale, OH", "id": "2eb3732437b18323", "name": "Springdale", "place_type": "city", "bounding_box": rectangle("-84.513022,39.269985 -84.447348,39.305817") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3974104, "cityName": "Springdale" } }
+{ "create_at": datetime("2015-11-23T16:14:32.000Z"), "id": 668945760357064706, "text": "The love ain't gone fool", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 379255975, "name": "Steph Bez", "screen_name": "stephbezanilla", "lang": "en", "location": "null", "create_at": date("2011-09-24"), "description": "¯\\_(ツ)_/¯", "followers_count": 373, "friends_count": 306, "statues_count": 10706 }, "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("2015-11-23T16:14:32.000Z"), "id": 668945760482934784, "text": "cold asf man .. this sht real asf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1255400606, "name": "kebadasss..", "screen_name": "keNicademo", "lang": "en", "location": "252", "create_at": date("2013-03-09"), "description": "stay positive .. ya time coming\nsc: shawnnn0402", "followers_count": 802, "friends_count": 798, "statues_count": 18631 }, "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("2015-11-23T16:14:32.000Z"), "id": 668945760696668160, "text": "Great finish", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 468191126, "name": "Burt Macklin", "screen_name": "Kylewagn", "lang": "en", "location": "Bloomington, Indiana", "create_at": date("2012-01-19"), "description": "Count your age by friends, not years. Count your life by smiles, not tears. - John Lennon", "followers_count": 388, "friends_count": 447, "statues_count": 15510 }, "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("2015-11-23T16:14:32.000Z"), "id": 668945761128837125, "text": "Bondieu pa'm nan gran .,", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 33559092, "name": "joëy", "screen_name": "idjory", "lang": "en", "location": "Haiti/Miami/Heaven", "create_at": date("2009-04-20"), "description": "#NewHeaven ,#NewEarth . #Redemption , #Salvation . [ #SpirituallySpeaking ] idjory@icloud .", "followers_count": 7927, "friends_count": 7480, "statues_count": 10687 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Golden Glades, FL", "id": "dbdbdadee4fbdda9", "name": "Golden Glades", "place_type": "city", "bounding_box": rectangle("-80.229746,25.892265 -80.163078,25.929961") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1226375, "cityName": "Golden Glades" } }
+{ "create_at": datetime("2015-11-23T16:14:32.000Z"), "id": 668945761212567552, "text": "@ryleeevalynn nose goes! ����", "in_reply_to_status": 668945311860043780, "in_reply_to_user": 1646731280, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1646731280 }}, "user": { "id": 515790941, "name": "Sensei Jorge.", "screen_name": "whoreeeheyyy", "lang": "en", "location": "null", "create_at": date("2012-03-05"), "description": "Rylee Valyn Bates❤", "followers_count": 560, "friends_count": 407, "statues_count": 13609 }, "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("2015-11-23T16:14:32.000Z"), "id": 668945761300795392, "text": "Hoo hoo loosiers! I love it! I hope this is a really bad week in sports for them!! https://t.co/vSQs2dxAcY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 927990439, "name": "Matt Folk", "screen_name": "tgateking", "lang": "en", "location": "null", "create_at": date("2012-11-05"), "description": "null", "followers_count": 201, "friends_count": 723, "statues_count": 3238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westfield, IN", "id": "e4f5e6a715f4c8cd", "name": "Westfield", "place_type": "city", "bounding_box": rectangle("-86.183588,39.999226 -86.089631,40.072257") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18057, "countyName": "Hamilton", "cityID": 1882700, "cityName": "Westfield" } }
+{ "create_at": datetime("2015-11-23T16:14:32.000Z"), "id": 668945761300848640, "text": "@ADimeBack so why hasn 't the Gampel court been named for Jim Calhoun yet? Don 't want to hear it's because of Geno. #uconn #BleedBlue", "in_reply_to_status": 668941939723603968, "in_reply_to_user": 512018486, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "uconn", "BleedBlue" }}, "user_mentions": {{ 512018486 }}, "user": { "id": 3130107371, "name": "Tom Morache", "screen_name": "TomMorache", "lang": "en", "location": "null", "create_at": date("2015-03-30"), "description": "Mets-Uconn-Giants-Knicks-Rangers", "followers_count": 121, "friends_count": 511, "statues_count": 2343 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Meriden, CT", "id": "b5b8c8c3b9775063", "name": "Meriden", "place_type": "city", "bounding_box": rectangle("-72.859208,41.495568 -72.744752,41.578899") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 946450, "cityName": "Meriden" } }
+{ "create_at": datetime("2015-11-23T16:14:32.000Z"), "id": 668945761573277696, "text": "@Kteskey7 Probably not I'm leaving today to go see the Mormons", "in_reply_to_status": 668943831711154176, "in_reply_to_user": 1005117493, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1005117493 }}, "user": { "id": 805829461, "name": "Billy The Kid", "screen_name": "will_curless", "lang": "en", "location": "MPC Baseball", "create_at": date("2012-09-05"), "description": "Practice throwing hard to throw hard #stayhungry #LetItEat", "followers_count": 569, "friends_count": 592, "statues_count": 3503 }, "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.596818 -121.789746,36.654798") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 670742, "cityName": "Seaside" } }
+{ "create_at": datetime("2015-11-23T16:14:32.000Z"), "id": 668945761745399808, "text": "Visiting my daughter, Cheryl Hart with her boyfriend James Kent. (at @ConsulateHealth Care of St. Petersburg) https://t.co/DsL0yBU3Vb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.76900685,27.8400726"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2180408996 }}, "user": { "id": 514792075, "name": "Joan Greene Watkins", "screen_name": "GreeneJoan", "lang": "en", "location": "St. Petersburg FL", "create_at": date("2012-03-04"), "description": "married to Tracy Watkins, daughter Cheryl, son Marc, deceased son Scott, cat Baby, home health care RN, Tampa Bay Lightning STM", "followers_count": 268, "friends_count": 773, "statues_count": 2535 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seminole, FL", "id": "01686caa11f87b5e", "name": "Seminole", "place_type": "city", "bounding_box": rectangle("-82.81611,27.80979 -82.752081,27.884351") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1264975, "cityName": "Seminole" } }
+{ "create_at": datetime("2015-11-23T16:14:32.000Z"), "id": 668945761942437888, "text": "@xo_ashiaa you worked today?", "in_reply_to_status": -1, "in_reply_to_user": 294164156, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 294164156 }}, "user": { "id": 335646464, "name": "B.Brees", "screen_name": "lilbriana_", "lang": "en", "location": "La✈️HTX", "create_at": date("2011-07-14"), "description": "#ARMYSTRONG |F.O.E| ❤️ get you a BRI", "followers_count": 929, "friends_count": 739, "statues_count": 21101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:32.000Z"), "id": 668945762525429760, "text": "Join the The Cheesecake Factory team! See our latest #Hospitality #job opening here: https://t.co/nUoACLd2pD #Arcadia, CA #Veterans #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.0486188,34.1349511"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Arcadia", "Veterans", "Hiring" }}, "user": { "id": 88003804, "name": "TMJ-CA HRTA Jobs", "screen_name": "tmj_ca_hrta", "lang": "en", "location": "California", "create_at": date("2009-11-06"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 415, "friends_count": 297, "statues_count": 10993 }, "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("2015-11-23T16:14:32.000Z"), "id": 668945762940805120, "text": "I'm only going to this last class because I'm expecting food lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 564058625, "name": "Anthony Tillman", "screen_name": "A_Tillman3", "lang": "en", "location": "null", "create_at": date("2012-04-26"), "description": "If folks wanna pop off, dab on dem folks.", "followers_count": 545, "friends_count": 471, "statues_count": 37121 }, "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("2015-11-23T16:14:32.000Z"), "id": 668945763024674816, "text": "@Ruscle93 nevermind he won't be here.....", "in_reply_to_status": 668929969851080705, "in_reply_to_user": 363703584, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27177671 }}, "user": { "id": 363703584, "name": "Chris McCroskey", "screen_name": "ChrisMcCroskey_", "lang": "en", "location": "Anderson SC ", "create_at": date("2011-08-28"), "description": "Jesus Changed the Game.", "followers_count": 373, "friends_count": 253, "statues_count": 6848 }, "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("2015-11-23T16:14:32.000Z"), "id": 668945763116974081, "text": "When I die I want all the women to go to my funeral and talk about how I'm the \"one who got away ...from being their asexual buddy!\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21322778, "name": "Evan Morgenstern", "screen_name": "evanjm02", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2009-02-19"), "description": "Comedian", "followers_count": 1059, "friends_count": 2056, "statues_count": 13925 }, "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:33.000Z"), "id": 668945763204882432, "text": "why does this always happen 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": 1909925504, "name": "avery marie", "screen_name": "averygarza13", "lang": "en", "location": "null", "create_at": date("2013-09-26"), "description": "null", "followers_count": 725, "friends_count": 302, "statues_count": 18088 }, "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("2015-11-23T16:14:33.000Z"), "id": 668945763452358656, "text": "Can you recommend anyone for this #job? QC - https://t.co/RwTxWD6kVx #parttime #Bloomingdale, IL #Hospitality https://t.co/EPSeicP8x6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.112453,41.9391985"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "parttime", "Bloomingdale", "Hospitality" }}, "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": 140, "friends_count": 0, "statues_count": 12454 }, "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("2015-11-23T16:14:33.000Z"), "id": 668945763997777920, "text": "@MarinaCityEric that's unfortunate for them ��", "in_reply_to_status": 668940371611754496, "in_reply_to_user": 18929459, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18929459 }}, "user": { "id": 136736464, "name": "A$AP Sassy ✌️", "screen_name": "audrianaism", "lang": "en", "location": "Chicago l Atlanta l LA✈", "create_at": date("2010-04-24"), "description": "'So you like him and his dad?' 'Yeah, is that wrong?' Greg: I gotta stop doing bongs in my sleep", "followers_count": 830, "friends_count": 361, "statues_count": 64895 }, "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:33.000Z"), "id": 668945764102610945, "text": "ISS, NASA celebrate 15-year anniversary https://t.co/d0v92w4Upv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.03849792,29.57830048"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2874408405, "name": "Seabrook BubbleLife", "screen_name": "BLifeSeabrook", "lang": "en", "location": "Seabrook, Texas", "create_at": date("2014-11-12"), "description": "null", "followers_count": 1, "friends_count": 0, "statues_count": 6815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Lago, TX", "id": "c779f4b8a1a2bf1b", "name": "El Lago", "place_type": "city", "bounding_box": rectangle("-95.058923,29.562638 -95.033921,29.582408") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4823164, "cityName": "El Lago" } }
+{ "create_at": datetime("2015-11-23T16:14:33.000Z"), "id": 668945764169707520, "text": "Anddddd I hate IU basketball again.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 86838824, "name": "Jack Newby", "screen_name": "JNEWBY94", "lang": "en", "location": "B-Town/Indy ", "create_at": date("2009-11-01"), "description": "IU 17.", "followers_count": 252, "friends_count": 239, "statues_count": 2610 }, "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("2015-11-23T16:14:33.000Z"), "id": 668945764358316033, "text": "I fucking hate working sales/commission but I also hate sitting at a desk for 8 hours a day staring at the wall waiting for work be over..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 469851072, "name": "Jan ♔ Bilzerian", "screen_name": "wtfjvckiee", "lang": "en", "location": "lost angeles, ca", "create_at": date("2012-01-20"), "description": "ucla | cali grown | bimmer girl | meow | dilligaf | ♔ ♍ ✌ ☆ ☮ ❥", "followers_count": 1888, "friends_count": 1523, "statues_count": 5219 }, "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("2015-11-23T16:14:33.000Z"), "id": 668945764362637312, "text": "how does wake beat the #13 team in the country but get smacked by richmond 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": 707247208, "name": "cameron caroway", "screen_name": "cameroncaroway", "lang": "en", "location": "clemmons,nc", "create_at": date("2013-10-09"), "description": "2019", "followers_count": 505, "friends_count": 549, "statues_count": 2133 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clemmons, NC", "id": "290f62c6f654e14f", "name": "Clemmons", "place_type": "city", "bounding_box": rectangle("-80.432988,35.93042 -80.339893,36.078123") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37067, "countyName": "Forsyth", "cityID": 3712960, "cityName": "Clemmons" } }
+{ "create_at": datetime("2015-11-23T16:14:33.000Z"), "id": 668945764417011713, "text": "@KAITGOD go with the Tundra! Id like to have a truck but the payments and gas are was to much. Maybe someday", "in_reply_to_status": 668945249432006656, "in_reply_to_user": 580860398, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 580860398 }}, "user": { "id": 1398753212, "name": "sadly Connor", "screen_name": "muffinlord666", "lang": "en", "location": "In-n-out", "create_at": date("2013-05-02"), "description": "kings hockey since '05. I dont owe the world a god damn thing", "followers_count": 605, "friends_count": 473, "statues_count": 42951 }, "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("2015-11-23T16:14:33.000Z"), "id": 668945764647768064, "text": "@Fallout Why won't you let me romance Nick? Why Curie and not this lovable P.I. #cannickbemyvalentine?", "in_reply_to_status": 668927374864576517, "in_reply_to_user": 112511880, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "cannickbemyvalentine" }}, "user_mentions": {{ 112511880 }}, "user": { "id": 399190201, "name": "Kelsy Rice", "screen_name": "RogueIntentions", "lang": "en", "location": "SR-2 Normandy; Earth colony L2", "create_at": date("2011-10-26"), "description": "null", "followers_count": 40, "friends_count": 112, "statues_count": 139 }, "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("2015-11-23T16:14:33.000Z"), "id": 668945764647874560, "text": "Here's a scene featured in this week's Comet Over Hollywood Musical Monday. Read more:\nhttps://t.co/tT8U2WLRTk https://t.co/HDmDvSzwEj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 49195090, "name": "Jessica (J. P.)", "screen_name": "HollywoodComet", "lang": "en", "location": "Shelby, NC", "create_at": date("2009-06-20"), "description": "Classic film fan and owner of Comet Over Hollywood. Southerner, public relations for a healthcare system, former reporter. PR for @HerrmannMovie", "followers_count": 2418, "friends_count": 1458, "statues_count": 77022 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shelby, NC", "id": "5506172b34d4c9a3", "name": "Shelby", "place_type": "city", "bounding_box": rectangle("-81.611574,35.237295 -81.460676,35.347086") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37045, "countyName": "Cleveland", "cityID": 3761200, "cityName": "Shelby" } }
+{ "create_at": datetime("2015-11-23T16:14:33.000Z"), "id": 668945764844982272, "text": "When you on weight but can't alpha until Wednesday, I'm salty 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": 3247575435, "name": "Devonte", "screen_name": "MidDistanceDevo", "lang": "en", "location": "null", "create_at": date("2015-05-12"), "description": "I run track and wrestle. Future Marine", "followers_count": 350, "friends_count": 202, "statues_count": 3122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Niles, MI", "id": "01d195f82afb41f9", "name": "Niles", "place_type": "city", "bounding_box": rectangle("-86.317101,41.80097 -86.219488,41.884575") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26021, "countyName": "Berrien", "cityID": 2657760, "cityName": "Niles" } }
+{ "create_at": datetime("2015-11-23T16:14:33.000Z"), "id": 668945764962406401, "text": "My niece Taya which means \"perfectly formed\" or \"princess\" I have to agree. ������✨ https://t.co/PW7wSbUK9P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 64644975, "name": "Chelsea", "screen_name": "chelseakboyd", "lang": "en", "location": "null", "create_at": date("2009-08-10"), "description": "null", "followers_count": 841, "friends_count": 423, "statues_count": 19855 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Canal Fulton, OH", "id": "01806ebe439e92dd", "name": "Canal Fulton", "place_type": "city", "bounding_box": rectangle("-81.623096,40.842646 -81.536917,40.91569") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39151, "countyName": "Stark", "cityID": 3911304, "cityName": "Canal Fulton" } }
+{ "create_at": datetime("2015-11-23T16:14:33.000Z"), "id": 668945764962476032, "text": "Hello, Winter https://t.co/aowe9LLoXM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18823424, "name": "MariaXimenaDC", "screen_name": "MARIAXIMENA", "lang": "es", "location": "Washington DC", "create_at": date("2009-01-09"), "description": "Speaker of the House", "followers_count": 748, "friends_count": 715, "statues_count": 8875 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Bethesda, MD", "id": "d3c4d514e13810f3", "name": "North Bethesda", "place_type": "city", "bounding_box": rectangle("-77.155886,39.012634 -77.084619,39.074304") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2456337, "cityName": "North Bethesda" } }
+{ "create_at": datetime("2015-11-23T16:14:33.000Z"), "id": 668945765138493440, "text": "Hey stay up! https://t.co/basoIsThmc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1487213899, "name": "Moleboy", "screen_name": "raulmendeezy", "lang": "en", "location": "Gardena, Cali", "create_at": date("2013-06-06"), "description": "Vine: Raul Mendez Instagram: raul.santi Snapchat: raraul", "followers_count": 275, "friends_count": 433, "statues_count": 1044 }, "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("2015-11-23T16:14:33.000Z"), "id": 668945765151219712, "text": "I'm sobbing https://t.co/181x3BmPC2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 200989337, "name": "champagne auntie", "screen_name": "coolandgoth", "lang": "en", "location": "hell", "create_at": date("2010-10-10"), "description": "Sam/Balls. just gonna be talking about Drake, cows, models, or how shitty my life is instagram: sam.balls", "followers_count": 411, "friends_count": 525, "statues_count": 28897 }, "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("2015-11-23T16:14:33.000Z"), "id": 668945765339762688, "text": "When you show up to work 2 and a half hours early...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 718175318, "name": "Anna Hurley", "screen_name": "annahurley95", "lang": "en", "location": "null", "create_at": date("2012-07-26"), "description": "null", "followers_count": 465, "friends_count": 539, "statues_count": 1814 }, "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("2015-11-23T16:14:33.000Z"), "id": 668945765478236160, "text": "Woah. @CCChronicle was cut in half. Very sad to see as former editor. https://t.co/42czhvCuZU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17095421 }}, "user": { "id": 18790820, "name": "Mina Bloom", "screen_name": "mina_bloom_", "lang": "en", "location": "Chicago, IL", "create_at": date("2009-01-08"), "description": "@DNAinfochi reporter - Lincoln Park, Old Town and Near North. Tips: mbloom@dnainfo.com/847-370-9735. RTs ≠ Es. My wu-tang clan/hello kitty shirt says it all.", "followers_count": 923, "friends_count": 745, "statues_count": 3252 }, "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:33.000Z"), "id": 668945765511770112, "text": "@SkyFather_ true and it also depends on what you want to do over there and I want to do a lot lol but good luck on planning your trip!", "in_reply_to_status": 668945172579815424, "in_reply_to_user": 220587523, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 220587523 }}, "user": { "id": 828511777, "name": "Kim εїз", "screen_name": "k_i_m_______", "lang": "en", "location": "Los Angeles", "create_at": date("2012-09-16"), "description": "Goals not dreams, goals. Happy with the love of my life. 081712☀️", "followers_count": 377, "friends_count": 324, "statues_count": 15360 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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:33.000Z"), "id": 668945765545324545, "text": "@Annex_DeMo Brackets have been tweeted out. #MESGears", "in_reply_to_status": 668944645867487233, "in_reply_to_user": 544172532, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MESGears" }}, "user_mentions": {{ 544172532 }}, "user": { "id": 2188896816, "name": "MES", "screen_name": "MESEvents", "lang": "en", "location": "Michigan ", "create_at": date("2013-11-11"), "description": "- North America's Fastest Growing LAN Provider - Online Tournaments: @MESLiveTV - Partners: @CinchGaming, @DXRacer, @JerkyXP, @GamerSaloon, @GamingCases", "followers_count": 11924, "friends_count": 1325, "statues_count": 22056 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burton, MI", "id": "d4f8c782da0a3b7b", "name": "Burton", "place_type": "city", "bounding_box": rectangle("-83.692863,42.959625 -83.573753,43.048152") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2612060, "cityName": "Burton" } }
+{ "create_at": datetime("2015-11-23T16:14:33.000Z"), "id": 668945765641773056, "text": "And if you got something stupid to say about it you can get hit wit that one sword style 36 caliber phoenix", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 98424110, "name": "London4x™", "screen_name": "_ImTheAnswer", "lang": "en", "location": "G'z Up Hoes Down", "create_at": date("2009-12-21"), "description": "Limitations only exist If you let them|Those who don't know true pain can never know true peace|400 Million Bounty On My Head|Leader of a group of savages", "followers_count": 3311, "friends_count": 2449, "statues_count": 111817 }, "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("2015-11-23T16:14:33.000Z"), "id": 668945765751005184, "text": "@javier2flycx @misterjewan @dopemexican69 @feoboi @Emmanuel414K @Asvp_Butter ���������� https://t.co/ZhfaWjJv8f", "in_reply_to_status": -1, "in_reply_to_user": 2549195330, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2549195330, 3146934738, 2610650935, 2265546175, 3161794648, 268964754 }}, "user": { "id": 1210430858, "name": "kun", "screen_name": "SELASOR7", "lang": "en", "location": "madrid, manchester", "create_at": date("2013-02-22"), "description": "atletico and man citayy", "followers_count": 235, "friends_count": 208, "statues_count": 4716 }, "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("2015-11-23T16:14:33.000Z"), "id": 668945765943934976, "text": "Pretty brown skin , long blonde hair ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1644484260, "name": "Swaggy Zay❗️", "screen_name": "Jose_TmT_", "lang": "en", "location": "patterson highschool gymnasium", "create_at": date("2013-08-03"), "description": "#splurgegang5eva x #steeeeeazyy patterson varsity athlete", "followers_count": 1386, "friends_count": 727, "statues_count": 43790 }, "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:33.000Z"), "id": 668945766082310144, "text": "Cute DIY Christmas ideas are my favorite and always meNs the most", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.31763483,41.22590169"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2340980586, "name": "T-cheap", "screen_name": "torijaidynjones", "lang": "en", "location": "osceola IN", "create_at": date("2014-02-12"), "description": "Penn Girls Rugby. Bells Speed Club. \nim the laziest most active person I know", "followers_count": 173, "friends_count": 95, "statues_count": 1306 }, "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": 18003, "countyName": "Allen", "cityID": 1812610, "cityName": "Churubusco" } }
+{ "create_at": datetime("2015-11-23T16:14:33.000Z"), "id": 668945766136827904, "text": "@Josh_Wood40 haha you do you", "in_reply_to_status": 668945561253449728, "in_reply_to_user": 2779632374, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2779632374 }}, "user": { "id": 72144330, "name": "Karli Bunch", "screen_name": "karlibunch", "lang": "en", "location": "California University of PA", "create_at": date("2009-09-06"), "description": "surround me w/ pugs & french bulldogs #feelthebern2016", "followers_count": 309, "friends_count": 522, "statues_count": 8279 }, "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("2015-11-23T16:14:33.000Z"), "id": 668945766161846272, "text": "We're #hiring! Read about our latest #job opening here: Business Intelligence Developer - https://t.co/BR5862Gm5t #Denver, CO #IT", "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", "IT" }}, "user": { "id": 3015856778, "name": "Fly Frontier Jobs", "screen_name": "FlyFrontierJobs", "lang": "en", "location": "null", "create_at": date("2015-02-09"), "description": "We offer service to more than 75 destinations in the US, Mexico, Costa Rica, & the Dominican Republic. Check here for current openings.", "followers_count": 156, "friends_count": 498, "statues_count": 58 }, "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:33.000Z"), "id": 668945766233169921, "text": "In the last 2 weeks I've gotten 8 phone calls, 4 emails, and a text from my daughter's school or @KCMOSchools. Messaging overload.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 38002196 }}, "user": { "id": 29246704, "name": "Matt Nugent", "screen_name": "MattNuge", "lang": "en", "location": "Squier Park | Kanscity", "create_at": date("2009-04-06"), "description": "Someone once told me that the perfect is the enemy of the good. Tweets and media are my own, obvs.", "followers_count": 512, "friends_count": 656, "statues_count": 12926 }, "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("2015-11-23T16:14:33.000Z"), "id": 668945766313009155, "text": "Being alt in the NYU politics dept means liking Hillary and using a PC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16228175, "name": "Nick", "screen_name": "nick_bailey", "lang": "en", "location": "New York City", "create_at": date("2008-09-10"), "description": "I don't care, just get me a quote.", "followers_count": 275, "friends_count": 258, "statues_count": 17002 }, "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("2015-11-23T16:14:33.000Z"), "id": 668945766329786368, "text": "Feel like I'm dying", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 295458084, "name": "SMASH WILLIAMS", "screen_name": "carson_willie", "lang": "en", "location": "null", "create_at": date("2011-05-08"), "description": "hi im carson i enjoy long walks on the beach", "followers_count": 644, "friends_count": 489, "statues_count": 23511 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hilltop, OH", "id": "012804cc7b82ba2c", "name": "Hilltop", "place_type": "city", "bounding_box": rectangle("-80.768834,41.147302 -80.732769,41.170461") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39155, "countyName": "Trumbull", "cityID": 3935613, "cityName": "Hilltop" } }
+{ "create_at": datetime("2015-11-23T16:14:33.000Z"), "id": 668945766543568897, "text": "@millymoosthecat I'm definitely giving this test to my dog ��", "in_reply_to_status": 668392752204414976, "in_reply_to_user": 3870828688, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3870828688 }}, "user": { "id": 2342637824, "name": "Kathryn Dean", "screen_name": "RealKathrynDean", "lang": "en", "location": "Los Angeles", "create_at": date("2014-02-13"), "description": "My album #HitTheLights is available now! Listen, enjoy & share. http://apple.co/1HUt9nW ❤️ #KathrynDean #iTunes #Spotify #Malhacao #Music #ToldYouSo", "followers_count": 21478, "friends_count": 8215, "statues_count": 2231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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:33.000Z"), "id": 668945766614888448, "text": "@Only_Swarren gryffindor", "in_reply_to_status": 668944994258972672, "in_reply_to_user": 2392081225, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2392081225 }}, "user": { "id": 2257449158, "name": "aaliyah .", "screen_name": "liyahmcnealy", "lang": "en", "location": "null", "create_at": date("2013-12-22"), "description": "what a treacherous thing it is to believe a person is more than a person", "followers_count": 268, "friends_count": 249, "statues_count": 1965 }, "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:33.000Z"), "id": 668945766631641088, "text": "@danikilburn I hate you", "in_reply_to_status": 668945692996440064, "in_reply_to_user": 2705566537, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2705566537 }}, "user": { "id": 715178191, "name": "Taylor Hays", "screen_name": "taylorhays_", "lang": "en", "location": "STL, MO • PHS 2017", "create_at": date("2012-07-24"), "description": "Great ambition is the passion of a great character.", "followers_count": 505, "friends_count": 412, "statues_count": 18895 }, "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("2015-11-23T16:14:33.000Z"), "id": 668945766828773377, "text": "Good music & twilight drives", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1363268036, "name": "ashlyn", "screen_name": "ashitudeprobs", "lang": "en", "location": "null", "create_at": date("2013-04-18"), "description": "love the universe & it will love you back", "followers_count": 254, "friends_count": 232, "statues_count": 2865 }, "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("2015-11-23T16:14:33.000Z"), "id": 668945766908608512, "text": "Mom and dads new pup they're getting tomorrow. This picture makes me laugh so hard lol https://t.co/yWo7oNBVh1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 60803067, "name": "Pody", "screen_name": "PodyCotter", "lang": "en", "location": "Delaware, USA", "create_at": date("2009-07-27"), "description": "ohio is home", "followers_count": 597, "friends_count": 219, "statues_count": 32941 }, "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("2015-11-23T16:14:33.000Z"), "id": 668945766933622784, "text": "@jmecandice https://t.co/KNvSjtanLR", "in_reply_to_status": -1, "in_reply_to_user": 53835799, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 53835799 }}, "user": { "id": 245610728, "name": "Haydin Buterbaugh", "screen_name": "haydin_nicole", "lang": "en", "location": "Anywhere and everywhere", "create_at": date("2011-01-31"), "description": "The official Twitter account for all things Haydin. Enjoy.", "followers_count": 183, "friends_count": 214, "statues_count": 5313 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Purcell, OK", "id": "1f9f17a2260b4ff8", "name": "Purcell", "place_type": "city", "bounding_box": rectangle("-97.395705,34.971543 -97.347925,35.058867") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40087, "countyName": "McClain", "cityID": 4061150, "cityName": "Purcell" } }
+{ "create_at": datetime("2015-11-23T16:14:33.000Z"), "id": 668945767185313793, "text": "@NJDavidD and it’s not just you, it’s others-", "in_reply_to_status": 668944959689646080, "in_reply_to_user": 101582811, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 101582811 }}, "user": { "id": 550091576, "name": "Mike Brandes, CTS", "screen_name": "MikeBrandesAV", "lang": "en", "location": "Minneapolis, MN", "create_at": date("2012-04-10"), "description": "A/V, IT and other thoughts. Freelance Writer. Consultant. Former scratch golfer and alligator trapper", "followers_count": 937, "friends_count": 142, "statues_count": 7224 }, "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("2015-11-23T16:14:33.000Z"), "id": 668945767311286272, "text": "@Homework_Fairy yea no one really gives a shit about u to be honest people tend to address stupidity when they can", "in_reply_to_status": 668945286710980608, "in_reply_to_user": 238811523, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 238811523 }}, "user": { "id": 1525575896, "name": "camille", "screen_name": "CamilleZzoe", "lang": "en", "location": "nhs", "create_at": date("2013-06-17"), "description": "slumped", "followers_count": 288, "friends_count": 167, "statues_count": 6428 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Noblesville, IN", "id": "013e3bc05a18abdc", "name": "Noblesville", "place_type": "city", "bounding_box": rectangle("-86.090354,39.993436 -85.932944,40.10716") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18057, "countyName": "Hamilton", "cityID": 1854180, "cityName": "Noblesville" } }
+{ "create_at": datetime("2015-11-23T16:14:34.000Z"), "id": 668945767600640001, "text": "I can't wait to go to bed tonight ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2615716805, "name": "мιкαуℓα ☠", "screen_name": "mikaylafreeman4", "lang": "en", "location": "Sulphur, OK", "create_at": date("2014-06-16"), "description": "I'm hungry and I just ate. straight outta Azkaban ⚡️", "followers_count": 213, "friends_count": 308, "statues_count": 4402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sulphur, OK", "id": "7fe71783bb4b2eed", "name": "Sulphur", "place_type": "city", "bounding_box": rectangle("-96.998351,34.491603 -96.949858,34.526851") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40099, "countyName": "Murray", "cityID": 4071350, "cityName": "Sulphur" } }
+{ "create_at": datetime("2015-11-23T16:14:34.000Z"), "id": 668945767613136896, "text": "Out if surgery 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": 3048011623, "name": "mikayla.", "screen_name": "90sgothmom", "lang": "en", "location": "Pullman, WA", "create_at": date("2015-02-27"), "description": "My favorite character from The Office is Angela.", "followers_count": 119, "friends_count": 164, "statues_count": 617 }, "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("2015-11-23T16:14:34.000Z"), "id": 668945767713783808, "text": "He loves mamas chilli! ❤️���� https://t.co/OeWat0AciU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 262945991, "name": "Amanda..✌️", "screen_name": "AmandaWilke14", "lang": "en", "location": "cornfield town,iowa", "create_at": date("2011-03-08"), "description": "//22 years•mommy of two•married•CNA•Smalltown iowa girl•music•family•friends•God• ❤️\\\\", "followers_count": 599, "friends_count": 1368, "statues_count": 3827 }, "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("2015-11-23T16:14:34.000Z"), "id": 668945767743270913, "text": "@Ljpeak0 what's good big man ?", "in_reply_to_status": -1, "in_reply_to_user": 3087882468, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3087882468 }}, "user": { "id": 2623174209, "name": "K2⃣", "screen_name": "KobeRodgers2", "lang": "en", "location": "Gaffney,Sc", "create_at": date("2014-06-19"), "description": "If You're Doubting Me , You're Doubting God's Plans Morgan Whitley ❤️", "followers_count": 923, "friends_count": 910, "statues_count": 4512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gaffney, SC", "id": "a3b5533ca11e0324", "name": "Gaffney", "place_type": "city", "bounding_box": rectangle("-81.719658,35.03446 -81.599161,35.172674") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45021, "countyName": "Cherokee", "cityID": 4528060, "cityName": "Gaffney" } }
+{ "create_at": datetime("2015-11-23T16:14:34.000Z"), "id": 668945767747354624, "text": "We're #hiring! Click to apply: Product Manager, storage - https://t.co/abLsZWmR1k #Engineering #Intel #SantaClara, California #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.9552356,37.3541079"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Engineering", "Intel", "SantaClara", "Job", "Jobs" }}, "user": { "id": 23363279, "name": "San Jose Eng. Jobs", "screen_name": "tmj_sjc_eng", "lang": "en", "location": "San Jose, CA", "create_at": date("2009-03-08"), "description": "Follow this account for geo-targeted Engineering job tweets in San Jose, CA. Need help? Tweet us at @CareerArc!", "followers_count": 494, "friends_count": 245, "statues_count": 529 }, "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("2015-11-23T16:14:34.000Z"), "id": 668945767864758273, "text": "#oldschoolers @ Carroll Gardens, Brooklyn https://t.co/IIqbXZBvCA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.99869019,40.68010796"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "oldschoolers" }}, "user": { "id": 14097162, "name": "Jim", "screen_name": "bowlerjim", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2008-03-07"), "description": "CEO of Livin'", "followers_count": 509, "friends_count": 781, "statues_count": 5237 }, "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:34.000Z"), "id": 668945767873179649, "text": "@IamAkademiks aye what was the song by 50 cent that you said you listend to because it was real gangsta rap?", "in_reply_to_status": -1, "in_reply_to_user": 32501175, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 32501175 }}, "user": { "id": 1592964252, "name": "Ray", "screen_name": "rayp2000", "lang": "en", "location": "under your bed", "create_at": date("2013-07-14"), "description": "tryna leave andress rn", "followers_count": 217, "friends_count": 219, "statues_count": 3809 }, "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("2015-11-23T16:14:34.000Z"), "id": 668945767906844674, "text": "A Moment of Surrender...\nThis Feels Dangerous...\nA Beautiful Sacrifice...\n#newcandles… https://t.co/gwcjfVy4rZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.40271,34.0666809"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "newcandles" }}, "user": { "id": 456282592, "name": "12.29", "screen_name": "1229journal", "lang": "en", "location": "Paris | New York", "create_at": date("2012-01-05"), "description": "12.29 is the premier olfactive branding company that designs custom scents for luxury brands around the world. #onlyscentremains", "followers_count": 554, "friends_count": 409, "statues_count": 1482 }, "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("2015-11-23T16:14:34.000Z"), "id": 668945768116563968, "text": "tsemyworld is literally a genius loves these pictures soo much������ #tiffanyisperf #la #smokedancing @… https://t.co/GaLqBH9Vhm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.27459679,34.13487136"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "tiffanyisperf", "la", "smokedancing" }}, "user": { "id": 736277312, "name": "smiles☺️", "screen_name": "saraa_llanos", "lang": "en", "location": "Pomona, CA", "create_at": date("2012-08-04"), "description": "Talent Factory Teacher & Choreographer| MtSac Dance Team| sarallanos95@gmail.com", "followers_count": 425, "friends_count": 390, "statues_count": 9915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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:34.000Z"), "id": 668945768200318976, "text": "NASH GRIER IS SO UGLY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2567725806, "name": "plastic demi", "screen_name": "demidiamandis", "lang": "en", "location": "null", "create_at": date("2014-06-14"), "description": "you caught me winking now my winking days are over!", "followers_count": 155, "friends_count": 235, "statues_count": 1809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:34.000Z"), "id": 668945768477233152, "text": "Come and put your name 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": 152046982, "name": "Brandon☯", "screen_name": "brandonfsu_", "lang": "en", "location": "bow down to the bullet nation ", "create_at": date("2010-06-04"), "description": "be confident • california allstars", "followers_count": 1441, "friends_count": 950, "statues_count": 83950 }, "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("2015-11-23T16:14:34.000Z"), "id": 668945768489684992, "text": "What do I want 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": 2869346986, "name": "• m a d s •", "screen_name": "madisonpaaigee", "lang": "en", "location": "w chey", "create_at": date("2014-11-09"), "description": "@tristencannonn_", "followers_count": 435, "friends_count": 384, "statues_count": 13426 }, "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("2015-11-23T16:14:34.000Z"), "id": 668945768498094081, "text": "LIKE U MEAN IT || Hanging with @KelseaBallerini at her #LA sound check! #KelseaBallerini #country #music https://t.co/o9AETnjNgy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LA", "KelseaBallerini", "country", "music" }}, "user_mentions": {{ 44794040 }}, "user": { "id": 16185787, "name": "sweetyhigh", "screen_name": "sweetyhigh", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2008-09-08"), "description": "We are a site where girls can express their creativity and we produce our own shows! #SweetBeatTV || \n\nFollow us on Instagram & Add us on Snapchat! @sweetyhigh", "followers_count": 373019, "friends_count": 822, "statues_count": 19320 }, "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("2015-11-23T16:14:34.000Z"), "id": 668945768779239425, "text": "damn right�� https://t.co/wLybvs7UWc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3370259002, "name": "Cj", "screen_name": "ceeejayy1", "lang": "en", "location": "null", "create_at": date("2015-07-10"), "description": "null", "followers_count": 213, "friends_count": 142, "statues_count": 4160 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Williamston, NC", "id": "24c0c57bd6f498bd", "name": "Williamston", "place_type": "city", "bounding_box": rectangle("-77.107345,35.832515 -77.036503,35.86699") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37117, "countyName": "Martin", "cityID": 3774220, "cityName": "Williamston" } }
+{ "create_at": datetime("2015-11-23T16:14:34.000Z"), "id": 668945768812666880, "text": "Can you recommend anyone for this #SkilledTrade #job? https://t.co/ASP9m5l6Q2 #Helena, MT #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.0270306,46.5958056"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SkilledTrade", "job", "Helena", "Hiring", "CareerArc" }}, "user": { "id": 119400199, "name": "TMJ-MT Skl. Trades", "screen_name": "tmj_MT_skltrd", "lang": "en", "location": "Montana", "create_at": date("2010-03-03"), "description": "Follow this account for geo-targeted Skilled Trade job tweets in Montana Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 294, "friends_count": 272, "statues_count": 15 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Helena, MT", "id": "9c7ce7eed4cb3dee", "name": "Helena", "place_type": "city", "bounding_box": rectangle("-112.106268,46.568875 -111.935822,46.626111") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30049, "countyName": "Lewis and Clark", "cityID": 3035600, "cityName": "Helena" } }
+{ "create_at": datetime("2015-11-23T16:14:34.000Z"), "id": 668945768892510208, "text": "@RishyCupTV I'm doing well, I can't complain. ������. What's new?", "in_reply_to_status": 668945617352269824, "in_reply_to_user": 3145364439, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3145364439 }}, "user": { "id": 2427191110, "name": "Herbie", "screen_name": "herbiemerrin", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2014-03-21"), "description": "Lyft Driver in New Jersey.\n#Love #Loyalty #Longevity #Legacy...\n#GratitudeCard", "followers_count": 1754, "friends_count": 658, "statues_count": 26867 }, "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:34.000Z"), "id": 668945769093664768, "text": "RT https://t.co/7URMybBah5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 739368733, "name": "Megs❥", "screen_name": "Megan_henninger", "lang": "en", "location": "null", "create_at": date("2012-08-05"), "description": "its always our favorite sins that do us in", "followers_count": 91, "friends_count": 208, "statues_count": 614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:34.000Z"), "id": 668945769634897920, "text": "So many people say well I'm not worried about it bcz I'm not in that position, food for thought what… https://t.co/SDj6oMksk0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.8746033,37.3164787"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48071219, "name": "Chef Jimbo Carrol", "screen_name": "jimbocarrol", "lang": "en", "location": "Santa Clara, California", "create_at": date("2009-06-17"), "description": "Animated Chef, Your Personal Culinary Super Hero, Global Traveler, Life Coach, Online/Offline marketing consultant, Scuba instructor, Guidance/Personal Coach", "followers_count": 1627, "friends_count": 2111, "statues_count": 1474 }, "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:34.000Z"), "id": 668945769962020864, "text": "SC✈️NY #845bound", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "845bound" }}, "user": { "id": 271141943, "name": "Young Sling", "screen_name": "Cheddar_Rivv", "lang": "en", "location": "NY|SC", "create_at": date("2011-03-23"), "description": "Luke1:37 ''For Nothing is Impossible With God'' 'Limestone '17 ⚾️ #Natitude #Uhhyuu 'TRF'", "followers_count": 585, "friends_count": 839, "statues_count": 18964 }, "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("2015-11-23T16:14:34.000Z"), "id": 668945769995472897, "text": "@ChrisSchanzer It's a trap! :)", "in_reply_to_status": 668944391713656832, "in_reply_to_user": 81771140, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 81771140 }}, "user": { "id": 3075601, "name": "Allen Nemo", "screen_name": "mr_modular", "lang": "en", "location": "Austin, TX", "create_at": date("2007-03-31"), "description": "Full Time Dilettante", "followers_count": 292, "friends_count": 326, "statues_count": 6940 }, "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:34.000Z"), "id": 668945770310045696, "text": "@briannaarama you don't text me how would you know ?", "in_reply_to_status": 668945170176475136, "in_reply_to_user": 52602004, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 52602004 }}, "user": { "id": 47151236, "name": "❕Shemmy❕", "screen_name": "LittleBigShemmy", "lang": "en", "location": "West Tyler ✈️San Antonio", "create_at": date("2009-06-14"), "description": "Overdo$e of $auce", "followers_count": 3160, "friends_count": 1482, "statues_count": 145388 }, "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:34.000Z"), "id": 668945770343567362, "text": "Let Us Give Thanks To The Lord!\n-\nWe would love for you to join us for our Thanksgiving service this… https://t.co/UU5L5aqnPO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.83365044,33.63058741"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 49178092, "name": "MarinersChurch", "screen_name": "marinerschurch", "lang": "en", "location": "Irvine, Ca", "create_at": date("2009-06-20"), "description": "Welcome to Mariners Church! We would like to wish you and you're family a wonderful Thanksgiving!Sat 5p, Sun 9 & 11a or 7p in the Community Center.", "followers_count": 3268, "friends_count": 68, "statues_count": 2802 }, "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("2015-11-23T16:14:34.000Z"), "id": 668945770448420865, "text": "@mirandahmoney cool. Thought we might need to get you on the drug test list for December! ��������", "in_reply_to_status": 668945248706387968, "in_reply_to_user": 1199333276, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1199333276 }}, "user": { "id": 432700663, "name": "chris southard", "screen_name": "csouthard96", "lang": "en", "location": "Frisco, TX", "create_at": date("2011-12-09"), "description": "You'll never make it to the bigs with fungus on your shower shoes!", "followers_count": 596, "friends_count": 1985, "statues_count": 4909 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Elm, TX", "id": "bd331d141f66eead", "name": "Little Elm", "place_type": "city", "bounding_box": rectangle("-96.983818,33.140885 -96.881861,33.199847") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4843012, "cityName": "Little Elm" } }
+{ "create_at": datetime("2015-11-23T16:14:34.000Z"), "id": 668945770477850624, "text": "Holy crap 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": 3164831329, "name": "Christian", "screen_name": "2gchristian", "lang": "en", "location": "H", "create_at": date("2015-04-19"), "description": "null", "followers_count": 86, "friends_count": 329, "statues_count": 3977 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:34.000Z"), "id": 668945770624626688, "text": "DJ Khaled is fuckin ridiculous. He's getting out of hand lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1515564157, "name": "jumpman", "screen_name": "akadeathwish", "lang": "en", "location": "Bedford, TX", "create_at": date("2013-06-13"), "description": "Mayson | 20 | TX | IG: akadeathwish | water, women and weed", "followers_count": 1056, "friends_count": 417, "statues_count": 62532 }, "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("2015-11-23T16:14:34.000Z"), "id": 668945770645622784, "text": "atleast you gotta she �� https://t.co/dukNUQ9j13", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2764904317, "name": "☁️", "screen_name": "coreyizpolo", "lang": "en", "location": "octobers very own", "create_at": date("2014-08-24"), "description": "15. idk. idc.", "followers_count": 809, "friends_count": 229, "statues_count": 30446 }, "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("2015-11-23T16:14:34.000Z"), "id": 668945770863661056, "text": "We're #hiring! Click to apply: Material Handler - https://t.co/GngbSqX3nL #Manufacturing #Modesto, CA #Veterans #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-120.9968782,37.6390972"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Manufacturing", "Modesto", "Veterans", "Job", "Jobs", "CareerArc" }}, "user": { "id": 395498959, "name": "CA Veteran Jobs", "screen_name": "tmj_CA_vets", "lang": "en", "location": "California Non-Metro", "create_at": date("2011-10-21"), "description": "Follow this account for geo-targeted Veteran job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 191, "friends_count": 127, "statues_count": 498 }, "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("2015-11-23T16:14:34.000Z"), "id": 668945770884808704, "text": "@kenzieibd Ugh. There's a sentence I've never uttered. (I'm not a Windows guy.)", "in_reply_to_status": 668945231224508416, "in_reply_to_user": 2608753447, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2608753447 }}, "user": { "id": 2945994771, "name": "Rob", "screen_name": "RobW521", "lang": "en", "location": "Philadelphia", "create_at": date("2014-12-28"), "description": "Husband, father, & tortured fan of all 4 Philly sports teams. Love music--esp blues. How else do I survive Philly sports? Also a big fan of Philly's @muralarts", "followers_count": 110, "friends_count": 85, "statues_count": 9665 }, "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:34.000Z"), "id": 668945771081932800, "text": "Let's do a Q&A here! Ask me!!! #NoXcuses @ The Setai https://t.co/xzPPh3DLh7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.12798117,25.79598723"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NoXcuses" }}, "user": { "id": 20906609, "name": "EDX Maurizio Colella", "screen_name": "edxMusic", "lang": "en", "location": "Zürich/Miami", "create_at": date("2009-02-15"), "description": "Is it any wonder dance music bible Mixmag hailed EDX as a “GENIUS”. Member of the Helvetic Nerds...", "followers_count": 99467, "friends_count": 403, "statues_count": 12657 }, "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:34.000Z"), "id": 668945771190964224, "text": "Words have never resonated like these one right here ���� https://t.co/TzuTGd9Phz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 151048212, "name": "Snapchat: kwazinkomo", "screen_name": "Kwazi_Nkomo", "lang": "en", "location": "Johannesburg", "create_at": date("2010-06-02"), "description": "Optimist. I am because you are.", "followers_count": 123, "friends_count": 91, "statues_count": 2291 }, "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("2015-11-23T16:14:34.000Z"), "id": 668945771245342720, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1628889524, "name": "Mattie Holly", "screen_name": "mattisonholly", "lang": "en", "location": "null", "create_at": date("2013-07-28"), "description": "HH//CC", "followers_count": 772, "friends_count": 572, "statues_count": 1936 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945771526418433, "text": "its so cute when my little puppy growls at things awh you think your intimidating", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 298676783, "name": "ugly", "screen_name": "adensullivan", "lang": "en", "location": "Texas, MI", "create_at": date("2011-05-14"), "description": "young street sweeper flexin and finnessin since 98", "followers_count": 180, "friends_count": 177, "statues_count": 3526 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945771635408896, "text": "Want to work at Norterra Dental Group and Orthod...? We're #hiring in #Phoenix, AZ! Click for details: https://t.co/SNZmnDJxLJ #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.0740373,33.4483771"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Phoenix", "Job", "Jobs" }}, "user": { "id": 38023469, "name": "TMJ-PHX Jobs", "screen_name": "tmj_phx_jobs", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2009-05-05"), "description": "Follow this account for geo-targeted Other job tweets in Phoenix, AZ from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 639, "friends_count": 540, "statues_count": 409 }, "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:35.000Z"), "id": 668945771648143360, "text": "MFs deleting tweets..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 460597982, "name": "IG: RoyDatBoy™", "screen_name": "Da_RealRoy", "lang": "en", "location": "Out the way", "create_at": date("2012-01-10"), "description": "Too busy doing me to worry bout yall.", "followers_count": 961, "friends_count": 830, "statues_count": 20106 }, "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:35.000Z"), "id": 668945771790635008, "text": "We're #hiring! Read about our latest #job opening here: Sr Technical Account Manager - Symantec - https://t.co/Jzrn1Zc0Vi #Lindon, Utah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.7207608,40.3432857"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Lindon" }}, "user": { "id": 72053789, "name": "UT Cust. Srv. Jobs", "screen_name": "tmj_UT_cstsrv", "lang": "en", "location": "Utah", "create_at": date("2009-09-06"), "description": "Follow this account for geo-targeted Customer Service job tweets in Utah Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 277, "friends_count": 269, "statues_count": 30 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945771866157056, "text": "When them \"I got time\" excuses finally catch up with you and now you got 4 essays and 5 exams due in 2 days https://t.co/llZTYB12eu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 272226903, "name": "A$AP HAM", "screen_name": "WoahderJamon", "lang": "en", "location": "Souf Arlington, Texas. ", "create_at": date("2011-03-25"), "description": "Jamon(JAY-mon). 20. Groovy Nigga.", "followers_count": 1449, "friends_count": 1034, "statues_count": 69284 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945772080050176, "text": "True RT @iDntWearCondoms: The real relationship doesn't start until Yall break up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.666124,30.202293"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 139747863 }}, "user": { "id": 1371679909, "name": "Sergio", "screen_name": "BerlangaS_", "lang": "en", "location": "Austin/Brownsville, TX", "create_at": date("2013-04-22"), "description": "UT Brownsville graduate '14", "followers_count": 494, "friends_count": 381, "statues_count": 5159 }, "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:35.000Z"), "id": 668945772197494784, "text": "Intel Corporation: Electrical Validation Engineer (#Phoenix, Arizona) https://t.co/AzvIZmbOwm #Engineering #Intel #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.0740373,33.4483771"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "Phoenix", "Engineering", "Intel", "Job", "Jobs", "Hiring" }}, "user": { "id": 23357261, "name": "TMJ-PHX Engin. Jobs", "screen_name": "tmj_phx_eng", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2009-03-08"), "description": "Follow this account for geo-targeted Engineering job tweets in Phoenix, AZ from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 345, "friends_count": 229, "statues_count": 222 }, "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:35.000Z"), "id": 668945772482818049, "text": "It's TOO fucking cold ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36696973, "name": "Samantha Harper", "screen_name": "SamLePearl", "lang": "en", "location": "Wherever My Dreams Take Me ", "create_at": date("2009-04-30"), "description": "null", "followers_count": 748, "friends_count": 664, "statues_count": 7690 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945772650479616, "text": "Wastin all that good watermelón on the rind https://t.co/I0Z5Fh4rRH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2639400257, "name": "chocolate flavour", "screen_name": "flexass", "lang": "en", "location": "RPISD", "create_at": date("2014-06-24"), "description": "ayo", "followers_count": 268, "friends_count": 433, "statues_count": 6720 }, "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:35.000Z"), "id": 668945772654804993, "text": "Absolutely Dylan Larkin! #CapitalsTalk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CapitalsTalk" }}, "user": { "id": 256816807, "name": "Allison Jane", "screen_name": "allirawrus", "lang": "en", "location": "Fairfax, Virginia", "create_at": date("2011-02-23"), "description": "null", "followers_count": 151, "friends_count": 270, "statues_count": 543 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945772885516289, "text": "when everyone hates tanner in the gc https://t.co/44Slql79MQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 611378612, "name": "$ar", "screen_name": "sar_bear80", "lang": "en", "location": "Centerville, OH", "create_at": date("2012-06-17"), "description": "null", "followers_count": 351, "friends_count": 488, "statues_count": 5693 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Centerville, OH", "id": "edacd050f402302c", "name": "Centerville", "place_type": "city", "bounding_box": rectangle("-84.21092,39.579164 -84.098555,39.674261") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39113, "countyName": "Montgomery", "cityID": 3913190, "cityName": "Centerville" } }
+{ "create_at": datetime("2015-11-23T16:14:35.000Z"), "id": 668945773023768576, "text": "@Hannahvic22 ARE YOU SERIOUS ��", "in_reply_to_status": 668942974726750208, "in_reply_to_user": 1195150692, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1195150692 }}, "user": { "id": 50929457, "name": "Mads Camps", "screen_name": "MaddieCampos95", "lang": "en", "location": "null", "create_at": date("2009-06-25"), "description": "Most likely to have the same music taste as your grandpa. UI ΔΔΔ.", "followers_count": 378, "friends_count": 366, "statues_count": 13108 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Center, WA", "id": "002ccc2b715b1ba7", "name": "La Center", "place_type": "city", "bounding_box": rectangle("-122.690591,45.856105 -122.64744,45.873393") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5336710, "cityName": "La Center" } }
+{ "create_at": datetime("2015-11-23T16:14:35.000Z"), "id": 668945773153935361, "text": "Lol https://t.co/QypMHthi5Y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3165560103, "name": "Chief_Cakes", "screen_name": "Still_Sylvia", "lang": "en", "location": "null", "create_at": date("2015-04-13"), "description": "Born & raised in Chicago. Young, but I'm old school...Here for sum discussion, to VENT, but mostly to LAUGH!! So don't u dare kill my vibe! Unless uz a....", "followers_count": 309, "friends_count": 231, "statues_count": 28751 }, "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:35.000Z"), "id": 668945773170524160, "text": "Traci All On Twitter But Can't Text A Nigga Back ! �� He's A Fake Ass Friend ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 232034740, "name": "Taméraaaaaaa ❤️➰", "screen_name": "Teeskiiiii__", "lang": "en", "location": "Cleveland, OH", "create_at": date("2010-12-29"), "description": "Gotta Keep Ya Heels High And Ya Standards Higher ✊✨❤️ ! , Rest Easy Uncle EC , JaRio's World ❤️ BIFC ♠️ , #2016 , REST UP JaRio ❤️ ~ ATL", "followers_count": 1416, "friends_count": 1368, "statues_count": 15340 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maple Heights, OH", "id": "76e0fd2d1efd1f7e", "name": "Maple Heights", "place_type": "city", "bounding_box": rectangle("-81.585463,41.372318 -81.527204,41.425052") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3947306, "cityName": "Maple Heights" } }
+{ "create_at": datetime("2015-11-23T16:14:35.000Z"), "id": 668945773204144129, "text": "@lift_everyday Every inch perfect", "in_reply_to_status": 668945599123755008, "in_reply_to_user": 2216527952, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2216527952 }}, "user": { "id": 9953462, "name": "Michael Murdock", "screen_name": "docmurdock", "lang": "en", "location": "Scottsdale, AZ", "create_at": date("2007-11-04"), "description": "aka Dr Mike Murdock, Mike Murdock, Pixar's Mac Sys Eng in Toy Story 1995. US Navy VETERAN - Patriot - Photog - DroneJock - Conservative - Domainer", "followers_count": 14823, "friends_count": 12631, "statues_count": 64131 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945773581635584, "text": "@rtelmore soooo you're the chief data officer for the state now?", "in_reply_to_status": 668913561201016832, "in_reply_to_user": 15869364, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15869364 }}, "user": { "id": 47436444, "name": "Josh Montague", "screen_name": "jrmontag", "lang": "en", "location": "Boulder / Golden", "create_at": date("2009-06-15"), "description": "counting tweets @Gnip/@TwitterBoulder. previously counted phonons @CUBoulder || mostly made of ☕️", "followers_count": 2412, "friends_count": 2210, "statues_count": 35377 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945773795545088, "text": "I'm broke af & my mom refuses to buy me shit for black friday bc I have a job��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 598765968, "name": "Chelin", "screen_name": "jocelynlara_", "lang": "en", "location": "null", "create_at": date("2012-06-03"), "description": "ponte las pilas // tengo el nopal en la frente aúnque no me vea. Saint Xavier University 19'", "followers_count": 372, "friends_count": 231, "statues_count": 11410 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945774068105217, "text": "@FilmSpectrum awesome. That rules", "in_reply_to_status": 668945700672176128, "in_reply_to_user": 2968058583, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2968058583 }}, "user": { "id": 87263759, "name": "Zach", "screen_name": "Zachw56", "lang": "en", "location": "Salt Lake City, Utah", "create_at": date("2009-11-03"), "description": "Im a special ed teacher from slc. #UtahJazz #Utes #Film #Animals Im married to an amazing woman. Also, I try to stay positive but I sometimes forget about that.", "followers_count": 206, "friends_count": 260, "statues_count": 16562 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murray, UT", "id": "5811c847bb5f82b6", "name": "Murray", "place_type": "city", "bounding_box": rectangle("-111.932066,40.623952 -111.862386,40.684428") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4953230, "cityName": "Murray" } }
+{ "create_at": datetime("2015-11-23T16:14:35.000Z"), "id": 668945774089252864, "text": "#1in5Muslims like froyo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "1in5Muslims" }}, "user": { "id": 29558719, "name": "Ryan's thoughts", "screen_name": "Ryan_ABP", "lang": "en", "location": "United States", "create_at": date("2009-04-07"), "description": "Catholic/Christian, 25 years old, college student, and weather enthusiast extraordinaire.", "followers_count": 429, "friends_count": 282, "statues_count": 15769 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westbrook Center, CT", "id": "009c8eac35ebae31", "name": "Westbrook Center", "place_type": "city", "bounding_box": rectangle("-72.494,41.269209 -72.416739,41.325637") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9007, "countyName": "Middlesex", "cityID": 981750, "cityName": "Westbrook Center" } }
+{ "create_at": datetime("2015-11-23T16:14:35.000Z"), "id": 668945774248460288, "text": "Aha como no������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 1925037216, "name": "chipo", "screen_name": "sandyiee_cheeks", "lang": "en", "location": "Maywood, CA", "create_at": date("2013-10-01"), "description": "Don't worry about it -Coach Alan", "followers_count": 374, "friends_count": 188, "statues_count": 8021 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945774311546881, "text": "\"The bugs up his ass aren't in him, they're in Russia\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 170967377, "name": "dinora", "screen_name": "_cuntfused", "lang": "en", "location": "null", "create_at": date("2010-07-25"), "description": "teen star", "followers_count": 385, "friends_count": 570, "statues_count": 19243 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945774462369792, "text": "So 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": 430135833, "name": "gabby canibeyaz", "screen_name": "GCanibeyaz", "lang": "en", "location": "null", "create_at": date("2011-12-06"), "description": "WNC SOFTBALL. ➕Phil 4:13➕", "followers_count": 272, "friends_count": 217, "statues_count": 10157 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945774538018816, "text": "I'm at Kroger in Little Rock, AR https://t.co/EjZInrmKVy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.34562397,34.66949973"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157228710, "name": "Ernesto Vergara", "screen_name": "ark_net", "lang": "es", "location": "ÜT: 19.3614787,-99.1638289", "create_at": date("2010-06-18"), "description": "ARQUITECTO co fundador #greenarquitectura _ CDMX", "followers_count": 241, "friends_count": 218, "statues_count": 5375 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945774634475520, "text": "10.\n\n10/10 https://t.co/YoaUHugvKu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1411875378, "name": "Brah Tosin", "screen_name": "TAkinyinka", "lang": "en", "location": "Osogbo. ", "create_at": date("2013-05-07"), "description": "Red/Yoruba Devil.", "followers_count": 170, "friends_count": 149, "statues_count": 7629 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945774680662020, "text": "Remember how much you used to love color-by-numbers as a kid? Well, please remember your semester… https://t.co/ILbt4Y6zAX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.36006944,40.95960833"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1267984405, "name": "Leslie Smith", "screen_name": "MrsSmithLHS", "lang": "en", "location": "Ohio, USA", "create_at": date("2013-03-14"), "description": "Do what you can, with what you have, where you are - T. Roosevelt", "followers_count": 162, "friends_count": 152, "statues_count": 3499 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hartville, OH", "id": "01e8637171768219", "name": "Hartville", "place_type": "city", "bounding_box": rectangle("-81.386821,40.913691 -81.305215,40.988871") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39151, "countyName": "Stark", "cityID": 3934328, "cityName": "Hartville" } }
+{ "create_at": datetime("2015-11-23T16:14:35.000Z"), "id": 668945774793916420, "text": "shoot her a text & let her know she important ❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1122442465, "name": "BABYGIRL", "screen_name": "yung_belleezy", "lang": "en", "location": "Detroit, MI", "create_at": date("2013-01-26"), "description": "null", "followers_count": 1503, "friends_count": 984, "statues_count": 40024 }, "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:35.000Z"), "id": 668945774890213376, "text": "Why Wilt isnt top 5 https://t.co/qqcigpmIYy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2314603190, "name": "el drewski", "screen_name": "DrrewskiDrew", "lang": "en", "location": "North Oakland / LA ", "create_at": date("2014-01-27"), "description": "if you ain't got sauce, then... ya lost. but, you also can get lost in the sauce☝️ - 1 Matthew 5:7", "followers_count": 472, "friends_count": 396, "statues_count": 42916 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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:35.000Z"), "id": 668945774894587905, "text": "Take a page out of the Giants playbook! https://t.co/TOlNwBj3MK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 304627207, "name": "WiL", "screen_name": "WilRuizNYG", "lang": "en", "location": "Bronx, NY", "create_at": date("2011-05-24"), "description": "#NYG l #Scorpio l #BxNY l @NYGTweets", "followers_count": 2773, "friends_count": 2585, "statues_count": 131454 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945774953168896, "text": "Which one are you??? https://t.co/P30yiJhyM6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 426803622, "name": "Joff Paradise", "screen_name": "JoffParadise", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2011-12-02"), "description": "health and fitness expert.", "followers_count": 352, "friends_count": 737, "statues_count": 920 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945774965710848, "text": "IT'S HOCKEY O'CLOCK! #LetsGoCaps #RockTheRed #CapsOilers https://t.co/n8U80h4JFp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LetsGoCaps", "RockTheRed", "CapsOilers" }}, "user": { "id": 350599825, "name": "Ovie the Bulldog", "screen_name": "OvietheBulldog", "lang": "en", "location": "San Diego, CA", "create_at": date("2011-08-07"), "description": "I'm a Bulldog! I sleep, eat, & snore all day long. Go Caps! I've moved to SoCal & to spread my love for Capitals Hockey to SoCal! Instagram: Oviethebulldog", "followers_count": 3857, "friends_count": 918, "statues_count": 17800 }, "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:35.000Z"), "id": 668945774965825536, "text": "Date Night!!!! Lobster Bake with a Malibu Hurricane :) (at @RedLobster in Franklin, TN) https://t.co/jezaRAe0jS https://t.co/HHci2d2K9Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.81042,35.964415"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 6018732 }}, "user": { "id": 15526160, "name": "Mikey Miller", "screen_name": "Freak4Harvick", "lang": "en", "location": "Ridgecrest CA Lyles TN", "create_at": date("2008-07-21"), "description": "O/O Holland Enterprises @Freak4Harvick on Periscope Dad of 3. NASCAR #4 #harvicknation #4thecup #4thewin COWBOYS TROJANS BAMA DODGERS #thankatruck", "followers_count": 1266, "friends_count": 2023, "statues_count": 27952 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945775091695616, "text": "why she try to hoe her doe�� https://t.co/OpFj8JYOp3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2763340855, "name": "Jesh✨", "screen_name": "jeshaunna_", "lang": "en", "location": "null", "create_at": date("2014-08-24"), "description": "#ripYing ❤️ #ripMel #canilive #goodvibesonly", "followers_count": 825, "friends_count": 717, "statues_count": 8914 }, "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:35.000Z"), "id": 668945775183851520, "text": "@ImInYourHead101 yessssss my home state", "in_reply_to_status": 668942145416380418, "in_reply_to_user": 1882187016, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1882187016 }}, "user": { "id": 1974277008, "name": "bre", "screen_name": "brex0_", "lang": "en", "location": "Anchorage, AK", "create_at": date("2013-10-20"), "description": "null", "followers_count": 1770, "friends_count": 707, "statues_count": 17106 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945775313858560, "text": "@FactsOfSchool Derby��", "in_reply_to_status": 668667478898294784, "in_reply_to_user": 312230865, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 312230865 }}, "user": { "id": 390035799, "name": "Jacob Clark", "screen_name": "JacobClark316", "lang": "en", "location": "null", "create_at": date("2011-10-13"), "description": "Junior at Derby High School and a Freshman at Butler Community College. Majoring in Digital media.\n@NxtLevelVisuals", "followers_count": 598, "friends_count": 426, "statues_count": 4348 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945775322267649, "text": "STILL TIME TO GET THIS MNF PLAY AND THE PLAY OF THE YEAR THIS FRIDAY!!!\n\nONLY $10\n\nDM NOW!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2966777357, "name": "Britt Jacks", "screen_name": "SBJ35215", "lang": "en", "location": "Orange Beach, Alabama", "create_at": date("2015-01-07"), "description": "null", "followers_count": 116, "friends_count": 514, "statues_count": 464 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945775490011136, "text": "Memphis bleek x Infatuated ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2935711849, "name": "Destinee", "screen_name": "desxalyssa", "lang": "en", "location": "Carson, CA", "create_at": date("2014-12-18"), "description": "null", "followers_count": 449, "friends_count": 411, "statues_count": 15070 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945775557087232, "text": "Poor baby ������ https://t.co/AdfUOJSK2g", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 848375120, "name": "Mami", "screen_name": "JazyE_", "lang": "en", "location": "vegas", "create_at": date("2012-09-26"), "description": "✨ Chicana | @AntsDopee is my baby ♡ ✨", "followers_count": 3350, "friends_count": 322, "statues_count": 43967 }, "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("2015-11-23T16:14:35.000Z"), "id": 668945775683108864, "text": "@bryttxoxo_ I hate her honestly ����", "in_reply_to_status": 668945687111864320, "in_reply_to_user": 2922190267, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2922190267 }}, "user": { "id": 730423830, "name": "adri.", "screen_name": "adriana_mariiie", "lang": "en", "location": "#RIPMichael #RIPVincent ❤️", "create_at": date("2012-08-01"), "description": "don't break bread with people you didn't starve with. #FreeGucci", "followers_count": 1473, "friends_count": 1393, "statues_count": 35156 }, "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("2015-11-23T16:14:36.000Z"), "id": 668945775729094656, "text": "You give me purpose.❤", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2854313320, "name": "tay.", "screen_name": "tay_tave", "lang": "en", "location": "Mineola, TX", "create_at": date("2014-10-31"), "description": "sc/ taylantave_03 • IG/ t.tavee", "followers_count": 98, "friends_count": 105, "statues_count": 1012 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Quitman, TX", "id": "001f723c0fc31b93", "name": "Quitman", "place_type": "city", "bounding_box": rectangle("-95.46105,32.771667 -95.41827,32.808226") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48499, "countyName": "Wood", "cityID": 4860188, "cityName": "Quitman" } }
+{ "create_at": datetime("2015-11-23T16:14:36.000Z"), "id": 668945775867633664, "text": "I don't want to study. I just want to paint and get everything off 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": 735244771, "name": "Emmy B", "screen_name": "emalee0430", "lang": "en", "location": "Ohio, USA", "create_at": date("2012-08-03"), "description": "Trust in The Lord with all your heart and do not lean on your own understanding. In all your ways acknowledge Him and he will make straight your paths.", "followers_count": 633, "friends_count": 499, "statues_count": 13066 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Heath, OH", "id": "01cd07bc1dcbe038", "name": "Heath", "place_type": "city", "bounding_box": rectangle("-82.485166,39.985879 -82.3707,40.043707") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39089, "countyName": "Licking", "cityID": 3934748, "cityName": "Heath" } }
+{ "create_at": datetime("2015-11-23T16:14:36.000Z"), "id": 668945775913660416, "text": "Be careful or the truth might speak\nIt's amazing when it's there in black and white for all to see\nWe sow and you reap", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 194599186, "name": "My Endless Summer", "screen_name": "Mr_Anymeans", "lang": "en", "location": "VA", "create_at": date("2010-09-24"), "description": "Just a young nigga making dreams become reality. C.R.E.W Changing Rhythms Every Where.... IG WestknowBest", "followers_count": 350, "friends_count": 474, "statues_count": 14713 }, "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("2015-11-23T16:14:36.000Z"), "id": 668945776031068161, "text": "#BieberOnAMAs ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "BieberOnAMAs" }}, "user": { "id": 1445488970, "name": "TN JØJØ♕", "screen_name": "JoJozarate14", "lang": "en", "location": "Bakersfield, CA", "create_at": date("2013-05-20"), "description": "Far Far Far Away From Home..... #StraightOutOfTennessee #BITCH", "followers_count": 805, "friends_count": 859, "statues_count": 11530 }, "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("2015-11-23T16:14:36.000Z"), "id": 668945776127557633, "text": "Buddy the Elf what's your favorite color?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1648357802, "name": "Macus Murrphy", "screen_name": "murphhh44", "lang": "en", "location": "Gilbert, AZ", "create_at": date("2013-08-05"), "description": "Mesquite High School Football, Hockey #21", "followers_count": 567, "friends_count": 371, "statues_count": 14416 }, "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("2015-11-23T16:14:36.000Z"), "id": 668945776320634881, "text": "@haleyyschaeffer smile, you're beautiful.", "in_reply_to_status": 668944705720295424, "in_reply_to_user": 363484934, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 363484934 }}, "user": { "id": 41958573, "name": "Liv", "screen_name": "liv_timmerman", "lang": "en", "location": "574 ", "create_at": date("2009-05-22"), "description": "Livin' for today, because tomorrow is never promised.", "followers_count": 1322, "friends_count": 669, "statues_count": 16876 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warsaw, IN", "id": "9a4adc08e83b58df", "name": "Warsaw", "place_type": "city", "bounding_box": rectangle("-85.944986,41.190359 -85.765358,41.304979") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18085, "countyName": "Kosciusko", "cityID": 1880306, "cityName": "Warsaw" } }
+{ "create_at": datetime("2015-11-23T16:14:36.000Z"), "id": 668945776358244352, "text": "Ugly lil thang �� https://t.co/QilOOdFYhQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 725709380, "name": "Young Zy • GC", "screen_name": "YoungZy6_Bruhh", "lang": "en", "location": "Riverside, California ", "create_at": date("2012-07-30"), "description": "I'm just A Bird #XO #GoldenClass #VanitySlave #MoneyGangMealClique", "followers_count": 772, "friends_count": 1183, "statues_count": 9037 }, "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:36.000Z"), "id": 668945776421289985, "text": "RFT!! https://t.co/FzkAsRDSwM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2156160340, "name": "leslie", "screen_name": "_queenleslie_", "lang": "en", "location": "null", "create_at": date("2013-10-27"), "description": "null", "followers_count": 689, "friends_count": 601, "statues_count": 10252 }, "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("2015-11-23T16:14:36.000Z"), "id": 668945776459038720, "text": "Had a good practice even though it was cold��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2870065757, "name": "BUCK$", "screen_name": "_DICE___", "lang": "en", "location": "Philly", "create_at": date("2014-11-09"), "description": "RIP.Grandmom | ImhotepFootball", "followers_count": 442, "friends_count": 369, "statues_count": 2013 }, "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:36.000Z"), "id": 668945776643477504, "text": "Thats cute TY...... ☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺☺ https://t.co/5PzqwDYXT9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2872572826, "name": "Tayy ♈♈", "screen_name": "__tayywadd", "lang": "en", "location": "null", "create_at": date("2014-11-11"), "description": "HTX | I'm an artist | RIP Nana |", "followers_count": 441, "friends_count": 472, "statues_count": 3062 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:36.000Z"), "id": 668945777188802560, "text": "@seayrrabiggs_16 I'm snuggling with this cutie! Want to join us? https://t.co/6ZIx6ILxXo", "in_reply_to_status": 668658311181565953, "in_reply_to_user": 1115648870, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1115648870 }}, "user": { "id": 982681662, "name": "Bridge", "screen_name": "BridgetMcDowell", "lang": "en", "location": "John 16:33", "create_at": date("2012-12-01"), "description": "...somewhere between raising hell and amazing grace...", "followers_count": 313, "friends_count": 725, "statues_count": 9846 }, "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("2015-11-23T16:14:36.000Z"), "id": 668945777230786561, "text": "Honestly why do falls fall 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": 2655537214, "name": "Carolina Taylor ❥", "screen_name": "carolinaataylor", "lang": "en", "location": "North Carolina, USA", "create_at": date("2014-06-29"), "description": "Gang gang", "followers_count": 620, "friends_count": 915, "statues_count": 1791 }, "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("2015-11-23T16:14:36.000Z"), "id": 668945777557815299, "text": "������ https://t.co/6FmCijfrkS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 53521906, "name": "♡ turkey senpai ♡", "screen_name": "LaurennRaquel", "lang": "en", "location": "Lubbock", "create_at": date("2009-07-03"), "description": "#TTU18 | 19 | ΑΔΠ | insta/snap: @laurennraquel @FearlessST // PR & EMC major // ♇ http://laurenpurser-photography.tumblr.com", "followers_count": 301, "friends_count": 236, "statues_count": 14409 }, "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("2015-11-23T16:14:36.000Z"), "id": 668945777784307716, "text": "Fuck Kansas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "et", "is_retweet": false, "user": { "id": 3511299734, "name": "Jason", "screen_name": "jasongoicochea", "lang": "en", "location": "Salina, KS", "create_at": date("2015-09-09"), "description": "SBVC✈️KWU ⚽️ SC: goicochea12", "followers_count": 169, "friends_count": 219, "statues_count": 1157 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salina, KS", "id": "e7bae49f1ac7f22e", "name": "Salina", "place_type": "city", "bounding_box": rectangle("-97.676631,38.768801 -97.557719,38.885242") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20169, "countyName": "Saline", "cityID": 2062700, "cityName": "Salina" } }
+{ "create_at": datetime("2015-11-23T16:14:36.000Z"), "id": 668945778241474560, "text": "sorry is never enough, fuck \"sometimes\".", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2984619439, "name": "حب نفسك", "screen_name": "Keviyonn", "lang": "en", "location": "null", "create_at": date("2015-01-15"), "description": "sc: keviyonn", "followers_count": 844, "friends_count": 752, "statues_count": 5969 }, "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("2015-11-23T16:14:36.000Z"), "id": 668945778254196736, "text": "���������� https://t.co/HFEkZA7nkQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 578269269, "name": "Strawberry Cheekcake", "screen_name": "Ohhstephhh", "lang": "en", "location": "Miami, FL", "create_at": date("2012-05-12"), "description": "Beard & beer loving woman @_TheRealGemini ❤️", "followers_count": 529, "friends_count": 404, "statues_count": 34842 }, "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("2015-11-23T16:14:36.000Z"), "id": 668945778472300544, "text": "At the end of the day, all i got is family. Since y'all wanna act like squares.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2152671430, "name": "KYNG 〽️", "screen_name": "___RH3", "lang": "en", "location": "Yampa, FL", "create_at": date("2013-10-25"), "description": "National Recruit....", "followers_count": 916, "friends_count": 730, "statues_count": 3809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inverness, FL", "id": "8cb4875f846d7f46", "name": "Inverness", "place_type": "city", "bounding_box": rectangle("-82.390434,28.810462 -82.266814,28.877726") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12017, "countyName": "Citrus", "cityID": 1233950, "cityName": "Inverness" } }
+{ "create_at": datetime("2015-11-23T16:14:36.000Z"), "id": 668945778866434048, "text": "Everyone is so annoying ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1127215704, "name": "bby pris", "screen_name": "58Pris", "lang": "en", "location": "null", "create_at": date("2013-01-27"), "description": "sucks to suck", "followers_count": 1070, "friends_count": 787, "statues_count": 50855 }, "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("2015-11-23T16:14:36.000Z"), "id": 668945778975477760, "text": "I'm attracted to you in ways I can't even explain", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3230333971, "name": "s", "screen_name": "SmithSerinah", "lang": "en", "location": "still a spartonian", "create_at": date("2015-05-29"), "description": "mhs snapchat//rinah101937", "followers_count": 488, "friends_count": 459, "statues_count": 1041 }, "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("2015-11-23T16:14:36.000Z"), "id": 668945779130781696, "text": "happy birthday!!!! @akingsley_ ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2274049737 }}, "user": { "id": 606722558, "name": "hales", "screen_name": "_haleyfowler_", "lang": "en", "location": "null", "create_at": date("2012-06-12"), "description": "junior @ be", "followers_count": 212, "friends_count": 363, "statues_count": 862 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Washington, KY", "id": "707da7205af94a9b", "name": "Mount Washington", "place_type": "city", "bounding_box": rectangle("-85.659493,38.011877 -85.516381,38.075499") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21029, "countyName": "Bullitt", "cityID": 2154228, "cityName": "Mount Washington" } }
+{ "create_at": datetime("2015-11-23T16:14:36.000Z"), "id": 668945779235684352, "text": "still can't believe my grandpa is gone.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 317605892, "name": "desh.", "screen_name": "__PilotJoness", "lang": "en", "location": "Luther Anthony ❤️", "create_at": date("2011-06-14"), "description": "ig: ar.mvni | sc: desharmani | rip MFLjr |", "followers_count": 1894, "friends_count": 1326, "statues_count": 80758 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little River, SC", "id": "0159b42c2f3f2207", "name": "Little River", "place_type": "city", "bounding_box": rectangle("-78.712564,33.834265 -78.581295,33.895781") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4542010, "cityName": "Little River" } }
+{ "create_at": datetime("2015-11-23T16:14:36.000Z"), "id": 668945779495559168, "text": "@kindakookykeara thanks ��", "in_reply_to_status": 668945093395656704, "in_reply_to_user": 404337880, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 404337880 }}, "user": { "id": 15173851, "name": "Hiba ;", "screen_name": "HibaIssa", "lang": "en", "location": "Salt Lake City, UT", "create_at": date("2008-06-19"), "description": "The Doctor on the street, but The Master in the sheets. \n\nNERD :: RSL :: SHENANIGANS :: MENTAL HEALTH ADVOCATE", "followers_count": 600, "friends_count": 521, "statues_count": 75373 }, "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("2015-11-23T16:14:37.000Z"), "id": 668945779931938816, "text": "I'm at Dollar General in Mangham, LA https://t.co/uTJwL0ecqb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-91.77626002,32.31724903"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 92890677, "name": "Lee Jones", "screen_name": "LeeJ07", "lang": "en", "location": "Monroe,LA", "create_at": date("2009-11-26"), "description": "Handsome southern charmer who loves to travel(At least, that's what I tell everyone off the record. I may have a few secrets!!).", "followers_count": 449, "friends_count": 2004, "statues_count": 20348 }, "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": 22083, "countyName": "Richland", "cityID": 2248260, "cityName": "Mangham" } }
+{ "create_at": datetime("2015-11-23T16:14:37.000Z"), "id": 668945779948695552, "text": "she's crazy ������ https://t.co/tQZpw5r3hx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 750832531, "name": "bs", "screen_name": "brittneysizer_", "lang": "en", "location": "new london, ct", "create_at": date("2012-08-11"), "description": "ig: brittneysizer_", "followers_count": 862, "friends_count": 748, "statues_count": 16784 }, "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("2015-11-23T16:14:37.000Z"), "id": 668945779977879552, "text": "I want some fried fish man", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 384807609, "name": "war❤️", "screen_name": "ynijru", "lang": "en", "location": "Born on October 9", "create_at": date("2011-10-04"), "description": "ibeen-on:)", "followers_count": 1995, "friends_count": 1335, "statues_count": 34570 }, "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("2015-11-23T16:14:37.000Z"), "id": 668945779977916416, "text": "It really bothers me that there are not an even amount of days in a week. #sixdaysaweek", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sixdaysaweek" }}, "user": { "id": 28260080, "name": "Jamison", "screen_name": "jamisonhudson", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-04-01"), "description": "Actress. Writer. Comedian. \nOccasional Life Enthusiast.", "followers_count": 268, "friends_count": 266, "statues_count": 10131 }, "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("2015-11-23T16:14:37.000Z"), "id": 668945780011573248, "text": "But the smile always the same..\n#teamsmile #teamclaudeasy \nTrying to find something, within to keep… https://t.co/YYXAkGbnb6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.90725036,36.40138522"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "teamsmile", "teamclaudeasy" }}, "user": { "id": 182958871, "name": "ClaudeL Y. Noël", "screen_name": "CLaudeasY_Son", "lang": "en", "location": "U.S.A, Florida", "create_at": date("2010-08-25"), "description": "Dear World: Everything in my (our) worldly lives is temporary. It's like having a buzz- Claudel Y. Noel", "followers_count": 104, "friends_count": 111, "statues_count": 5400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enid, OK", "id": "4ba5405c68ac4670", "name": "Enid", "place_type": "city", "bounding_box": rectangle("-97.981671,36.335631 -97.818889,36.448929") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40047, "countyName": "Garfield", "cityID": 4023950, "cityName": "Enid" } }
+{ "create_at": datetime("2015-11-23T16:14:37.000Z"), "id": 668945780250660866, "text": "@CaptainKirk_ please advise why I cannot find any phot galleries on the dm's website", "in_reply_to_status": -1, "in_reply_to_user": 45410406, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 45410406 }}, "user": { "id": 227136860, "name": "Ignacio Murillo", "screen_name": "igmurillo", "lang": "en", "location": "New York, NY", "create_at": date("2010-12-15"), "description": "Photo Assistant @HarpersBazaarus | Ole Miss Alumnus | #HottyToddy", "followers_count": 500, "friends_count": 743, "statues_count": 25854 }, "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("2015-11-23T16:14:37.000Z"), "id": 668945780305080321, "text": "#hinthint ������ https://t.co/zgyVYYoO6H", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "hinthint" }}, "user": { "id": 90050348, "name": "Monica Miller ∞", "screen_name": "Monica__Miller", "lang": "en", "location": "In Love", "create_at": date("2009-11-14"), "description": "Gentlemen prefer blondes.", "followers_count": 511, "friends_count": 277, "statues_count": 18219 }, "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:37.000Z"), "id": 668945780313604097, "text": "This is true https://t.co/N09shmckKI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 844401685, "name": "CE$AR UCHIHA", "screen_name": "oeanother40", "lang": "en", "location": "null", "create_at": date("2012-09-24"), "description": "Wassup :)", "followers_count": 208, "friends_count": 192, "statues_count": 13020 }, "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("2015-11-23T16:14:37.000Z"), "id": 668945780342935553, "text": "Фото 105 https://t.co/Ek5GjBAbt0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-123.0116773,37.69943359"), "retweet_count": 0, "lang": "ru", "is_retweet": false, "user": { "id": 3171272281, "name": "June Ramos", "screen_name": "jurasoma", "lang": "en", "location": "null", "create_at": date("2015-04-24"), "description": "null", "followers_count": 4, "friends_count": 0, "statues_count": 23420 }, "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("2015-11-23T16:14:37.000Z"), "id": 668945780388958212, "text": "Seems right... https://t.co/ov4ZvrG7lv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 341143819, "name": "Danny Duron", "screen_name": "dannyboy_9161", "lang": "en", "location": "Rancho | Los Angeles | Canyon", "create_at": date("2011-07-23"), "description": "Sound and Color WTAMU \n ΦΔΘ", "followers_count": 792, "friends_count": 624, "statues_count": 9010 }, "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("2015-11-23T16:14:37.000Z"), "id": 668945780615585792, "text": "Go away plz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2377709350, "name": "Katie Lavender", "screen_name": "KatieLavender", "lang": "en", "location": "null", "create_at": date("2014-03-02"), "description": "Sc- Katie.lavender", "followers_count": 542, "friends_count": 418, "statues_count": 9499 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Perry, GA", "id": "374487a83865e492", "name": "Perry", "place_type": "city", "bounding_box": rectangle("-83.764667,32.434659 -83.61914,32.506656") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13153, "countyName": "Houston", "cityID": 1360340, "cityName": "Perry" } }
+{ "create_at": datetime("2015-11-23T16:14:37.000Z"), "id": 668945780783337472, "text": "@AlexGarcia0323 ������", "in_reply_to_status": 668944889078546433, "in_reply_to_user": 876709010, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 876709010 }}, "user": { "id": 353338041, "name": "Sabrina", "screen_name": "s_furches", "lang": "en", "location": "Boone, NC", "create_at": date("2011-08-11"), "description": "aquarius - ASUHC '19", "followers_count": 495, "friends_count": 372, "statues_count": 4270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boone, NC", "id": "ecdf2813e684c066", "name": "Boone", "place_type": "city", "bounding_box": rectangle("-81.7384,36.183417 -81.57739,36.246767") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37189, "countyName": "Watauga", "cityID": 3707080, "cityName": "Boone" } }
+{ "create_at": datetime("2015-11-23T16:14:37.000Z"), "id": 668945781085204480, "text": "watching part one of #SaintsandStrangers on @NatGeoChannel about the native americans and pilgrims.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SaintsandStrangers" }}, "user_mentions": {{ 18244358 }}, "user": { "id": 2480263290, "name": "ॐ Alanna ॐ", "screen_name": "AlanaFelice_", "lang": "en", "location": "The Bahamas ✈️ VA", "create_at": date("2014-05-06"), "description": "success get them jealous. ig:@alanafelice||snapchat:@alannnafelice Bahamian • Good Vibes • Wanderlaust", "followers_count": 987, "friends_count": 1500, "statues_count": 17804 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Caroline, VA", "id": "008d8dbb54d9e8d0", "name": "Lake Caroline", "place_type": "city", "bounding_box": rectangle("-77.546401,37.978516 -77.503721,37.99874") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51033, "countyName": "Caroline", "cityID": 5143359, "cityName": "Lake Caroline" } }
+{ "create_at": datetime("2015-11-23T16:14:37.000Z"), "id": 668945781085364224, "text": "hbd @micahswildheart �� ur gonna be the best ariel out there (besides me ofcourse) https://t.co/kFGfgW2wmR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 254181506 }}, "user": { "id": 612309667, "name": "❁ Bri ❁", "screen_name": "I3rianaaa", "lang": "en", "location": "mb", "create_at": date("2012-06-18"), "description": ":))", "followers_count": 811, "friends_count": 628, "statues_count": 11610 }, "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("2015-11-23T16:14:37.000Z"), "id": 668945781286572032, "text": "@SkyWilliams Bruh, why you treating yourself with the leftover salt from the rift?", "in_reply_to_status": 668945439626911744, "in_reply_to_user": 291859009, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 291859009 }}, "user": { "id": 386899884, "name": "Brennen Wong", "screen_name": "BrennenWong", "lang": "en", "location": "San Marcos", "create_at": date("2011-10-07"), "description": "You don't have a soul; You are a soul, you have a body. Calgary, Alberta, Canada --- Richmond, Texas", "followers_count": 203, "friends_count": 280, "statues_count": 11078 }, "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("2015-11-23T16:14:37.000Z"), "id": 668945781290893314, "text": "���������������� https://t.co/f1r7QVX2JM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 576736685, "name": "Spicy Butterfly", "screen_name": "nicolekpardillo", "lang": "en", "location": "null", "create_at": date("2012-05-10"), "description": "null", "followers_count": 645, "friends_count": 260, "statues_count": 21271 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mountain Home, ID", "id": "e6a9ea72aaf42774", "name": "Mountain Home", "place_type": "city", "bounding_box": rectangle("-115.717233,43.106867 -115.657366,43.162128") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16039, "countyName": "Elmore", "cityID": 1654730, "cityName": "Mountain Home" } }
+{ "create_at": datetime("2015-11-23T16:14:37.000Z"), "id": 668945781542359040, "text": "�� https://t.co/ggo5TA0xih", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2840603040, "name": "Kayla", "screen_name": "Goodxxviibez", "lang": "en", "location": "Sam Houston State University", "create_at": date("2014-10-04"), "description": "null", "followers_count": 714, "friends_count": 649, "statues_count": 119 }, "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("2015-11-23T16:14:37.000Z"), "id": 668945781685112832, "text": "Have you seen @BenAffleck around Boston!? Check out some of these shots from the movie set!! \nhttps://t.co/7bOUdK7QV6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 329746058 }}, "user": { "id": 3115545573, "name": "Kevin Kennedy", "screen_name": "KevKennedy1025", "lang": "en", "location": "Boston, MA", "create_at": date("2015-03-25"), "description": "Kevin Kennedy heard weekdays, 2p-7p on Boston's #1 For New Hit Country, @Country1025wklb", "followers_count": 2039, "friends_count": 2881, "statues_count": 982 }, "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("2015-11-23T16:14:37.000Z"), "id": 668945782050062337, "text": "WHAT WVENTGSJNJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3078679912, "name": "Reindeer Rach☄pinned", "screen_name": "EthansSmile", "lang": "en", "location": "spirtually with may", "create_at": date("2015-03-07"), "description": "Ethan And Grayson make my jingle bells ring", "followers_count": 1631, "friends_count": 788, "statues_count": 28352 }, "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("2015-11-23T16:14:37.000Z"), "id": 668945782402236417, "text": "For get a Prius, get a BMW X1! https://t.co/3tJSkng0Tt", "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": "Young man interested in Architecture. Actions speak louder than words. Virgin Billionaires Series. https://randolphharris3508.wordpress.com/", "followers_count": 1045, "friends_count": 1948, "statues_count": 40365 }, "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:37.000Z"), "id": 668945782557376512, "text": "Prediction: @SRCHS_Athletics has won last 3 meetings w/@bigvalleycs by avg score of 39-6. Knights will defend D7 title but not by shutout.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3237118486, 3331752093 }}, "user": { "id": 19582084, "name": "James Burns", "screen_name": "jburns1980", "lang": "en", "location": "Central Valley, California", "create_at": date("2009-01-26"), "description": "Reporter with the Modesto Bee and California kid with a passion for telling stories with words, pictures and video. Email me at jburns@modbee.com.", "followers_count": 1048, "friends_count": 705, "statues_count": 4786 }, "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("2015-11-23T16:14:37.000Z"), "id": 668945782859395072, "text": "FEELIN THE BERN TONIGHT IN ATLANTA! ! https://t.co/OHpROZoysA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 123952796, "name": "ELET T. VALENTINE", "screen_name": "ETVALENTINE", "lang": "en", "location": "Denver, Colorado", "create_at": date("2010-03-17"), "description": "null", "followers_count": 199, "friends_count": 541, "statues_count": 873 }, "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:37.000Z"), "id": 668945783228469248, "text": "Era sarcasmo, Annie! https://t.co/DKIwHYhPLh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 322318102, "name": "*Sniper Texano*", "screen_name": "Sethsalgado17", "lang": "es", "location": "Houston, Tx.", "create_at": date("2011-06-22"), "description": "Para los amigos amistad, justicia y sacrificio; para los traidores como @jorgevergara solo justicia! #ChivaHermanos Sigo siendo AntiVergara #MotherFuckerLiar", "followers_count": 1863, "friends_count": 1844, "statues_count": 33706 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:37.000Z"), "id": 668945784063283200, "text": "@Jaynice1738 nah Fam Lmao They said 2 fights they didn't even know about the ones in the back of the school", "in_reply_to_status": 668945480815128577, "in_reply_to_user": 3135036489, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3135036489 }}, "user": { "id": 3344470319, "name": "Jared.", "screen_name": "BasedGodJae", "lang": "en", "location": "MD,", "create_at": date("2015-06-24"), "description": "Live Happy.", "followers_count": 52, "friends_count": 51, "statues_count": 80 }, "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("2015-11-23T16:14:38.000Z"), "id": 668945784285601793, "text": "Just ready for the Patriots to play", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2759148140, "name": "Raquez.", "screen_name": "theyreprocky_", "lang": "en", "location": "ATL", "create_at": date("2014-08-23"), "description": "West Virginia Tech Commit\n#FreeMilk", "followers_count": 381, "friends_count": 288, "statues_count": 6275 }, "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("2015-11-23T16:14:38.000Z"), "id": 668945784386097152, "text": "There's construction in the airport tunnel AND Eastern, sooooo dumb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 111493139, "name": "Seungkoo Han", "screen_name": "seungguri95", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2010-02-04"), "description": "gooooooooooooalㅣUNLVㅣblacksheep", "followers_count": 177, "friends_count": 157, "statues_count": 11421 }, "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("2015-11-23T16:14:38.000Z"), "id": 668945784482734080, "text": "*laughs* no, that's Kendrick https://t.co/eYrZdZi98r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1146735673, "name": "S. Morales", "screen_name": "TribeCalledSash", "lang": "en", "location": "Columbus, OH", "create_at": date("2013-02-03"), "description": "CAHS Alumna|College University", "followers_count": 841, "friends_count": 675, "statues_count": 98738 }, "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("2015-11-23T16:14:38.000Z"), "id": 668945784503730178, "text": "I JUST WANT ONE!!! https://t.co/yPJsaO0DGI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2277263877, "name": "Matt wixted", "screen_name": "NewYork_Pr0", "lang": "en", "location": "Lorton, VA", "create_at": date("2014-01-12"), "description": "null", "followers_count": 97, "friends_count": 243, "statues_count": 401 }, "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("2015-11-23T16:14:38.000Z"), "id": 668945784688087040, "text": "Nahh if that th case get a # twibbon to support my LAkings lol #GKG https://t.co/UMQaR9Yuup", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GKG" }}, "user": { "id": 55465125, "name": "Marco Mckelligan", "screen_name": "hotspur55", "lang": "en", "location": "LOS ANGELES", "create_at": date("2009-07-09"), "description": "C.A.I. oasis #mclaren massive fan of LAkings,tottenham hotspur,CELTICglasgow & @clubsantos hate barca,arsenal & sevconians", "followers_count": 3223, "friends_count": 2528, "statues_count": 27529 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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:38.000Z"), "id": 668945784742760448, "text": "生放送のネタは...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user": { "id": 2764874039, "name": "Tarucci@彼女いないぼっち系実況者", "screen_name": "Tarucci", "lang": "ja", "location": "Philadelphia, PA", "create_at": date("2014-09-08"), "description": "フィラデルフィア在住の実況者です。 音楽、ゲーム、海外生活、以前住んでいたNYやロンドンについてツイートします。 動画Part1→http://goo.gl/8BS2yB コミュ→http://goo.gl/AuNfx1 Skype→tarucci68\nブログ→http://goo.gl/W0xBFH", "followers_count": 62, "friends_count": 58, "statues_count": 2208 }, "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:38.000Z"), "id": 668945784885387264, "text": "can stand his ass sometimes ' but he's all I want right now �������� https://t.co/UJWrZlCyL3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2709075831, "name": "Stephanie benitez", "screen_name": "_benitezsteph", "lang": "en", "location": "null", "create_at": date("2014-07-13"), "description": "With pain, comes strength", "followers_count": 146, "friends_count": 184, "statues_count": 1474 }, "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("2015-11-23T16:14:38.000Z"), "id": 668945784964935681, "text": "I was dead af during lunch sns ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1325480947, "name": "yanis", "screen_name": "Yanis_Amador", "lang": "en", "location": "null", "create_at": date("2013-04-03"), "description": "❣ band", "followers_count": 807, "friends_count": 420, "statues_count": 14855 }, "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:38.000Z"), "id": 668945785225003008, "text": "I got the Nobel peace prize when I was 9 I have 4 Grammys and 10 VMA awards and yesterday I won a award at the AMAs https://t.co/ujCrJTGU2N", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880245419, "name": "Peasant Esau", "screen_name": "TrapGodEsau", "lang": "en", "location": "On The 2nd Floor", "create_at": date("2014-10-27"), "description": "☯☯", "followers_count": 137, "friends_count": 163, "statues_count": 2031 }, "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("2015-11-23T16:14:38.000Z"), "id": 668945785355182080, "text": "So there was a shooting at Citrus Park mall??", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1140222540, "name": "Jules", "screen_name": "JuliaLinton", "lang": "en", "location": "Tampa, FL", "create_at": date("2013-02-01"), "description": "FHS '17 @mickifier", "followers_count": 1260, "friends_count": 992, "statues_count": 8944 }, "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("2015-11-23T16:14:38.000Z"), "id": 668945785568923648, "text": "Putting the finishing touches on this beauty this afternoon #painting #worship https://t.co/RWUGPq9m20", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "painting", "worship" }}, "user": { "id": 1548812671, "name": "Jen Street", "screen_name": "StreetNJen", "lang": "en", "location": "Redding, CA", "create_at": date("2013-06-26"), "description": "Independent Distributor with It Works - prophetic artist - mother - lover of Jesus - transformer of society https://www.facebook.com/jen.street.58", "followers_count": 93, "friends_count": 221, "statues_count": 874 }, "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("2015-11-23T16:14:38.000Z"), "id": 668945785648603136, "text": "@319E17th ...that MSU's weakness was their secondary, and, what, 2 throws downfield. Irresponsible. (I sound like @jbook)...", "in_reply_to_status": 668945571600867328, "in_reply_to_user": 22337786, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 118101086, 15182620 }}, "user": { "id": 22337786, "name": "eddi vulić", "screen_name": "zidaya", "lang": "en", "location": "Bellingham", "create_at": date("2009-02-28"), "description": "Teacher of mathematical habits of mind, Ohio State fan, Tolkien consultant, medievalist in temperament, reader of books", "followers_count": 339, "friends_count": 535, "statues_count": 14453 }, "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("2015-11-23T16:14:38.000Z"), "id": 668945785665560576, "text": "Your relationship won't last if you let other people come in between it, and let them get involved.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1468681291, "name": "free crack 2", "screen_name": "avery_40oz", "lang": "en", "location": "Bronx, NY", "create_at": date("2013-05-29"), "description": "Ig: @avery.sosa | Evan ❤️", "followers_count": 1234, "friends_count": 654, "statues_count": 1406 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manchester, CT", "id": "f1d134c7fd204d74", "name": "Manchester", "place_type": "city", "bounding_box": rectangle("-72.583489,41.733619 -72.465121,41.820226") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 944690, "cityName": "Manchester" } }
+{ "create_at": datetime("2015-11-23T16:14:38.000Z"), "id": 668945786177060865, "text": "@brentblum @rachtodd1 ���� https://t.co/bJVuy2lj0o", "in_reply_to_status": 668938454743433216, "in_reply_to_user": 18180408, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 18180408, 2391735470 }}, "user": { "id": 2427686280, "name": "Eli James", "screen_name": "EliJames27", "lang": "en", "location": "Iowa State University", "create_at": date("2014-04-04"), "description": "Hockey| SC: Eli_James27| Future is bright |ΠΚΦ|", "followers_count": 348, "friends_count": 237, "statues_count": 9207 }, "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("2015-11-23T16:14:38.000Z"), "id": 668945786219003905, "text": "LMAO https://t.co/dVtxfxFd6A", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2700529234, "name": "adreanaamariee", "screen_name": "adreanaam_", "lang": "en", "location": "null", "create_at": date("2014-07-11"), "description": "I'm a brat", "followers_count": 648, "friends_count": 454, "statues_count": 16622 }, "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("2015-11-23T16:14:38.000Z"), "id": 668945786252562432, "text": "@ovoxojakie @AyeeItsCheetos ������������������������������������������������������������������������������������", "in_reply_to_status": 668945355602432000, "in_reply_to_user": 67428778, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 67428778, 909177968 }}, "user": { "id": 635903128, "name": "Jr", "screen_name": "JrSoleChill", "lang": "en", "location": "Houston, TX", "create_at": date("2012-07-14"), "description": "yes im 5'5-.-", "followers_count": 764, "friends_count": 349, "statues_count": 38368 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:38.000Z"), "id": 668945786462273536, "text": "Missing this hottie�� https://t.co/eeKQawGH0R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2783251460, "name": "Molly Hoffman", "screen_name": "molyll", "lang": "en", "location": "Corpus Christi, TX", "create_at": date("2014-08-31"), "description": "My two interests are Scarlet and coffee...\n\nDo not be afraid; have faith Mark 5:36\nIWA Junior", "followers_count": 109, "friends_count": 153, "statues_count": 1008 }, "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("2015-11-23T16:14:38.000Z"), "id": 668945786613465088, "text": "Let me lead the trolls out from under the bridges...Tom Crean is NOT a good coach! #MauiInvitational", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MauiInvitational" }}, "user": { "id": 878531786, "name": "D. J. S.", "screen_name": "dsheridan52", "lang": "en", "location": "South Carolina", "create_at": date("2012-10-13"), "description": "I dislike know-it-all sportswriters who never played organized sports. Which I think describes most of them.", "followers_count": 125, "friends_count": 281, "statues_count": 2762 }, "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("2015-11-23T16:14:38.000Z"), "id": 668945786709737473, "text": "@domghosthunter Hey Dom how far from Superior Wisconsin are you brother?", "in_reply_to_status": -1, "in_reply_to_user": 518186788, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 518186788 }}, "user": { "id": 2759198191, "name": "Chris Twomey", "screen_name": "bdckool", "lang": "en", "location": "TN, USA #BeastMode #ISYMFS", "create_at": date("2014-08-23"), "description": "Amateur bodybuilder, had an awesome time in The Last Castle Now cast as Richard Williams in #Desiree the movie! Assoc. producer @burdenofreedom & @desireefilm7", "followers_count": 1468, "friends_count": 1195, "statues_count": 15377 }, "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("2015-11-23T16:14:38.000Z"), "id": 668945786919497728, "text": "@_glazeddoughnut @HaleyScibelli https://t.co/JjoZoXQPkM", "in_reply_to_status": 668942297401167872, "in_reply_to_user": 599685493, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 599685493, 2245998673 }}, "user": { "id": 2300509987, "name": "ca$h turkey", "screen_name": "keelyeld", "lang": "en", "location": "Grants Pass", "create_at": date("2014-01-19"), "description": "Birkenstocks aren't cool", "followers_count": 208, "friends_count": 249, "statues_count": 7040 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grants Pass, OR", "id": "00a8b2b85821e087", "name": "Grants Pass", "place_type": "city", "bounding_box": rectangle("-123.360581,42.386741 -123.283133,42.471891") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41033, "countyName": "Josephine", "cityID": 4130550, "cityName": "Grants Pass" } }
+{ "create_at": datetime("2015-11-23T16:14:38.000Z"), "id": 668945787024490496, "text": "@united Over 2 weeks since my Mom got food poisoning on one of your flights and almost a $3k ER bill to pay for. Something must be done.", "in_reply_to_status": -1, "in_reply_to_user": 260907612, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 260907612 }}, "user": { "id": 82273221, "name": "merissa", "screen_name": "iheartuofm", "lang": "en", "location": ":: NYC ::", "create_at": date("2009-10-13"), "description": "Made in the ✋. Est. 1985.", "followers_count": 198, "friends_count": 285, "statues_count": 3051 }, "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("2015-11-23T16:14:38.000Z"), "id": 668945787062095872, "text": "House goals https://t.co/OOsObbQHDR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 270631964, "name": "Lindsey Murray", "screen_name": "Lindssseeeyy", "lang": "en", "location": "Berkley, MI", "create_at": date("2011-03-22"), "description": "null", "followers_count": 654, "friends_count": 349, "statues_count": 21870 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkley, MI", "id": "5cbea7748a197682", "name": "Berkley", "place_type": "city", "bounding_box": rectangle("-83.203431,42.488362 -83.158788,42.511326") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2607660, "cityName": "Berkley" } }
+{ "create_at": datetime("2015-11-23T16:14:38.000Z"), "id": 668945787225661440, "text": "@MBridegam census numbers on vacant units are not reliable IMO. Methodology grossly overstates.", "in_reply_to_status": 668945396299751424, "in_reply_to_user": 558143796, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 558143796 }}, "user": { "id": 5818162, "name": "Ed", "screen_name": "eparillon", "lang": "en", "location": "San Francisco, CA", "create_at": date("2007-05-06"), "description": "BGI - POS - NJ - NYC - SFO", "followers_count": 866, "friends_count": 237, "statues_count": 29190 }, "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:38.000Z"), "id": 668945787775242240, "text": "@__BbyNelly T-Mobile my brother in law works there he can help you!", "in_reply_to_status": 668945591200780288, "in_reply_to_user": 518035089, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 518035089 }}, "user": { "id": 237391516, "name": "Kelly", "screen_name": "kellyollitsac_", "lang": "en", "location": "null", "create_at": date("2011-01-12"), "description": "@bryanv_30", "followers_count": 196, "friends_count": 115, "statues_count": 7839 }, "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("2015-11-23T16:14:38.000Z"), "id": 668945787783544832, "text": "@ReverseMadManW sorry i took so long Will, i was at my mothers", "in_reply_to_status": 667906996352356352, "in_reply_to_user": 3820957033, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3820957033 }}, "user": { "id": 3362389639, "name": "Lisa Garcia", "screen_name": "LliliGarcia", "lang": "en", "location": "null", "create_at": date("2015-08-27"), "description": "null", "followers_count": 470, "friends_count": 1666, "statues_count": 2840 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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:38.000Z"), "id": 668945787867426816, "text": "@cristiankrumz im not about to rock that Selena smh", "in_reply_to_status": 668944931856117760, "in_reply_to_user": 2759982328, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2759982328 }}, "user": { "id": 41918262, "name": "b¡nk! 182", "screen_name": "JayTheCorruptor", "lang": "en", "location": "USA, TX", "create_at": date("2009-05-22"), "description": "⠀ ⠀⠀ ⠀⠀⠀⠀ @UnityTX⠀⠀⠀ ⠀⠀ ⠀ ⠀ ⠀⠀ ⠀⠀⠀ ⠀⠀⠀ ⠀⠀ ⠀ ⠀⠀ SKIPHOPNATIVE ⠀ ⠀⠀ ⠀ ⠀⠀ ⠀ ⠀⠀ ⠀ ⠀⠀ ⠀⠀ ⠀ ⠀", "followers_count": 2787, "friends_count": 980, "statues_count": 120079 }, "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("2015-11-23T16:14:38.000Z"), "id": 668945787947106304, "text": "Happy birthday @Caitie_Geee hope it was a great one!! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 942643440 }}, "user": { "id": 459840600, "name": "Alex Trachtman #351", "screen_name": "16atrachtman", "lang": "en", "location": "null", "create_at": date("2012-01-09"), "description": "HH '16 PU '20", "followers_count": 328, "friends_count": 219, "statues_count": 4129 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cicero, IN", "id": "511072be76369652", "name": "Cicero", "place_type": "city", "bounding_box": rectangle("-86.037728,40.10145 -86.008362,40.146474") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18057, "countyName": "Hamilton", "cityID": 1812628, "cityName": "Cicero" } }
+{ "create_at": datetime("2015-11-23T16:14:38.000Z"), "id": 668945788140191744, "text": "�� https://t.co/WdQPEP5ndw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 700728185, "name": "MARSHALLSTSIDDY⛄️", "screen_name": "marshallstsiddy", "lang": "en", "location": "NORTHPHILLY", "create_at": date("2013-10-03"), "description": "DONT GET LOST TRYNA FIT IN , DO YOU & KEEP IT MOVING NO ONE LOVE YOU IN THESE STREETS✈️", "followers_count": 16641, "friends_count": 17882, "statues_count": 2463 }, "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:38.000Z"), "id": 668945788152758272, "text": "It must have been something I said", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.896028,35.0168389"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2809889586, "name": "AHH", "screen_name": "cinatwist1", "lang": "en", "location": "null", "create_at": date("2014-09-14"), "description": "coolin'", "followers_count": 232, "friends_count": 229, "statues_count": 2433 }, "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("2015-11-23T16:14:39.000Z"), "id": 668945788337131520, "text": "Blue ⭕ https://t.co/HLk0kwF4rp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 366897412, "name": "Cash Money", "screen_name": "Cash2_Cool", "lang": "en", "location": "DTX ", "create_at": date("2011-09-02"), "description": "#NavarroCollege'17 Making Dreams Become Reality", "followers_count": 1437, "friends_count": 798, "statues_count": 121364 }, "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("2015-11-23T16:14:39.000Z"), "id": 668945788555231232, "text": "@yofatgirl_gabss peep the outfit colors doe https://t.co/H718p5Hwp0", "in_reply_to_status": -1, "in_reply_to_user": 3653228714, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3653228714 }}, "user": { "id": 2155319896, "name": "princess ✨", "screen_name": "desseraae", "lang": "en", "location": "iriam, tx ", "create_at": date("2013-10-26"), "description": "im fab dab | singleeeeee asf ✨", "followers_count": 1472, "friends_count": 534, "statues_count": 73260 }, "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("2015-11-23T16:14:39.000Z"), "id": 668945788613976064, "text": "I always smell like warm vanilla sugar ☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1132812492, "name": "nat $$", "screen_name": "natalietrigg", "lang": "en", "location": "on the track ", "create_at": date("2013-01-29"), "description": "track&field$$", "followers_count": 706, "friends_count": 309, "statues_count": 17672 }, "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("2015-11-23T16:14:39.000Z"), "id": 668945788895006721, "text": "@Alicia_M_X69 @cZECHyerself yeah y'all must be bored ��", "in_reply_to_status": 668945664928157696, "in_reply_to_user": 615478795, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 615478795, 531939129 }}, "user": { "id": 229741378, "name": "Shannon Evans II ™", "screen_name": "hollywood_XI", "lang": "en", "location": "Suffolk, VA", "create_at": date("2010-12-22"), "description": "Son of Shannon Sr. & Armona Evans. Brother to Ashley & Myles Evans. Student-Athlete at Arizona State University #ForksUp", "followers_count": 2318, "friends_count": 959, "statues_count": 17461 }, "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("2015-11-23T16:14:39.000Z"), "id": 668945788995633152, "text": "Miss you too bestfriend! �� https://t.co/KRNIFnlDuT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 584237452, "name": "Marcus Taylor", "screen_name": "kiddsteez23", "lang": "en", "location": "null", "create_at": date("2012-05-18"), "description": "i just wanna be great 3️⃣1️⃣8️⃣⚜➡️5️⃣8️⃣0️⃣", "followers_count": 746, "friends_count": 705, "statues_count": 8898 }, "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("2015-11-23T16:14:39.000Z"), "id": 668945789071269888, "text": "Tom crean 's gotta be the most hated coach by his own fan base in America lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 536424337, "name": "Carson", "screen_name": "C_somerville1", "lang": "en", "location": "Jeffersonville, Indiana ", "create_at": date("2012-03-25"), "description": "JHS Wrestling #BBN", "followers_count": 280, "friends_count": 514, "statues_count": 11414 }, "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("2015-11-23T16:14:39.000Z"), "id": 668945789192945666, "text": "@WinObs It's pretty decent, mostly predictable, albeit with a twist. Worth a dime.", "in_reply_to_status": 668745115582767104, "in_reply_to_user": 15434432, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15434432 }}, "user": { "id": 123109301, "name": "Anthony Raynor", "screen_name": "awraynor", "lang": "en", "location": "Spartanburg, SC", "create_at": date("2010-03-14"), "description": "null", "followers_count": 215, "friends_count": 321, "statues_count": 4922 }, "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("2015-11-23T16:14:39.000Z"), "id": 668945789499101184, "text": "@kelseyballs ������������", "in_reply_to_status": 668941863416672257, "in_reply_to_user": 368006055, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 368006055 }}, "user": { "id": 368006055, "name": "❁", "screen_name": "kelseyballs", "lang": "en", "location": "734", "create_at": date("2011-09-04"), "description": "whatever happens, happens.", "followers_count": 771, "friends_count": 197, "statues_count": 45458 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden City, MI", "id": "6e4934d73c0cb460", "name": "Garden City", "place_type": "city", "bounding_box": rectangle("-83.379378,42.310149 -83.310862,42.340625") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2631420, "cityName": "Garden City" } }
+{ "create_at": datetime("2015-11-23T16:14:39.000Z"), "id": 668945789557821441, "text": "@Divergent338 KENSI INNOCENTLY SAYS DAIRY IS HEALTHY", "in_reply_to_status": 668943303665029120, "in_reply_to_user": 2670549326, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2670549326 }}, "user": { "id": 55325084, "name": "Jess", "screen_name": "dayisfading", "lang": "en", "location": "the curtains of the waterfall ", "create_at": date("2009-07-09"), "description": "east tennessee girl on the crystal coast. vol for life. orange&white til the end. go preds go. pharmacist. wannabe writer, professional fangirl.", "followers_count": 192, "friends_count": 152, "statues_count": 177760 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morehead City, NC", "id": "01fea1ed0b8cdbb6", "name": "Morehead City", "place_type": "city", "bounding_box": rectangle("-76.90017,34.705977 -76.670032,34.762852") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37031, "countyName": "Carteret", "cityID": 3744320, "cityName": "Morehead City" } }
+{ "create_at": datetime("2015-11-23T16:14:39.000Z"), "id": 668945789687717889, "text": "@SaulSaulh96 good shii", "in_reply_to_status": 668940611802632192, "in_reply_to_user": 1485307225, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1485307225 }}, "user": { "id": 720958608, "name": "ant", "screen_name": "extendoant", "lang": "en", "location": "east, tx", "create_at": date("2012-07-27"), "description": "lexi's daddy", "followers_count": 1097, "friends_count": 376, "statues_count": 27408 }, "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("2015-11-23T16:14:39.000Z"), "id": 668945789691932672, "text": "@iwasrun I was hopeful :(", "in_reply_to_status": 668945643189088256, "in_reply_to_user": 90907952, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 90907952 }}, "user": { "id": 271689432, "name": "Michael Katz", "screen_name": "MichaelLKatz", "lang": "en", "location": "Twin Falls, Idaho", "create_at": date("2011-03-24"), "description": "Boise State football, CSI & prep sports reporter for the Times-News. Formerly at OC Register. Proud USC & Redlands alum. IG: michaellkatz mkatz@magicvalley.com", "followers_count": 1015, "friends_count": 1284, "statues_count": 12127 }, "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("2015-11-24T10:00:00.000Z"), "id": 669213892678017024, "text": "See our latest #Omaha, NE #job and click to apply: Shift Supervisor - https://t.co/ROKz1xzBw2 #parttime #Hospitality https://t.co/eUdBr842ZG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.1319583,41.228817"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Omaha", "job", "parttime", "Hospitality" }}, "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": 139, "friends_count": 0, "statues_count": 12454 }, "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("2015-11-24T10:00:00.000Z"), "id": 669213892719984640, "text": "I have to kill V nasty", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 423157169, "name": "lil mama crying", "screen_name": "coleneiers", "lang": "en", "location": "houstatlantavegas", "create_at": date("2011-11-27"), "description": "he/him", "followers_count": 3692, "friends_count": 826, "statues_count": 101003 }, "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("2015-11-24T10:00:00.000Z"), "id": 669213892732563456, "text": "@NotoriousBanks so precious", "in_reply_to_status": 509086347483959298, "in_reply_to_user": 163932446, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 163932446 }}, "user": { "id": 2199850015, "name": "Palepissbaby", "screen_name": "Lorde_Squid", "lang": "en", "location": "null", "create_at": date("2013-11-17"), "description": "21. Juggalo Mosh Mom. AZ.", "followers_count": 481, "friends_count": 292, "statues_count": 7782 }, "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-24T10:00:00.000Z"), "id": 669213892875173889, "text": "See our latest #Bridgewater, NJ #job and click to apply: Counter Sales (Retail) - https://t.co/Wm6f8zOuqq #Sales #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.6283316,40.5971482"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Bridgewater", "job", "Sales", "Hiring", "CareerArc" }}, "user": { "id": 21629556, "name": "TMJ - NJN Sales Jobs", "screen_name": "tmj_njn_sales", "lang": "en", "location": "Northern NJ", "create_at": date("2009-02-22"), "description": "Follow this account for geo-targeted Sales job tweets in Northern NJ. Need help? Tweet us at @CareerArc!", "followers_count": 405, "friends_count": 312, "statues_count": 351 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bridgewater, NJ", "id": "0036e10c05fcd097", "name": "Bridgewater", "place_type": "city", "bounding_box": rectangle("-74.673756,40.5739 -74.597615,40.636758") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34035, "countyName": "Somerset", "cityID": 3444430, "cityName": "Martinsville" } }
+{ "create_at": datetime("2015-11-24T10:00:00.000Z"), "id": 669213893068242945, "text": "I need to buy a cheap but very pigmented eyeshadow palette��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 247595800, "name": "Tyra", "screen_name": "_tyrarivera", "lang": "en", "location": "Florida, USA", "create_at": date("2011-02-04"), "description": "11/22❤️", "followers_count": 1993, "friends_count": 511, "statues_count": 98902 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winter Haven, FL", "id": "6098c1080dfc7af3", "name": "Winter Haven", "place_type": "city", "bounding_box": rectangle("-81.770135,27.939559 -81.627638,28.095052") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12105, "countyName": "Polk", "cityID": 1278275, "cityName": "Winter Haven" } }
+{ "create_at": datetime("2015-11-24T10:00:00.000Z"), "id": 669213893185683460, "text": "@mrskarz you weren't apart of that. i didn't come up with these rules", "in_reply_to_status": 669211499982929920, "in_reply_to_user": 2597315281, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2597315281 }}, "user": { "id": 85857465, "name": "Lenny Claus", "screen_name": "leisurely_lenny", "lang": "en", "location": "Dimmsdale", "create_at": date("2009-10-28"), "description": "a meaty man", "followers_count": 678, "friends_count": 293, "statues_count": 31811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Grove, IL", "id": "a1d8b540cb58cfca", "name": "Long Grove", "place_type": "city", "bounding_box": rectangle("-88.043344,42.153613 -87.963765,42.258903") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1744524, "cityName": "Long Grove" } }
+{ "create_at": datetime("2015-11-24T10:00:00.000Z"), "id": 669213893403787264, "text": "Space and Time are as different as X & Y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.1118861,38.7281064"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 282292991, "name": "Mr. Awesome", "screen_name": "PatrickSedjro", "lang": "en", "location": "United States", "create_at": date("2011-04-14"), "description": "live the life your 12 year old self would be proud of.", "followers_count": 257, "friends_count": 231, "statues_count": 7043 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Vernon, VA", "id": "6be71d3134ce7613", "name": "Mount Vernon", "place_type": "city", "bounding_box": rectangle("-77.140882,38.697083 -77.072778,38.747275") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5154144, "cityName": "Mount Vernon" } }
+{ "create_at": datetime("2015-11-24T10:00:00.000Z"), "id": 669213893554626560, "text": "Hey #Nashville, @rachelelynae says you gotta try Sweet Beet! :: 12thavesouth @ White's Mercantile https://t.co/s4BKHphNOf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.7902069,36.1209602"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nashville" }}, "user_mentions": {{ 69211533 }}, "user": { "id": 2189216348, "name": "Franklin Juice Co.", "screen_name": "franklinjuice", "lang": "en", "location": "Franklin, Tennessee", "create_at": date("2013-11-11"), "description": "A Cold-Pressed Juicery at the Factory in Franklin, TN // Juices. Cleanses. Elixirs // strictly ORGANIC ingredients // #FJCO", "followers_count": 556, "friends_count": 362, "statues_count": 1339 }, "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.99451 -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("2015-11-24T10:00:00.000Z"), "id": 669213893579771904, "text": "babysitting 4 girls ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4096547364, "name": "beccaaa", "screen_name": "beccaamaarie", "lang": "en", "location": "null", "create_at": date("2015-11-01"), "description": "012615", "followers_count": 84, "friends_count": 101, "statues_count": 370 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mandeville, LA", "id": "003d28304705c42a", "name": "Mandeville", "place_type": "city", "bounding_box": rectangle("-90.154712,30.348341 -90.003767,30.459998") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22103, "countyName": "St. Tammany", "cityID": 2248225, "cityName": "Mandeville" } }
+{ "create_at": datetime("2015-11-24T10:00:00.000Z"), "id": 669213893810499584, "text": "@ArceMariah and where they have you at? Lol", "in_reply_to_status": 669211666978967552, "in_reply_to_user": 602168149, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 602168149 }}, "user": { "id": 1090253341, "name": "o", "screen_name": "oliviarenae23", "lang": "en", "location": "HTX.", "create_at": date("2013-01-14"), "description": "18//UTA'20", "followers_count": 599, "friends_count": 308, "statues_count": 10251 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:00.000Z"), "id": 669213894032789504, "text": "@DBC_Jake seen it", "in_reply_to_status": 669198518989406208, "in_reply_to_user": 1572131498, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1572131498 }}, "user": { "id": 2262201223, "name": "Chris ⚓️", "screen_name": "CKhatch", "lang": "en", "location": "Fillet and Release", "create_at": date("2013-12-25"), "description": "RIP Seth Carter 06/15/99 - 05/25/15❤️", "followers_count": 396, "friends_count": 373, "statues_count": 6596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bishop, CA", "id": "01dcff4a7b134711", "name": "Bishop", "place_type": "city", "bounding_box": rectangle("-118.456101,37.345289 -118.381238,37.390448") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6027, "countyName": "Inyo", "cityID": 606798, "cityName": "Bishop" } }
+{ "create_at": datetime("2015-11-24T10:00:00.000Z"), "id": 669213894045360128, "text": "Apple Bottom windbreaker �������� https://t.co/fT0tsSsj8j", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 322056102, "name": "Almighty Juice God", "screen_name": "BadNews_BWard", "lang": "en", "location": "null", "create_at": date("2011-06-22"), "description": "LOUDPack CEO #LOUDPack - #LOUDPackGorilla - #LPMG - #FreeDward \nHTX.", "followers_count": 957, "friends_count": 1031, "statues_count": 74501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:00.000Z"), "id": 669213894175498240, "text": "GET REAL OBAMA!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4195092803, "name": "Sherry P. Rogers", "screen_name": "SherryPRogers2", "lang": "en", "location": "null", "create_at": date("2015-11-15"), "description": "Recently divorced after 36 yrs. marriage.... STARTING OVER!", "followers_count": 9, "friends_count": 41, "statues_count": 18 }, "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("2015-11-24T10:00:00.000Z"), "id": 669213894276218880, "text": "I'm at Shenandoah Joe Coffee in Charlottesville, VA https://t.co/qa7LjcxSDd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.51696968,38.04493956"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17949577, "name": "Manish Rathaur", "screen_name": "Manish_Rathaur", "lang": "en", "location": "Kuala Lumpur", "create_at": date("2008-12-07"), "description": "Consultant by profession lah, lighthearted impression lah, learning is obsession lah, stargazing fixation lah, blah, blah, blah.", "followers_count": 198, "friends_count": 436, "statues_count": 1245 }, "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("2015-11-24T10:00:00.000Z"), "id": 669213894905167872, "text": "@tarahoward75 ahahhahah best moment of my life", "in_reply_to_status": 669213755583148032, "in_reply_to_user": 805799534, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 805799534 }}, "user": { "id": 338604187, "name": "Riley O'Hara", "screen_name": "rileyohara1", "lang": "en", "location": "null", "create_at": date("2011-07-19"), "description": "null", "followers_count": 706, "friends_count": 834, "statues_count": 4802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:00.000Z"), "id": 669213895014223872, "text": "Anyone wanna go to south center mall 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": 2307924181, "name": "нαℓєιgн мιℓℓєя", "screen_name": "HaleighMiller94", "lang": "en", "location": "Washington, USA", "create_at": date("2014-01-23"), "description": "null", "followers_count": 113, "friends_count": 104, "statues_count": 1559 }, "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("2015-11-24T10:00:00.000Z"), "id": 669213895219929088, "text": "My personal favorite https://t.co/UH4blMALFb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18534565, "name": "danm scooter", "screen_name": "Baibaizoon", "lang": "en", "location": "yes", "create_at": date("2009-01-01"), "description": "jo jo biazzare adventure", "followers_count": 216, "friends_count": 219, "statues_count": 40553 }, "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-24T10:00:00.000Z"), "id": 669213895257493504, "text": "Ole women is messy in the hair salon ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 609485048, "name": "B R E ♥️", "screen_name": "BreyonnaRaney1", "lang": "en", "location": "Lake Charles , Louisiana ", "create_at": date("2012-06-15"), "description": "Snapchat: @ayoo_brezzy", "followers_count": 1059, "friends_count": 938, "statues_count": 7104 }, "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-24T10:00:00.000Z"), "id": 669213895400132608, "text": "Mornings filled with love and laughter�� https://t.co/b3dmbXvZvN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 580393389, "name": "Мαⅾⅾy", "screen_name": "Maddymms8", "lang": "en", "location": "CO ➳ AZ ", "create_at": date("2012-05-14"), "description": "Be devoted to one another in love. Romans 12:10", "followers_count": 605, "friends_count": 549, "statues_count": 5939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Golden, CO", "id": "b80654579d3abed8", "name": "Golden", "place_type": "city", "bounding_box": rectangle("-105.297836,39.698814 -105.161741,39.787281") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8059, "countyName": "Jefferson", "cityID": 830835, "cityName": "Golden" } }
+{ "create_at": datetime("2015-11-24T10:00:00.000Z"), "id": 669213895475789824, "text": "Rpm class ✔️. Crossfit ✔️", "in_reply_to_status": 669185363320115200, "in_reply_to_user": 393675579, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 393675579, "name": "Makenzie Clark", "screen_name": "makclark7", "lang": "en", "location": "null", "create_at": date("2011-10-18"), "description": "state champion ⚽️ GHS varsity cheerleader", "followers_count": 756, "friends_count": 452, "statues_count": 8316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winder, GA", "id": "44625785811504ba", "name": "Winder", "place_type": "city", "bounding_box": rectangle("-83.766197,33.954448 -83.680453,34.030116") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13013, "countyName": "Barrow", "cityID": 1383420, "cityName": "Winder" } }
+{ "create_at": datetime("2015-11-24T10:00:00.000Z"), "id": 669213895509307392, "text": "https://t.co/erpuiwPFXw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2769454761, "name": "colynoscopy", "screen_name": "colkirkwood", "lang": "en", "location": "ellicott city, maryland", "create_at": date("2014-09-13"), "description": "say I'm worth it", "followers_count": 294, "friends_count": 355, "statues_count": 2123 }, "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("2015-11-24T10:00:00.000Z"), "id": 669213895542702080, "text": "ain't missing a beat ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2423650946, "name": "Alana Glaspie", "screen_name": "alanalanae_", "lang": "en", "location": "Lake Charles, LA", "create_at": date("2014-04-02"), "description": "beauty is from within", "followers_count": 1333, "friends_count": 2094, "statues_count": 55298 }, "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("2015-11-24T10:00:00.000Z"), "id": 669213895811203072, "text": "My memory loves you, it asks about you 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": 911140028, "name": "Maria Vega", "screen_name": "mariavega1416", "lang": "en", "location": "null", "create_at": date("2012-10-28"), "description": "null", "followers_count": 85, "friends_count": 303, "statues_count": 411 }, "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("2015-11-24T10:00:00.000Z"), "id": 669213896104742912, "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": 325280106, "name": "Dαяκ Tεмρтαтıσп", "screen_name": "Authentic_Boss", "lang": "en", "location": "On The Block ; 18th & Mad.", "create_at": date("2011-06-27"), "description": "#TeamDarkSkinned | Class Of '14 Graduate | Follow Me On Instagram @Authentic_Boss", "followers_count": 888, "friends_count": 1239, "statues_count": 15200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Junction City, KS", "id": "36f8233528f9c1cf", "name": "Junction City", "place_type": "city", "bounding_box": rectangle("-96.891637,38.993428 -96.800328,39.067418") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20061, "countyName": "Geary", "cityID": 2035750, "cityName": "Junction City" } }
+{ "create_at": datetime("2015-11-24T10:00:00.000Z"), "id": 669213896209645568, "text": "Is everyone voting for Jensen? Time is running out... We gotta do this... https://t.co/IlrzzfJhbm https://t.co/wc6JwnA9p1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25807265, "name": "ღ♥ DEANA ♥ღ", "screen_name": "Deana_Deee", "lang": "en", "location": "Kansas City", "create_at": date("2009-03-22"), "description": "Hopelessly In Love with @JensenAckles Die Hard Supernatural Fan, Totally Obsessed #DeanGirl And I don't SHIP ANYONE. Deal With It...", "followers_count": 599, "friends_count": 346, "statues_count": 24182 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bonner Springs, KS", "id": "6ca2f5432d659026", "name": "Bonner Springs", "place_type": "city", "bounding_box": rectangle("-94.908753,39.047837 -94.843059,39.096687") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20209, "countyName": "Wyandotte", "cityID": 2007975, "cityName": "Bonner Springs" } }
+{ "create_at": datetime("2015-11-24T10:00:00.000Z"), "id": 669213896213925888, "text": "@MKBHD @USAudioTechnica y'all recommend a decent cable for these that includes inline mic / volume / play-pause? only thing missing.", "in_reply_to_status": 668567505703710720, "in_reply_to_user": 413534830, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 29873662, 509140285 }}, "user": { "id": 413534830, "name": "Eric Moskowitz", "screen_name": "eric_moz", "lang": "en", "location": "uws_ny", "create_at": date("2011-11-15"), "description": "null", "followers_count": 68, "friends_count": 77, "statues_count": 646 }, "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("2015-11-24T10:00:00.000Z"), "id": 669213896465620993, "text": "Our amazing Thanksgiving Feast last week!! Thankful for all our amazing students & their families! https://t.co/TvPUh7Ox1z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 108180045, "name": "The Lionheart School", "screen_name": "LionheartSchool", "lang": "en", "location": "Alpharetta, GA", "create_at": date("2010-01-24"), "description": "Non-profit school that provides a developmentally appropriate education for children who need small classes to support individual profiles and academic goals", "followers_count": 273, "friends_count": 266, "statues_count": 696 }, "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("2015-11-24T10:00:00.000Z"), "id": 669213896528543747, "text": "@_GanjaxGrxzzly im done talking to you im currently starving myself to loose weight bye im going to bed", "in_reply_to_status": 669213610112077825, "in_reply_to_user": 386246419, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 386246419 }}, "user": { "id": 1692651618, "name": "〽 D2Saucy 〽", "screen_name": "BigD_138", "lang": "en", "location": "grindin no matter where i am. ", "create_at": date("2013-08-22"), "description": "[CCHS SENIOR]\n[#2SaucyGang]\n[CCHS WRESTLING] \n[CCHS FOOTBALL #42]", "followers_count": 452, "friends_count": 568, "statues_count": 15523 }, "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("2015-11-24T10:00:00.000Z"), "id": 669213896541134848, "text": "This is what's going on right now ‼️ thanksgiving day cuts #fadetowin #barbersinctv… https://t.co/e5v4IrXAam", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.24985617,39.77958612"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "fadetowin", "barbersinctv" }}, "user": { "id": 35930887, "name": "✂KJ HICKS✂oo", "screen_name": "DreadfullyTattd", "lang": "en", "location": "Atl", "create_at": date("2009-04-27"), "description": "KEEP GOD FIRST! 24, father, licensed barber \n3172249971.. .. Follow me on instagram @dreadfullytatted", "followers_count": 1141, "friends_count": 607, "statues_count": 19718 }, "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("2015-11-24T10:00:01.000Z"), "id": 669213896784420865, "text": "Art is meant to cause an emotional reaction... If you don't agree with an artwork, or if it makes you angry, sad, or if 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": 2209415827, "name": "Katelyn Luce", "screen_name": "KateLucePhoto", "lang": "en", "location": "Long Island, NY", "create_at": date("2013-11-22"), "description": "Photographer & Food Lover | Snapchat: Kateluce | Instagram: katelynluce | For business inquiries: katelynlucephoto@gmail.com", "followers_count": 58, "friends_count": 87, "statues_count": 1377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Southold, NY", "id": "ab926bdc069980bb", "name": "Southold", "place_type": "city", "bounding_box": rectangle("-72.463022,41.030129 -72.389069,41.0984") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3669452, "cityName": "Southold" } }
+{ "create_at": datetime("2015-11-24T10:00:01.000Z"), "id": 669213897275084801, "text": "Things were so different a year ago", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 901028947, "name": "Risa Villaseñor", "screen_name": "risa_jlym", "lang": "en", "location": "Crown Point, IN", "create_at": date("2012-10-23"), "description": "God damn it homie", "followers_count": 434, "friends_count": 306, "statues_count": 13985 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salisbury, MD", "id": "00caf39d503a84e9", "name": "Salisbury", "place_type": "city", "bounding_box": rectangle("-75.71412,38.28923 -75.487032,38.431613") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24045, "countyName": "Wicomico", "cityID": 2469925, "cityName": "Salisbury" } }
+{ "create_at": datetime("2015-11-24T10:00:01.000Z"), "id": 669213897371480064, "text": "16284726cf56d928a2382a9542d77717c423f7f466d7cbda5828a3f239a4a589838dee02f56ef081284f8723f4ce3fc086bbf1ee2f096aa53b5516ea5c2301265cda70000000", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-133.8700191,55.56302372"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3048544857, "name": "GooGuns Lulz", "screen_name": "googuns_lulz", "lang": "en", "location": "(here)", "create_at": date("2015-02-20"), "description": "null", "followers_count": 112, "friends_count": 1, "statues_count": 595012 }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2015-11-24T10:00:01.000Z"), "id": 669213897623281665, "text": "11/24@13:00 - Temp 49.7F, WC 49.3F. Wind 3.0mph NW, Gust 10.0mph. Bar 30.367in, Falling slowly. Rain 0.00in. Hum 49%. UV 1.4. SolarRad 162.", "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": 45045 }, "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("2015-11-24T10:00:01.000Z"), "id": 669213897870700545, "text": "Until we outta school", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3774259342, "name": "BRONYYYYYYYYYYYYYYYY", "screen_name": "SexyGrassi", "lang": "en", "location": "Bronx, NY", "create_at": date("2015-09-25"), "description": "Multifandom - I love making friends,i follow back - but Im also insane XD - I joined @ 12:00am Sep 25 2015 - my @ is random asf", "followers_count": 1177, "friends_count": 3264, "statues_count": 8284 }, "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("2015-11-24T10:00:01.000Z"), "id": 669213898134958084, "text": "@Sydneylesaraa this is why we can't use Wikipedia for our papers������", "in_reply_to_status": 669212811197726720, "in_reply_to_user": 224819963, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 224819963 }}, "user": { "id": 64129850, "name": "$ V O Landen", "screen_name": "TheFlash84_", "lang": "en", "location": "Beaumont, TX✈Baltimore, MD", "create_at": date("2009-08-09"), "description": "$ V O|Morgan State Univ |Football | WR #85| #TheBelievers #KobeNumber8| Click The Link In My Bio | RIP Grandmother| Booking & Features: Lamal2@morgan.edu|", "followers_count": 2144, "friends_count": 1504, "statues_count": 48615 }, "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-24T10:00:01.000Z"), "id": 669213898227228672, "text": "what's for breakfast bruh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2757491777, "name": "stick", "screen_name": "BrokeNiggaTyler", "lang": "en", "location": "South Carolina", "create_at": date("2014-08-31"), "description": "6'6 | $$$$$", "followers_count": 1823, "friends_count": 501, "statues_count": 29446 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Sumter, SC", "id": "08e6f7a918a40e31", "name": "South Sumter", "place_type": "city", "bounding_box": rectangle("-80.352336,33.866788 -80.315737,33.906765") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45085, "countyName": "Sumter", "cityID": 4568177, "cityName": "South Sumter" } }
+{ "create_at": datetime("2015-11-24T10:00:01.000Z"), "id": 669213898273239040, "text": "My orders came in!!! ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 582562602, "name": "Elisabeth W.", "screen_name": "Lisa_Koma", "lang": "en", "location": "Sioux Falls, SD", "create_at": date("2012-05-16"), "description": "peace starts within the soul", "followers_count": 1530, "friends_count": 972, "statues_count": 22379 }, "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("2015-11-24T10:00:01.000Z"), "id": 669213898424348676, "text": "Así es. ���� https://t.co/P5cxn8FrTE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 874637874, "name": "♡", "screen_name": "KarlaFOrtuna1", "lang": "es", "location": "Salem, MA", "create_at": date("2012-10-11"), "description": "God♡", "followers_count": 1388, "friends_count": 870, "statues_count": 85276 }, "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("2015-11-24T10:00:01.000Z"), "id": 669213898440970241, "text": "@MrTitleist haha without showing any disrespect to the prestigious trophy...I'll just say it's \"One-of-Kind\"", "in_reply_to_status": 669190961348648961, "in_reply_to_user": 33262779, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 33262779 }}, "user": { "id": 313599565, "name": "Dominic Shelden", "screen_name": "Dominic_Shelden", "lang": "en", "location": "Missoula, Montana", "create_at": date("2011-06-08"), "description": "Sports Director for ABC/FOX airing in Missoula/Bozeman/Butte/Kallispell, etc. Host of Grizzly Sports Report // \n\nBoise, ID Native-Alum of WSU #GoCougs", "followers_count": 1042, "friends_count": 607, "statues_count": 10568 }, "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("2015-11-24T10:00:01.000Z"), "id": 669213898537631744, "text": "@EmazingLights help me out i bought this thing but it doesnt have bulbs . I only have 9/10 lights and now im screwed https://t.co/QrO7i7tFYw", "in_reply_to_status": -1, "in_reply_to_user": 161511739, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 161511739 }}, "user": { "id": 2820952595, "name": "յժ", "screen_name": "lerbigdickla", "lang": "en", "location": "null", "create_at": date("2014-10-09"), "description": "do u feel the buzz or is it just me", "followers_count": 212, "friends_count": 328, "statues_count": 5003 }, "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("2015-11-24T10:00:01.000Z"), "id": 669213898638249985, "text": "https://t.co/2eRWblNRR6.much. ❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3783872537, "name": "Kayla Barnes", "screen_name": "kabarnes1211", "lang": "en", "location": "null", "create_at": date("2015-09-26"), "description": "work so hard that one day your signature will be called an autograph.", "followers_count": 33, "friends_count": 35, "statues_count": 61 }, "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("2015-11-24T10:00:01.000Z"), "id": 669213898839425029, "text": "@REDBREAKINGNEWS Give me rest of my term and the next two", "in_reply_to_status": 669212406820634624, "in_reply_to_user": 3525238277, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3525238277 }}, "user": { "id": 853350110, "name": "marnel", "screen_name": "MarnelMn", "lang": "en", "location": "null", "create_at": date("2012-09-29"), "description": "null", "followers_count": 1268, "friends_count": 2063, "statues_count": 17381 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glen Carbon, IL", "id": "e3d941a68af042e7", "name": "Glen Carbon", "place_type": "city", "bounding_box": rectangle("-90.026138,38.738099 -89.92845,38.787788") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17119, "countyName": "Madison", "cityID": 1729639, "cityName": "Glen Carbon" } }
+{ "create_at": datetime("2015-11-24T10:00:01.000Z"), "id": 669213898877353984, "text": "Sittin in class dreaming about all the MASHED POTATOES I'm about to eat Thursday ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1152619838, "name": "Halie", "screen_name": "HalieManzo", "lang": "en", "location": "Columbus, OH", "create_at": date("2013-02-05"), "description": "Meredith Grey in training - OCU #24", "followers_count": 1074, "friends_count": 506, "statues_count": 5936 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Circleville, OH", "id": "bc04497c33fee903", "name": "Circleville", "place_type": "city", "bounding_box": rectangle("-82.961991,39.574744 -82.903716,39.633876") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39129, "countyName": "Pickaway", "cityID": 3915070, "cityName": "Circleville" } }
+{ "create_at": datetime("2015-11-24T10:00:01.000Z"), "id": 669213899162546176, "text": "@GavinEwbank why not?", "in_reply_to_status": 669213839116935168, "in_reply_to_user": 38510754, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 38510754 }}, "user": { "id": 490013684, "name": "Kyle Foley", "screen_name": "notkylefoley", "lang": "en", "location": "behind you", "create_at": date("2012-02-11"), "description": "Writer (@OUTSETmagazine), Rambler (@YoungGunsUSA), American. Not following me is what ISIS wants you to do. #KyleFoleyExperience #FireGavin", "followers_count": 23620, "friends_count": 1001, "statues_count": 2088 }, "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("2015-11-24T10:00:01.000Z"), "id": 669213899296641025, "text": "That's the best thing I've heard 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": 878574031, "name": "Austin Thrailkill", "screen_name": "austkill2015", "lang": "en", "location": "Derby, KS", "create_at": date("2012-10-13"), "description": "#BK13 RIP great grandma smith 8-6-13 jeeps make me happy - O||||||O", "followers_count": 744, "friends_count": 668, "statues_count": 5410 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Derby, KS", "id": "f116fb6cf470e97d", "name": "Derby", "place_type": "city", "bounding_box": rectangle("-97.287293,37.509588 -97.225971,37.591553") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20173, "countyName": "Sedgwick", "cityID": 2017800, "cityName": "Derby" } }
+{ "create_at": datetime("2015-11-24T10:00:01.000Z"), "id": 669213899829460992, "text": "Wind 2.0 mph SW. Barometer 30.382 in, Falling slowly. Temperature 54.0 °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": 37, "friends_count": 25, "statues_count": 15640 }, "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("2015-11-24T10:00:01.000Z"), "id": 669213899913195520, "text": "BI-LO #Retail #Job: Deli Associate (#GAFFNEY, SC) https://t.co/Qii0DaGM9V #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.658061,35.0820099"), "retweet_count": 0, "lang": "it", "is_retweet": false, "hashtags": {{ "Retail", "Job", "GAFFNEY", "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": 5, "friends_count": 0, "statues_count": 12116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gaffney, SC", "id": "a3b5533ca11e0324", "name": "Gaffney", "place_type": "city", "bounding_box": rectangle("-81.719658,35.03446 -81.599161,35.172674") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45021, "countyName": "Cherokee", "cityID": 4528060, "cityName": "Gaffney" } }
+{ "create_at": datetime("2015-11-24T10:00:01.000Z"), "id": 669213899913363456, "text": "People everywhere hate everyone. Just more so when the people are more different. \n@SonofBaldwin", "in_reply_to_status": 669213149594312704, "in_reply_to_user": 68003308, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 68003308 }}, "user": { "id": 2556904464, "name": "Ia(i)n Osbo(u)rn(e)", "screen_name": "IanAlekzi", "lang": "en", "location": "Lincoln, NE", "create_at": date("2014-06-09"), "description": "Nebraska. Knee. Bra. Ska. Think about it.", "followers_count": 188, "friends_count": 509, "statues_count": 611 }, "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("2015-11-24T10:00:01.000Z"), "id": 669213900076773377, "text": "I'm tired as shit idk why", "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": "Young Sinatra", "screen_name": "beffo_joe", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2014-06-30"), "description": "Views From The B || 约瑟夫 || Founder of The Struggle Co.", "followers_count": 394, "friends_count": 265, "statues_count": 23612 }, "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("2015-11-24T10:00:01.000Z"), "id": 669213900152303616, "text": "Manager, Process Operations - Cenpatico National: (#Austin, TX) https://t.co/09hkMLYk2r #Insurance #Job #Jobs #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": {{ "Austin", "Insurance", "Job", "Jobs", "Hiring" }}, "user": { "id": 302104655, "name": "Austin Insurance Job", "screen_name": "tmj_aus_ins", "lang": "en", "location": "Austin, TX", "create_at": date("2011-05-20"), "description": "Follow this account for geo-targeted Insurance job tweets in Austin, TX from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 148, "friends_count": 115, "statues_count": 61 }, "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-24T10:00:01.000Z"), "id": 669213900223574016, "text": "University of Maryland Medical S...: Primary Care Outpatient Parkton - UM St.... (#Parkton, MD) https://t.co/Y8077ZSQph #FamilyPractice", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.665284,39.5912298"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Parkton", "FamilyPractice" }}, "user": { "id": 72455389, "name": "MD Physician Jobs", "screen_name": "MD_physician", "lang": "en", "location": "Maryland", "create_at": date("2009-09-07"), "description": "Follow this account for geo-targeted Physician job tweets in Maryland Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 148, "friends_count": 134, "statues_count": 115 }, "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("2015-11-24T10:00:01.000Z"), "id": 669213900601081856, "text": "@ashonasty get cash a camera man", "in_reply_to_status": -1, "in_reply_to_user": 2472712617, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2472712617 }}, "user": { "id": 3140899748, "name": "Christian [XB1]", "screen_name": "Christian_SPURZ", "lang": "en", "location": "San Bernardino, CA", "create_at": date("2015-04-05"), "description": "When you want to succeed as bad as you want to breathe then you will be successful-Eric Thomas #88 #83", "followers_count": 38, "friends_count": 44, "statues_count": 783 }, "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("2015-11-24T10:00:02.000Z"), "id": 669213900827590656, "text": "Okay Like But Never Replied. Nice.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3897429432, "name": "Alice", "screen_name": "AlisonNunez27", "lang": "en", "location": "Pasadena", "create_at": date("2015-10-14"), "description": "Nothing Last Forever.", "followers_count": 129, "friends_count": 117, "statues_count": 1967 }, "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("2015-11-24T10:00:02.000Z"), "id": 669213900840243201, "text": "Revolution Deth's Tar Imperial Stout 2015 now available in bottles. https://t.co/52GqMKhPAY @revbrewchicago #BeerMenus", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.704492,41.81263"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BeerMenus" }}, "user_mentions": {{ 110514150 }}, "user": { "id": 1362333199, "name": "Liquorama Chicago", "screen_name": "LiquoramaChicag", "lang": "en", "location": "4430 S. Kedzie Chicago, IL", "create_at": date("2013-04-18"), "description": "Liquorama Wine and Spirits is a family owned and operated business that has been serving the Chicagoland area for over 40 years!", "followers_count": 193, "friends_count": 205, "statues_count": 2151 }, "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-24T10:00:02.000Z"), "id": 669213900907352064, "text": "Duh ✨ https://t.co/DZrAw1mH3O", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 32168365, "name": "Lil'Fairy✨", "screen_name": "YvngNino", "lang": "en", "location": "w/@Blvckmoses, FairytaleLand", "create_at": date("2009-04-16"), "description": "#704|21|big browneyed unicorn | ECU senior | YC⚓️| RIP Ang&RayRay ❤️✨", "followers_count": 1273, "friends_count": 388, "statues_count": 33847 }, "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("2015-11-24T10:00:02.000Z"), "id": 669213900940967937, "text": "Wind 4.0 mph SE. Barometer 30.234 in, Falling. Temperature 59.8 °F. Rain today 0.00 in. Humidity 42%", "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": 116, "friends_count": 0, "statues_count": 109100 }, "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("2015-11-24T10:00:02.000Z"), "id": 669213901184221184, "text": "Wind 2.9 mph SW. Barometer 30.36 in, Falling slowly. Temperature 56.1 °F. Rain today 0.00 in. Humidity 41%", "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": 79, "friends_count": 17, "statues_count": 300519 }, "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("2015-11-24T10:00:02.000Z"), "id": 669213901603651584, "text": "Wind 2.0 mph W. Barometer 30.329 in, Falling. Temperature 47.3F. Rain today 0.00 in. Humidity 45% | 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": 562, "friends_count": 759, "statues_count": 34088 }, "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("2015-11-24T10:00:02.000Z"), "id": 669213901683351552, "text": "Wind 0.0 mph ---. Barometer 30.34 in, Falling slowly. Temperature 57.2 °F. Rain today 0.00 in. Humidity 42%", "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": 3, "statues_count": 19129 }, "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("2015-11-24T10:00:02.000Z"), "id": 669213901721100288, "text": "@Applebees nothing I'm hungry lol", "in_reply_to_status": 669213242133053440, "in_reply_to_user": 74452613, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 74452613 }}, "user": { "id": 325183872, "name": "cass", "screen_name": "C_boo28", "lang": "en", "location": "the trap", "create_at": date("2011-06-27"), "description": "rwg daddy.❤️ 10.9.15 ❤️", "followers_count": 526, "friends_count": 732, "statues_count": 7884 }, "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("2015-11-24T10:00:02.000Z"), "id": 669213902454919168, "text": "I might just be gassin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1326010188, "name": "BigGucciSosa", "screen_name": "rxphillip", "lang": "en", "location": "Long Beach, CA", "create_at": date("2013-04-03"), "description": "19 and off some xannies. Part Time Producer.", "followers_count": 1892, "friends_count": 1541, "statues_count": 18309 }, "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("2015-11-24T10:00:02.000Z"), "id": 669213902518034440, "text": "Can't help but wait ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 324646381, "name": "Mia ♡", "screen_name": "TrillBeauty_x3", "lang": "en", "location": "null", "create_at": date("2011-06-26"), "description": "h o p e l e s s f o r e i g n r o m a n t i c", "followers_count": 905, "friends_count": 885, "statues_count": 41452 }, "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("2015-11-24T10:00:02.000Z"), "id": 669213902580793344, "text": "We're #hiring! Click to apply: Staff Physician - https://t.co/g0ozxAKykY #Job #LOGAN, OH #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.4284292,39.5471633"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Job", "LOGAN", "Jobs" }}, "user": { "id": 1860798708, "name": "JoinTeamHealth", "screen_name": "JoinTeamHealth", "lang": "en", "location": "null", "create_at": date("2013-09-13"), "description": "One of nation's largest providers of Emergency Medicine,Hospital Medicine,Anesthesia,Acute Care Surgery,OB/GYN Hospitalist & Orthopaedic Hospitalists services.", "followers_count": 335, "friends_count": 122, "statues_count": 2239 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Logan, OH", "id": "165c36c2b6d70b57", "name": "Logan", "place_type": "city", "bounding_box": rectangle("-82.440948,39.51934 -82.365573,39.553447") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39073, "countyName": "Hocking", "cityID": 3944632, "cityName": "Logan" } }
+{ "create_at": datetime("2015-11-24T10:00:02.000Z"), "id": 669213902639501313, "text": "Wind 0.4 mph WSW. Barometer 30.171 in, Falling. Temperature 35.8 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.98388889,47.66361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 67603939, "name": "Bemidji Weather", "screen_name": "BemidjiWx", "lang": "en", "location": "Bemidji, MN, USA", "create_at": date("2009-08-21"), "description": "Local Bemidji Weather", "followers_count": 274, "friends_count": 134, "statues_count": 94577 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minnesota, USA", "id": "9807c5c5f7a2c6ce", "name": "Minnesota", "place_type": "admin", "bounding_box": rectangle("-97.239256,43.499362 -89.483385,49.384359") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27007, "countyName": "Beltrami" } }
+{ "create_at": datetime("2015-11-24T10:00:02.000Z"), "id": 669213902794698753, "text": "@natalyyRivera7 dam you need to get a screen protector !", "in_reply_to_status": 669213768765706240, "in_reply_to_user": 2365179410, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2365179410 }}, "user": { "id": 2952894092, "name": "Princesita Sofi", "screen_name": "_Cecicampos", "lang": "en", "location": "null", "create_at": date("2014-12-30"), "description": "D.E.P Lalo Lara 02/23/95-08/10/14 ❤", "followers_count": 84, "friends_count": 82, "statues_count": 2209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cornelius, OR", "id": "385cc581ec7b6fcc", "name": "Cornelius", "place_type": "city", "bounding_box": rectangle("-123.070798,45.507172 -123.022844,45.528366") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4115550, "cityName": "Cornelius" } }
+{ "create_at": datetime("2015-11-24T10:00:02.000Z"), "id": 669213903033917440, "text": "@whateverfener jqtyuiopslfksjfnn�� https://t.co/lWFWuMNMby", "in_reply_to_status": -1, "in_reply_to_user": 3011533322, "favorite_count": 0, "retweet_count": 0, "lang": "no", "is_retweet": false, "user_mentions": {{ 3011533322 }}, "user": { "id": 3485983633, "name": "andrea", "screen_name": "muchixxx", "lang": "en", "location": "MIA", "create_at": date("2015-09-07"), "description": "je déteste t'aimer", "followers_count": 184, "friends_count": 149, "statues_count": 1021 }, "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("2015-11-24T10:00:02.000Z"), "id": 669213903071514624, "text": "A new shirt to wear around town to get involved. Is it too PC? @Bellanieve @Freak1791 @steph93065 @DavesBigWife \nhttps://t.co/ngmo5304nT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20198802, 1436047242, 261749735, 3059352001 }}, "user": { "id": 2831374452, "name": "Humans AgainstIslam", "screen_name": "DonAgainstislam", "lang": "en", "location": "Everywhere", "create_at": date("2014-09-25"), "description": "Conservative with edgy sarcastic POV on issues, I hate islam, I Honor Vets. HARDLINE STANCE. If ur easily insulted get lost, NOT PC. Edgy Humor", "followers_count": 4752, "friends_count": 2415, "statues_count": 42016 }, "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("2015-11-24T10:00:02.000Z"), "id": 669213903176339456, "text": "Happy Birthday bro, have a bad day.�� @OscarQqqq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1352747502 }}, "user": { "id": 3288061040, "name": "Marcos", "screen_name": "marcospineda96", "lang": "en", "location": "null", "create_at": date("2015-07-22"), "description": "#UT19", "followers_count": 84, "friends_count": 73, "statues_count": 36 }, "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("2015-11-24T10:00:02.000Z"), "id": 669213903247814656, "text": "@MatthCoyne ur a basterd", "in_reply_to_status": 669208733940101120, "in_reply_to_user": 356388294, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 356388294 }}, "user": { "id": 522211285, "name": "Liam Reilly", "screen_name": "lreilly03", "lang": "en", "location": "null", "create_at": date("2012-03-12"), "description": "If you aren't cute, you may as well be clever.", "followers_count": 128, "friends_count": 200, "statues_count": 4232 }, "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("2015-11-24T10:00:02.000Z"), "id": 669213903465771008, "text": "Where is Kitty Hawk on the map? Play the game at https://t.co/TWzFz3bzo1 #Kitty Hawk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.7057,36.0646"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Kitty" }}, "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": 937, "friends_count": 312, "statues_count": 2434908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kitty Hawk, NC", "id": "54ea8e51bae44578", "name": "Kitty Hawk", "place_type": "city", "bounding_box": rectangle("-75.722982,36.052565 -75.682025,36.101634") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37055, "countyName": "Dare", "cityID": 3736060, "cityName": "Kitty Hawk" } }
+{ "create_at": datetime("2015-11-24T10:00:02.000Z"), "id": 669213903717449728, "text": "Good morning, Laupahoehoe, HI. #UnitedStates https://t.co/vs91Xqg3TN https://t.co/m3V1Ichx54", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-155.2361145,19.98472214"), "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "UnitedStates" }}, "user": { "id": 17111948, "name": "BigZ", "screen_name": "ioDracy", "lang": "zh-cn", "location": "HangZhou", "create_at": date("2008-11-02"), "description": "@Hangzhou China,Android user,Google fan,Geeker,Blogger", "followers_count": 211, "friends_count": 223, "statues_count": 3764 }, "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", "cityID": 1544150, "cityName": "Laupahoehoe" } }
+{ "create_at": datetime("2015-11-24T10:00:02.000Z"), "id": 669213903834869760, "text": "We're #hiring! Read about our latest #job opening here: Driver Helper - https://t.co/BztmiykxSL #SPIRITLAKE, IA #Transportation #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.111409,43.422766"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "SPIRITLAKE", "Transportation", "CareerArc" }}, "user": { "id": 149711487, "name": "TMJ-IA Transport.", "screen_name": "tmj_IA_transp", "lang": "en", "location": "Indiana", "create_at": date("2010-05-29"), "description": "Follow this account for geo-targeted Transportation job tweets in Iowa Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 306, "friends_count": 282, "statues_count": 349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spirit Lake, IA", "id": "68fcc799aefa94c0", "name": "Spirit Lake", "place_type": "city", "bounding_box": rectangle("-95.155131,43.392918 -95.063912,43.430821") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19059, "countyName": "Dickinson", "cityID": 1974415, "cityName": "Spirit Lake" } }
+{ "create_at": datetime("2015-11-24T10:00:02.000Z"), "id": 669213903872593920, "text": "Ok..POTUS says ISIL, the media says ISIS, overseas leaders say DEISH...I say WTF!!!?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3004338055, "name": "h alan glover iii", "screen_name": "halangloveriii", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-01-30"), "description": "Mutant, Morehouse school for Gifted Students, Graffiti artist, Solution Giver, 6x a wk workout, Listener, Remover of Stress...Harlem Original.", "followers_count": 247, "friends_count": 658, "statues_count": 494 }, "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("2015-11-24T10:00:02.000Z"), "id": 669213904048750592, "text": "If I had a dollar for every time my parents didn't answer 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": 1365066379, "name": "FR/\\NKLIN", "screen_name": "JacobFranklinn", "lang": "en", "location": "null", "create_at": date("2013-04-19"), "description": "i like sports, a lot.", "followers_count": 150, "friends_count": 122, "statues_count": 3647 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-24T10:00:02.000Z"), "id": 669213904120053761, "text": "Wind 1.0 mph SSE. Barometer 30.530 in, Falling. Temperature 52.8 °F. Rain today 0.00 in. Humidity 27%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-106.69694444,34.80694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14656738, "name": "Andrew Madden", "screen_name": "AbqDrew", "lang": "en", "location": "Los Lunas, New Mexico", "create_at": date("2008-05-04"), "description": "31 year old atheist conservative living in Albuquerque, New Mexico. KCCO", "followers_count": 175, "friends_count": 186, "statues_count": 4836 }, "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("2015-11-24T10:00:02.000Z"), "id": 669213904388665344, "text": "@JIDsv @EarthGang @KillerMike @ThatBoyCurtis @1Strado (atl by way of sc) https://t.co/tpIaIs2rFj", "in_reply_to_status": -1, "in_reply_to_user": 108314159, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 108314159, 128418853, 21265120, 40600309, 168025492 }}, "user": { "id": 210293174, "name": "Gerald Johanssen", "screen_name": "TightToDef", "lang": "en", "location": "Firefly class Serenity", "create_at": date("2010-10-30"), "description": "I am Endor the Landstander Also I'm just a over Hyped Hipster.#CSU Peace+Power #ButterflyJenkins lifestyle", "followers_count": 151, "friends_count": 187, "statues_count": 12484 }, "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("2015-11-24T10:00:02.000Z"), "id": 669213904682250241, "text": "@caitlynhancock_ happy birthday' ��", "in_reply_to_status": -1, "in_reply_to_user": 1035802650, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1035802650 }}, "user": { "id": 2745853893, "name": "Bo$sŸ DÕs$y‼️", "screen_name": "damndarcie", "lang": "en", "location": "Larose, LA", "create_at": date("2014-08-17"), "description": "boo bitch", "followers_count": 691, "friends_count": 784, "statues_count": 5114 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Larose, LA", "id": "2c1fc4e6162b9e9a", "name": "Larose", "place_type": "city", "bounding_box": rectangle("-90.411556,29.540492 -90.323551,29.596034") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22057, "countyName": "Lafourche", "cityID": 2242135, "cityName": "Larose" } }
+{ "create_at": datetime("2015-11-24T10:00:02.000Z"), "id": 669213904711602177, "text": "Pollen love hotel @ Winter Park Central Park https://t.co/OLHnMzIBGt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3511734,28.5975704"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 223935062, "name": "Angelo Villagomez", "screen_name": "taotaotasi", "lang": "en", "location": "Washington, DC", "create_at": date("2010-12-07"), "description": "Make the Ocean Great Again! Godfather of the Mariana Trench Monument. Saipan's most popular blogger since ever since. @PewTrusts ocean advocate. Tweets my own.", "followers_count": 14111, "friends_count": 390, "statues_count": 15376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winter Park, FL", "id": "c87bc3adb59d807f", "name": "Winter Park", "place_type": "city", "bounding_box": rectangle("-81.375805,28.567893 -81.307654,28.632478") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1278300, "cityName": "Winter Park" } }
+{ "create_at": datetime("2015-11-24T10:00:02.000Z"), "id": 669213904799580160, "text": "@AinaKhiam Thanks for following :-)", "in_reply_to_status": -1, "in_reply_to_user": 3014522295, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3014522295 }}, "user": { "id": 724508209, "name": "Laura Nestor", "screen_name": "LauraNestor4", "lang": "en", "location": "L.A.Ca / Las Vegas, Nv USA", "create_at": date("2012-07-29"), "description": "60's child, Flower power Peace Love & enjoy many music genre,retired USPS Pls don't send porn to my DM", "followers_count": 7982, "friends_count": 7899, "statues_count": 63856 }, "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("2015-11-24T10:00:02.000Z"), "id": 669213904841613312, "text": "@BestComedyVine @Maggie_David01 we have to do this to Chad.", "in_reply_to_status": 669018539970461696, "in_reply_to_user": 329418923, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 329418923, 449288400 }}, "user": { "id": 758244295, "name": "missid", "screen_name": "Missidavid", "lang": "en", "location": "null", "create_at": date("2012-08-14"), "description": "null", "followers_count": 58, "friends_count": 122, "statues_count": 108 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Auburn, NY", "id": "f8e6f4aa8e4393c0", "name": "Auburn", "place_type": "city", "bounding_box": rectangle("-76.614535,42.898722 -76.522688,42.997447") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36011, "countyName": "Cayuga", "cityID": 3603078, "cityName": "Auburn" } }
+{ "create_at": datetime("2015-11-24T10:00:03.000Z"), "id": 669213905135144961, "text": "We're #hiring! Read about our latest #job opening here: Retail Store Positions - https://t.co/82fSkW3wVZ #Springfield, MO #Retail", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.26327,37.183577"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Springfield", "Retail" }}, "user": { "id": 563319506, "name": "CVS Health Jobs", "screen_name": "CVSHealthJobs", "lang": "en", "location": "null", "create_at": date("2012-04-25"), "description": "null", "followers_count": 426, "friends_count": 3, "statues_count": 53098 }, "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("2015-11-24T10:00:03.000Z"), "id": 669213905789562881, "text": "Re downloading Tinder when your top draft prospect turns out to be a bust. #CuffingSeason", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CuffingSeason" }}, "user": { "id": 159198923, "name": "Tim Cowpens", "screen_name": "Josef_Malkovich", "lang": "en", "location": "33.964352,-81.016493", "create_at": date("2010-06-24"), "description": "What kind of person woukd I be if I wore True Religion jeans and a Gucci belt?", "followers_count": 413, "friends_count": 576, "statues_count": 26478 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sumter, SC", "id": "cc3b4679df06d7c4", "name": "Sumter", "place_type": "city", "bounding_box": rectangle("-80.501997,33.863735 -80.303369,34.011051") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45085, "countyName": "Sumter", "cityID": 4570405, "cityName": "Sumter" } }
+{ "create_at": datetime("2015-11-24T10:00:03.000Z"), "id": 669213905927933952, "text": "cloudy -> showers\nhumidity up 93% -> 97%\nwind 3mph -> 0mph\npressure 29.81in rising\nvisibility 3mi -> 4mi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.76578,45.42508"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 216549316, "name": "Tigard Weather", "screen_name": "Tigard", "lang": "en", "location": "Tigard, OR", "create_at": date("2010-11-16"), "description": "Weather updates, forecast, warnings and information for Tigard, OR. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 28, "friends_count": 3, "statues_count": 19894 }, "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("2015-11-24T10:00:03.000Z"), "id": 669213906053632000, "text": "My group just won ������, we had a few haters tho ������.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3287070888, "name": "Sundai♐", "screen_name": "_sundaaai", "lang": "en", "location": "Dallas, TX", "create_at": date("2015-07-21"), "description": "Fmoi: @_sundaaai ☺ | @Drake ❤ | K.H.S Varsity Cheer Co - Captain ✨", "followers_count": 422, "friends_count": 366, "statues_count": 4957 }, "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-24T10:00:03.000Z"), "id": 669213906397741056, "text": "soon as my bro touchdown it's over wit. ❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1184310733, "name": "bèllà.", "screen_name": "prettysamoneee", "lang": "en", "location": "null", "create_at": date("2013-02-15"), "description": "unbothered.. R.I.P. Aries ❤️", "followers_count": 1190, "friends_count": 1292, "statues_count": 27784 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valley, AL", "id": "dd5d995aeda1ccf8", "name": "Valley", "place_type": "city", "bounding_box": rectangle("-85.224667,32.752675 -85.140283,32.856282") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1017, "countyName": "Chambers", "cityID": 178204, "cityName": "Valley" } }
+{ "create_at": datetime("2015-11-24T10:00:03.000Z"), "id": 669213906464804864, "text": "@Ke4DaKill �� wtf you used then? I know not those string bean arms ☺️�� jk", "in_reply_to_status": 669213494005407744, "in_reply_to_user": 547962059, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 547962059 }}, "user": { "id": 449789127, "name": "N ❣ K A", "screen_name": "nikaashay_", "lang": "en", "location": "||FVSU19||", "create_at": date("2011-12-29"), "description": "Perfection is not attainable, but if we chase perfection we can catch excellence. ❤", "followers_count": 656, "friends_count": 831, "statues_count": 2505 }, "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("2015-11-24T10:00:03.000Z"), "id": 669213906548600832, "text": "Can you recommend anyone for this #job? Operations Research Engineer - https://t.co/OrAq7yOiTq #Intel #Phoenix, Arizona #Manufacturing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.0740373,33.4483771"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Intel", "Phoenix", "Manufacturing" }}, "user": { "id": 21683960, "name": "TMJ - PHX Manuf Jobs", "screen_name": "tmj_phx_manuf", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2009-02-23"), "description": "Follow this account for geo-targeted Manufacturing job tweets in Phoenix, AZ from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 300, "friends_count": 203, "statues_count": 87 }, "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-24T10:00:03.000Z"), "id": 669213906569572352, "text": "Supplemental Health Care: Travel RN ER (#Dover, DE) https://t.co/ApSAp47ZDo #Healthcare #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.5243682,39.158168"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Dover", "Healthcare", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 29144075, "name": "TMJ-DED Health Jobs", "screen_name": "tmj_ded_health", "lang": "en", "location": "Northern Delaware", "create_at": date("2009-04-05"), "description": "Follow this account for geo-targeted Healthcare job tweets in Northern Delaware from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 353, "friends_count": 282, "statues_count": 46 }, "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("2015-11-24T10:00:03.000Z"), "id": 669213906674409472, "text": "63.6F (Feels: 63.6F) - Humidity: 60% - Wind: 5.4mph NE - Gust: 9.2mph - Pressure: 1020.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": 215277 }, "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("2015-11-24T10:00:03.000Z"), "id": 669213906695380992, "text": "Want to work at The Cheesecake Factory? We're #hiring in #Albany, NY! Click for details: https://t.co/OGfDzRklql #Hospitality #Veterans", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.8162072,42.7103294"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Albany", "Hospitality", "Veterans" }}, "user": { "id": 22487155, "name": "TMJ-NYA HRTA Jobs", "screen_name": "tmj_nya_hrta", "lang": "en", "location": "Albany, NY", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Albany, NY from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 350, "friends_count": 291, "statues_count": 141 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Latham, NY", "id": "00278f1677a13e8d", "name": "Latham", "place_type": "city", "bounding_box": rectangle("-73.832933,42.676375 -73.694717,42.801868") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36001, "countyName": "Albany", "cityID": 3601000, "cityName": "Albany" } }
+{ "create_at": datetime("2015-11-24T10:00:03.000Z"), "id": 669213906817056768, "text": "Interested in a #Nursing #job near #Alexandria, VA? This could be a great fit: https://t.co/hvyDUkEvd4 #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.0469214,38.8048355"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "Alexandria", "Hiring", "CareerArc" }}, "user": { "id": 120834836, "name": "TMJ-VAL Nursing Jobs", "screen_name": "tmj_VAL_NURSING", "lang": "en", "location": "Alexandria, VA", "create_at": date("2010-03-07"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Alexandria, VA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 370, "friends_count": 282, "statues_count": 122 }, "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("2015-11-24T10:00:03.000Z"), "id": 669213906913665027, "text": "@jodimeadows @octaviabooks I certainly hope so.", "in_reply_to_status": 669212504476639232, "in_reply_to_user": 22779258, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 22779258, 126673157 }}, "user": { "id": 88296327, "name": "Matthew Levine", "screen_name": "MatthewTwihard", "lang": "en", "location": "Kenner,La", "create_at": date("2009-11-07"), "description": "@TheRealGrimmie made 2/12/12 the best day ever by taking a picture with me. @Jordansblog made 3/16/12 and 3/26/12 by replying to me & then following me #CimFam", "followers_count": 2682, "friends_count": 585, "statues_count": 826878 }, "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("2015-11-24T10:00:03.000Z"), "id": 669213906984951808, "text": "@eluniadb @khurum80 @wahid334455 @ViktorMochalin @ValescadeOlive2 https://t.co/HFFuJ5URZl", "in_reply_to_status": 669210911123628032, "in_reply_to_user": 3783098301, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2860544601, 636254553, 1370457428, 2874617801, 952352449 }}, "user": { "id": 3783098301, "name": "Reginalaleman@1246@g", "screen_name": "regialeman51", "lang": "es", "location": "Florida, USA", "create_at": date("2015-09-26"), "description": "me siento bendecida .por tanta gente linda que hay en el mundo .lo..unico es tener \nmucha sabiduría ho ..inteligencia ....o tienes que haber estudiado sicología", "followers_count": 909, "friends_count": 199, "statues_count": 11832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gladeview, FL", "id": "bf9a75b27b940e78", "name": "Gladeview", "place_type": "city", "bounding_box": rectangle("-80.258078,25.830731 -80.209318,25.847033") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1225987, "cityName": "Gladeview" } }
+{ "create_at": datetime("2015-11-24T10:00:03.000Z"), "id": 669213906997411840, "text": "This little girl in here calling everyone bald headed and I'm weak ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 93540841, "name": "Helen Parr", "screen_name": "AlexjandriaSays", "lang": "en", "location": "Cali Bred. D[M]V Resident.", "create_at": date("2009-11-29"), "description": "brown skin lady drinking sake on a suzuki. future cat lady. 2 time cones of dunshire champ.", "followers_count": 332, "friends_count": 316, "statues_count": 48375 }, "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-24T10:00:03.000Z"), "id": 669213907022716928, "text": "������������ https://t.co/K7kqlvsF1r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 229304981, "name": "Swishersweet✨", "screen_name": "_stillQUIL", "lang": "en", "location": "in my o w n zone.", "create_at": date("2010-12-21"), "description": "null", "followers_count": 273, "friends_count": 332, "statues_count": 3278 }, "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("2015-11-24T10:00:03.000Z"), "id": 669213907177889792, "text": "https://t.co/bjhxdCCHl0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2746050982, "name": "itsclutchhooo", "screen_name": "blackmigoe", "lang": "en", "location": "null", "create_at": date("2014-08-17"), "description": "IG BIGGDAWGGB", "followers_count": 47, "friends_count": 87, "statues_count": 120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodlawn, MD", "id": "bf64a5dd9b5f1cf2", "name": "Woodlawn", "place_type": "city", "bounding_box": rectangle("-76.915396,38.938828 -76.711092,39.327477") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2486525, "cityName": "Woodlawn" } }
+{ "create_at": datetime("2015-11-24T10:00:03.000Z"), "id": 669213907274215424, "text": "@jujudouget in love with you ???!!!!", "in_reply_to_status": 669213835077754880, "in_reply_to_user": 790439964, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 790439964 }}, "user": { "id": 1102912495, "name": "ShotSkyForPrez", "screen_name": "skysurr", "lang": "en", "location": "Pine Beach, NJ", "create_at": date("2013-01-18"), "description": "love yourself", "followers_count": 493, "friends_count": 414, "statues_count": 25233 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pine Beach, NJ", "id": "01acc3244d90350e", "name": "Pine Beach", "place_type": "city", "bounding_box": rectangle("-74.208684,39.902751 -74.151564,39.941532") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean", "cityID": 3458590, "cityName": "Pine Beach" } }
+{ "create_at": datetime("2015-11-24T10:00:03.000Z"), "id": 669213907362295809, "text": "Want to work at Aerotek? We're #hiring in #StLouis, MO! Click for details: https://t.co/W3KfvV9W0L #Healthcare #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.224967,38.646991"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "StLouis", "Healthcare", "Job", "Jobs" }}, "user": { "id": 24415813, "name": "TM-STL Health Jobs", "screen_name": "tmj_stl_health", "lang": "en", "location": "Saint Louis, MO", "create_at": date("2009-03-14"), "description": "Follow this account for geo-targeted Healthcare job tweets in Saint Louis, MO. Need help? Tweet us at @CareerArc!", "followers_count": 429, "friends_count": 304, "statues_count": 568 }, "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.320464,38.533149 -90.175132,38.774349") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29510, "countyName": "St. Louis", "cityID": 2965000, "cityName": "St. Louis" } }
+{ "create_at": datetime("2015-11-24T10:00:03.000Z"), "id": 669213907974823936, "text": "@Emeryyy_N �� https://t.co/1qAtiYSMkc", "in_reply_to_status": -1, "in_reply_to_user": 2841736741, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2841736741 }}, "user": { "id": 614246976, "name": "Erin .", "screen_name": "_its_erinnnn", "lang": "en", "location": "919", "create_at": date("2012-06-21"), "description": "sc: eriinnn_00", "followers_count": 1497, "friends_count": 1966, "statues_count": 24521 }, "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.794651,35.767843") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3701520, "cityName": "Apex" } }
+{ "create_at": datetime("2015-11-24T10:00:03.000Z"), "id": 669213908444405763, "text": "Watkins and Shepard #Transportation #Job: Driver CDL-A Local Delivery Denver, CO (#Denver, CO) https://t.co/hQMqBZNGTQ #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.8337879,39.8086537"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Transportation", "Job", "Denver", "Jobs", "Hiring" }}, "user": { "id": 23496679, "name": "TMJ-DEN Transp. Jobs", "screen_name": "tmj_den_transp", "lang": "en", "location": "Denver, CO", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Transportation job tweets in Denver, CO. Need help? Tweet us at @CareerArc!", "followers_count": 351, "friends_count": 292, "statues_count": 269 }, "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": 8005, "countyName": "Arapahoe", "cityID": 804000, "cityName": "Aurora" } }
+{ "create_at": datetime("2015-11-24T10:00:03.000Z"), "id": 669213908771602432, "text": "Sunny this afternoon, high 72 (22 C). Low 39 (4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15953346, "name": "Amarillo Weather", "screen_name": "AmarilloWeather", "lang": "en", "location": "Amarillo, Texas", "create_at": date("2008-08-22"), "description": "Forecasts and advisories for Amarillo, Texas", "followers_count": 3991, "friends_count": 395, "statues_count": 8585 }, "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("2015-11-24T10:00:03.000Z"), "id": 669213909006594048, "text": "@SteveBreezyy @MaryKayCabot where's Joe Mixon when you need him?", "in_reply_to_status": 669212831028391937, "in_reply_to_user": 569674943, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 569674943, 35107066 }}, "user": { "id": 567814694, "name": "Don", "screen_name": "MatthewREKMEH", "lang": "en", "location": "Winona, MN", "create_at": date("2012-04-30"), "description": "Nobody likes us except for us #507", "followers_count": 416, "friends_count": 376, "statues_count": 3220 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213909237264384, "text": "#AbandonedVehicle at W Sand Lake Rd & International Dr. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4708489,28.4498649"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AbandonedVehicle", "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": 265, "friends_count": 1, "statues_count": 64915 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213909266599936, "text": "Commission created to celebrate 100th anniversary of women's suffrage movement https://t.co/IcsqIVl9WA #ROC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ROC" }}, "user": { "id": 17022537, "name": "Rochester D and C", "screen_name": "DandC", "lang": "en", "location": "Rochester, NY", "create_at": date("2008-10-28"), "description": "The best source of local news in Rochester. Our in-depth coverage includes breaking news updates, sports, culture, food, business, education and more.", "followers_count": 49908, "friends_count": 4261, "statues_count": 81710 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rochester, NY", "id": "2f1fc0d72969452b", "name": "Rochester", "place_type": "city", "bounding_box": rectangle("-92.558347,43.099801 -77.533421,44.108056") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3663000, "cityName": "Rochester" } }
+{ "create_at": datetime("2015-11-24T10:00:04.000Z"), "id": 669213909505736705, "text": "I h8 when people say I tweet a lot UNFOLLOW BUTTON ME.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3345115541, "name": "¿keishlaaa?", "screen_name": "keishlaaMx", "lang": "en", "location": "null", "create_at": date("2015-06-24"), "description": "we taught ya jesarry Hernandez! not interested.❤️", "followers_count": 347, "friends_count": 331, "statues_count": 6989 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213909946118145, "text": "I FEEL ACCOMPLISHED OH 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": 3321010386, "name": "˗ˏˋ kait ˎˊ˗", "screen_name": "astrokait", "lang": "en", "location": "drè", "create_at": date("2015-08-20"), "description": "it smells like dust & moonlight.", "followers_count": 135, "friends_count": 93, "statues_count": 4570 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213909967052801, "text": "Temp 49.1°F RH 45% Wind 2.9 WNW Gust 10.0 WNW SLP 30.457 in Falling slowly Rain 0.00 in Solar 494 UV 2.5 #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": 103, "friends_count": 65, "statues_count": 31303 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213910214565888, "text": "@kflay we have 2! See u soon!", "in_reply_to_status": 669212771238723584, "in_reply_to_user": 20012399, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20012399 }}, "user": { "id": 18916646, "name": "Katie Mowgli", "screen_name": "KatieMowgli", "lang": "en", "location": "818", "create_at": date("2009-01-12"), "description": "there's love in all of us. @themowglis.", "followers_count": 8792, "friends_count": 889, "statues_count": 12856 }, "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-24T10:00:04.000Z"), "id": 669213910277427200, "text": "Just want to listen to dope music and enjoy the weather. Just one more day then 4 day weekend!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 350684918, "name": "just", "screen_name": "justinchevere", "lang": "en", "location": "Broward", "create_at": date("2011-08-07"), "description": "passionate about success", "followers_count": 317, "friends_count": 457, "statues_count": 17372 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213910491226112, "text": "@phatlexx lol koti", "in_reply_to_status": 669213730799001600, "in_reply_to_user": 1465337689, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1465337689 }}, "user": { "id": 551969582, "name": "rae", "screen_name": "HlPPIESCUM", "lang": "en", "location": "ur bucketlist", "create_at": date("2012-04-12"), "description": "hippie scum not hippies cum - im in love dont hmu - ny/ca", "followers_count": 1021, "friends_count": 325, "statues_count": 4694 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213910503829505, "text": "The second #cantree is built and can be seen @SunriseMallTX and thanks to Sacramento Association of Realtors! https://t.co/ew6JS3eHhr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "cantree" }}, "user_mentions": {{ 56379538 }}, "user": { "id": 44841114, "name": "Thomas Stambaugh", "screen_name": "SArmyofHope", "lang": "en", "location": "Roseville, CA", "create_at": date("2009-06-05"), "description": "Sacramento Social Service Secretary", "followers_count": 453, "friends_count": 412, "statues_count": 2292 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213910638063616, "text": "Want to visit again ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1038692990, "name": "Juana La Cubana", "screen_name": "annemuyfresca", "lang": "en", "location": "TEXA$", "create_at": date("2012-12-26"), "description": "A1 SINCE DAY 1", "followers_count": 903, "friends_count": 600, "statues_count": 94698 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213910877265922, "text": "Of course. Newest EP of kdrama involves her best friend moving far,far away. Just shatter me to death already.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 304019592, "name": "KT", "screen_name": "FionnaTheHuman", "lang": "en", "location": "Fishbowl Space, Temp Agency", "create_at": date("2011-05-23"), "description": "WRRRRRYYYYY!!!!", "followers_count": 122, "friends_count": 205, "statues_count": 6788 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213911015526400, "text": "@xohillaryrose this totally happened to me the other day. And the worst part about it...it was an Akon song stuck in my head..ugh ��", "in_reply_to_status": 663768331103567872, "in_reply_to_user": 329529873, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 329529873 }}, "user": { "id": 333207542, "name": "alex geddes", "screen_name": "BiggRiver", "lang": "en", "location": "null", "create_at": date("2011-07-10"), "description": "my mom is chilean. that's why i'm so lively and colourful.", "followers_count": 585, "friends_count": 433, "statues_count": 5654 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213911221215233, "text": "Class was great today ☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 248033424, "name": "fuck y'all!", "screen_name": "lifeasarealdiva", "lang": "en", "location": "null", "create_at": date("2011-02-05"), "description": "Rule #1 do your own shit, & don't worry about nobody else shit. Rule #2 read the first rule! #collegegirl #princess", "followers_count": 962, "friends_count": 1040, "statues_count": 94841 }, "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-24T10:00:04.000Z"), "id": 669213911279865856, "text": "@NedToby @NFL who cares", "in_reply_to_status": 669020222574100483, "in_reply_to_user": 299791722, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 299791722, 19426551 }}, "user": { "id": 26812007, "name": "Colin Mahoney", "screen_name": "the_colin", "lang": "en", "location": "New York City", "create_at": date("2009-03-26"), "description": "Married / father of 1 / Public relations / public affairs / government relations professional focused on clean tech, health care and defense industries", "followers_count": 320, "friends_count": 665, "statues_count": 296 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213911355289600, "text": "Kar to rahaa huu'n hosh ki baate'n\nMera junuu'n nakaam nahi 'n hai\n\nIshq ki hai qismat me'n thokar\nHusn pe kuch ilzaam nahi'n hai...Bi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "hi", "is_retweet": false, "user": { "id": 611616646, "name": "binish", "screen_name": "binishraza", "lang": "en", "location": "Detroit, MI", "create_at": date("2012-06-18"), "description": "LOVE STRETCHES YOUR HEART AND MAKE'S YOU BIG INSIDE....\n\n15 Dec ki ek shaam ko naazil hua tha.....Sagittarius kahte hai'n log..", "followers_count": 133, "friends_count": 8, "statues_count": 5292 }, "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-24T10:00:04.000Z"), "id": 669213911489511424, "text": "This is the worst thing my ears have ever heard https://t.co/xCNPSZ0pts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315241300, "name": "Julian James", "screen_name": "LordJames23", "lang": "en", "location": "null", "create_at": date("2013-03-29"), "description": "what white nonsense was that?", "followers_count": 602, "friends_count": 416, "statues_count": 46251 }, "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.17314,33.884965") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1377652, "cityName": "Tucker" } }
+{ "create_at": datetime("2015-11-24T10:00:04.000Z"), "id": 669213911539916800, "text": "Man Meek Mill lost all his clout \n\nHe just a clown to 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": 29788074, "name": "João", "screen_name": "OG_JAG", "lang": "en", "location": "Miramar, FL", "create_at": date("2009-04-08"), "description": "RIP AZ", "followers_count": 2538, "friends_count": 764, "statues_count": 163621 }, "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-24T10:00:04.000Z"), "id": 669213911766294528, "text": "See our latest #Vernon, CA #job and click to apply: Warehouse - https://t.co/OYHnBKeC6V #Labor #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.230073,34.003903"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Vernon", "job", "Labor", "Hiring", "CareerArc" }}, "user": { "id": 117866119, "name": "TMJ-CA Labor Jobs", "screen_name": "tmj_CA_LABOR", "lang": "en", "location": "California", "create_at": date("2010-02-26"), "description": "Follow this account for geo-targeted General Labor job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 403, "friends_count": 294, "statues_count": 422 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vernon, CA", "id": "853e23ffbf7f0201", "name": "Vernon", "place_type": "city", "bounding_box": rectangle("-118.239764,33.98335 -118.167198,34.016181") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 682422, "cityName": "Vernon" } }
+{ "create_at": datetime("2015-11-24T10:00:04.000Z"), "id": 669213911908913153, "text": "#Bellevue, WA #BusinessMgmt #Job: Site Manager-Commercial at Ace Parking Management https://t.co/Vj96H15hy6 #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.2006786,47.610377"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "Bellevue", "BusinessMgmt", "Job", "Jobs", "Hiring" }}, "user": { "id": 2749163574, "name": "Ace Parking Jobs", "screen_name": "JoinAceParking", "lang": "en", "location": "null", "create_at": date("2014-08-20"), "description": "Welcome to Ace Parking Careers, a starting place for limitless opportunities and growth. Check here for our current opportunities.", "followers_count": 67, "friends_count": 3, "statues_count": 388 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213912013770753, "text": "I dont like suprises����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 771107994, "name": "jay", "screen_name": "_jay_palmer_", "lang": "en", "location": "brittany❤️", "create_at": date("2012-08-21"), "description": "null", "followers_count": 223, "friends_count": 390, "statues_count": 6458 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie Heights, WA", "id": "004ef34f34440354", "name": "Prairie Heights", "place_type": "city", "bounding_box": rectangle("-122.144272,47.140985 -122.058992,47.166363") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5356150, "cityName": "Prairie Heights" } }
+{ "create_at": datetime("2015-11-24T10:00:04.000Z"), "id": 669213912017997824, "text": "Sunny this afternoon, high 69 (21 C). Low 41 (5 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712333, "name": "LubbockWeather", "screen_name": "LubbockWeather", "lang": "en", "location": "Lubbock, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Lubbock, Texas", "followers_count": 1167, "friends_count": 93, "statues_count": 7918 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213912093491200, "text": "Dammit tiger your making me look bad bro. https://t.co/yDD0X3557Y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 835437121, "name": "Amanda Marie", "screen_name": "ARaganxo", "lang": "en", "location": "null", "create_at": date("2012-09-20"), "description": "↞ life starts all over again when it gets crisp in the fall ↠", "followers_count": 631, "friends_count": 1387, "statues_count": 6381 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213912173178880, "text": "@DR0girl use one of your mom's pens.", "in_reply_to_status": 669212909113909248, "in_reply_to_user": 14175286, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14175286 }}, "user": { "id": 17950262, "name": "Max", "screen_name": "FunkCaptMax", "lang": "en", "location": "Seattle", "create_at": date("2008-12-07"), "description": "Civil Engineer by day // in search of the New Orleans FUNK & spreading crushing bluegrass tunes by night // #sealivemusic!", "followers_count": 3221, "friends_count": 2502, "statues_count": 44179 }, "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-24T10:00:04.000Z"), "id": 669213912420651009, "text": "The best thing I did was fall out of love", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 291145366, "name": "Jlo ♒️", "screen_name": "just___jordyn", "lang": "en", "location": "Oklahoma city, Oklahoma", "create_at": date("2011-05-01"), "description": "all the world is made of faith, and trust, and pixie dust.. future RN! IG @simply__jordyn snapchat jordyn.flynn", "followers_count": 1114, "friends_count": 945, "statues_count": 11804 }, "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-24T10:00:04.000Z"), "id": 669213912554844160, "text": "Try https://t.co/htTb5z6nsV .55 Large Pizza with New Crust #Pizza #menu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.59478799,43.14760801"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Pizza", "menu" }}, "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": 3556, "friends_count": 507, "statues_count": 146975 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213912664055808, "text": "Dawg... https://t.co/gtRfLetQvt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2759919075, "name": "Hopkins not top 5", "screen_name": "remisahn", "lang": "en", "location": "Victory Road", "create_at": date("2014-09-02"), "description": "being a nobody isn't all bad\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#EliHive #Cp3Hive #BoiseHive", "followers_count": 466, "friends_count": 410, "statues_count": 75248 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bensville, MD", "id": "6465ef531b9bcb00", "name": "Bensville", "place_type": "city", "bounding_box": rectangle("-77.047754,38.582811 -76.958486,38.654784") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24017, "countyName": "Charles", "cityID": 2406662, "cityName": "Bensville" } }
+{ "create_at": datetime("2015-11-24T10:00:04.000Z"), "id": 669213912773042178, "text": "@maryyhousman why didn't u tell me it was ur birthday", "in_reply_to_status": -1, "in_reply_to_user": 1317394633, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1317394633 }}, "user": { "id": 2540574439, "name": "mollie kate", "screen_name": "Molliee_Kate", "lang": "en", "location": "Holland, MI", "create_at": date("2014-06-01"), "description": "dogs", "followers_count": 343, "friends_count": 285, "statues_count": 2883 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Holland, MI", "id": "ee258f4bb56d1711", "name": "Holland", "place_type": "city", "bounding_box": rectangle("-86.197409,42.728541 -86.022493,42.802658") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26139, "countyName": "Ottawa", "cityID": 2638640, "cityName": "Holland" } }
+{ "create_at": datetime("2015-11-24T10:00:04.000Z"), "id": 669213912806486017, "text": "I'm no longer wanted from the one I love she move on so I turned to alcohol and drugs my body hurts but Idgaf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3644940498, "name": "FUCKJOEY", "screen_name": "joziaaah", "lang": "en", "location": "just a fuck up", "create_at": date("2015-09-21"), "description": "trying to overdose", "followers_count": 33, "friends_count": 80, "statues_count": 444 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213912919838724, "text": "But when does it end ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 359625785, "name": "Maria", "screen_name": "marmar232", "lang": "en", "location": "null", "create_at": date("2011-08-21"), "description": "Spy Kids was a truly life changing movie. // If Todd were an animated character, he'd be the horse from Tangled", "followers_count": 224, "friends_count": 279, "statues_count": 4227 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wooster, OH", "id": "459a2284d088a7dd", "name": "Wooster", "place_type": "city", "bounding_box": rectangle("-81.990959,40.758969 -81.877989,40.872411") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39169, "countyName": "Wayne", "cityID": 3986548, "cityName": "Wooster" } }
+{ "create_at": datetime("2015-11-24T10:00:04.000Z"), "id": 669213912924037120, "text": "He did say in one of his songs he was gonna get her back. https://t.co/D6nZYbujKr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 37612458, "name": "Onoda Sakamichi", "screen_name": "Optimus_blu", "lang": "en", "location": "Staying low and building ", "create_at": date("2009-05-03"), "description": "No my name isn't Nigel.", "followers_count": 3181, "friends_count": 2944, "statues_count": 148697 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213913083351040, "text": "Want to work in #Denver, CO? View our latest opening: https://t.co/fvLX3sp18u #Engineering #Job #Jobs #Hiring", "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": {{ "Denver", "Engineering", "Job", "Jobs", "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": 23, "friends_count": 13, "statues_count": 1752 }, "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-24T10:00:04.000Z"), "id": 669213913095909376, "text": "Just what I need, my belly button is bleeding ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 85838139, "name": "Vera", "screen_name": "Vrabes_", "lang": "en", "location": "Ames, IA", "create_at": date("2009-10-28"), "description": "null", "followers_count": 1295, "friends_count": 1154, "statues_count": 9501 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213913167196160, "text": "Barely had breakfast and I'm excited for lunch �� avocado salad and tuna ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1853353680, "name": "BeeAna ♡", "screen_name": "brianakuuipo", "lang": "en", "location": "null", "create_at": date("2013-09-10"), "description": "standing on my own two feet", "followers_count": 202, "friends_count": 138, "statues_count": 3980 }, "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("2015-11-24T10:00:04.000Z"), "id": 669213913175736320, "text": "SAME https://t.co/U3l85oLEAl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 171207894, "name": "annmarie.moxley❤️", "screen_name": "AGattoB_TCH16", "lang": "en", "location": "Pennsylvania, USA", "create_at": date("2010-07-26"), "description": "hey! I'm Annmarie & I'm 19! I'm in love with @TheDeanAmbrose & @ItsNickBean ❤️ 9/11/15, 10/28/15 5Quad af. #single", "followers_count": 221, "friends_count": 1872, "statues_count": 10883 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Geistown, PA", "id": "019633ce4319cfbc", "name": "Geistown", "place_type": "city", "bounding_box": rectangle("-78.885747,40.267115 -78.820631,40.326938") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42021, "countyName": "Cambria", "cityID": 4228720, "cityName": "Geistown" } }
+{ "create_at": datetime("2015-11-24T10:00:04.000Z"), "id": 669213913196679168, "text": "Couldn't agree with you more; so condescending & low rent https://t.co/eGfb85jEnC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 22490171, "name": "gretchen genello", "screen_name": "genellog", "lang": "en", "location": "Baltimore, MD", "create_at": date("2009-03-02"), "description": "Best known for my pies and a knitter of sweaters, mittens and socks", "followers_count": 342, "friends_count": 1168, "statues_count": 4322 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garrison, MD", "id": "989ad13286a88138", "name": "Garrison", "place_type": "city", "bounding_box": rectangle("-76.776462,39.383624 -76.726496,39.421885") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2431625, "cityName": "Garrison" } }
+{ "create_at": datetime("2015-11-24T10:00:04.000Z"), "id": 669213913251205122, "text": "@DamnitNicoo @SportsCenter gotta be committed to the vision!!!", "in_reply_to_status": 669212880236118016, "in_reply_to_user": 340689246, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 340689246, 26257166 }}, "user": { "id": 622115655, "name": "Lexy Fisher", "screen_name": "lexy_fisher", "lang": "en", "location": "olentangy liberty 16", "create_at": date("2012-06-29"), "description": "live by faith", "followers_count": 1591, "friends_count": 628, "statues_count": 9133 }, "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("2015-11-24T10:00:05.000Z"), "id": 669213913448321025, "text": "Wind 1.0 mph S. Barometer 30.347 in, Steady. Temperature 44.9 °F. Rain today 0.00 in. Humidity 71%", "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": 362, "friends_count": 286, "statues_count": 6207 }, "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("2015-11-24T10:00:05.000Z"), "id": 669213913519558656, "text": "Can you recommend anyone for this #job? Driver Opportunity - Big Miles, Big Pay! - https://t.co/Ss5N0DRYIB #driver https://t.co/Id6H3GCynD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.9766671,41.6819935"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "driver" }}, "user": { "id": 2414879204, "name": "JOBS at AIM", "screen_name": "JobsAtAim", "lang": "en", "location": "www.aimntls.com", "create_at": date("2014-03-27"), "description": "STABILITY. ADVANCEMENT. IMPACT. We're looking for professionals like you to join our team!\nApply to our open #jobs below, and follow us here @aimntls", "followers_count": 27, "friends_count": 4, "statues_count": 71 }, "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("2015-11-24T10:00:05.000Z"), "id": 669213913657937921, "text": "DAVIES IS PREGO ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "lt", "is_retweet": false, "user": { "id": 3115941511, "name": "{Brunette Barbie}", "screen_name": "Rayleen_Cook", "lang": "en", "location": "probably on Cloud9", "create_at": date("2015-03-29"), "description": "|HHS| snap; lilshawtyy220 ❤️| @munchkinn_butt is life|", "followers_count": 441, "friends_count": 906, "statues_count": 2904 }, "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("2015-11-24T10:00:05.000Z"), "id": 669213913678921728, "text": "Thanks giving break is not a break but more time to out work the lazy!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1056097466, "name": "Christian Marullo", "screen_name": "Chris_Marullo11", "lang": "en", "location": "null", "create_at": date("2013-01-02"), "description": "null", "followers_count": 362, "friends_count": 283, "statues_count": 180 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:05.000Z"), "id": 669213913964109824, "text": "ben just told me to shut my cock holster", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1025719782, "name": "caila", "screen_name": "cailamacomber", "lang": "en", "location": "laurel", "create_at": date("2012-12-20"), "description": "kanye attitude w/ drake feelings", "followers_count": 302, "friends_count": 211, "statues_count": 4571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laurel, MT", "id": "01175c3e72c37044", "name": "Laurel", "place_type": "city", "bounding_box": rectangle("-108.821918,45.657378 -108.637895,45.716935") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30111, "countyName": "Yellowstone", "cityID": 3042700, "cityName": "Laurel" } }
+{ "create_at": datetime("2015-11-24T10:00:05.000Z"), "id": 669213914501128193, "text": "#ootd������ Come shop! hurry in! be quick! @ Maruchi https://t.co/tUwlcolyic", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.3224945,25.7333698"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ootd" }}, "user": { "id": 38845946, "name": "MARUCHI CLOTHING ", "screen_name": "Maruchioutlet", "lang": "en", "location": "Miami", "create_at": date("2009-05-09"), "description": "The Best Women's clothing Outlet", "followers_count": 390, "friends_count": 321, "statues_count": 11428 }, "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("2015-11-24T10:00:05.000Z"), "id": 669213914693943296, "text": "Wind 0.7 mph N. Barometer 29.97 in, Steady. Temperature 62.2 °F. Rain today 0.00 in. Humidity 21%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.13472222,33.81666667"), "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": 20, "friends_count": 92, "statues_count": 12781 }, "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("2015-11-24T10:00:05.000Z"), "id": 669213914903646209, "text": "Sunny this afternoon, high 68 (20 C). Low 45 (7 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712562, "name": "ElPasoWX", "screen_name": "ElPasoWX", "lang": "en", "location": "El Paso, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for El Paso, Texas", "followers_count": 525, "friends_count": 93, "statues_count": 7976 }, "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("2015-11-24T10:00:05.000Z"), "id": 669213914970787841, "text": "Can you recommend anyone for this #job? Hadoop Developer - https://t.co/YrBnxSkd1l #Durham, North Carolina #IT #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.898619,35.9940329"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Durham", "IT", "Hiring", "CareerArc" }}, "user": { "id": 20825790, "name": "Raleigh IT Jobs ", "screen_name": "tmj_rdu_it", "lang": "en", "location": "RDU, NC", "create_at": date("2009-02-13"), "description": "Follow this account for geo-targeted Software Dev. - General/IT job tweets in Raleigh/Durham, NC from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 710, "friends_count": 328, "statues_count": 195 }, "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("2015-11-24T10:00:05.000Z"), "id": 669213915021049856, "text": "Want to work in #Richmond, VA? View our latest opening: https://t.co/Og7CUFCnhq #Marketing #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.4386429,37.5410261"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Richmond", "Marketing", "Job", "Jobs", "Hiring" }}, "user": { "id": 26803368, "name": "TMJ-RCH Advert. Jobs", "screen_name": "tmj_rch_adv", "lang": "en", "location": "Richmond, VA", "create_at": date("2009-03-26"), "description": "Follow this account for geo-targeted Marketing/Ad/PR job tweets in Richmond, VA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 290, "friends_count": 195, "statues_count": 10 }, "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("2015-11-24T10:00:05.000Z"), "id": 669213915230826496, "text": "@LilBeautifulAss no don't do it ��", "in_reply_to_status": 669213743494987776, "in_reply_to_user": 323040626, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 323040626 }}, "user": { "id": 455498754, "name": "Shany.", "screen_name": "_xoshanyyyyy", "lang": "en", "location": "Houston, TX", "create_at": date("2012-01-04"), "description": "before I formed you in the womb, I knew you. XIXXIIMMXXII", "followers_count": 249, "friends_count": 216, "statues_count": 9731 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:05.000Z"), "id": 669213915402735616, "text": "1.96 magnitude #earthquake. 4 km from Orinda, CA, #UnitedStates https://t.co/3ffrOJzbt7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.2226639,37.8878326"), "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": 54432, "friends_count": 10, "statues_count": 88912 }, "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": 6013, "countyName": "Contra Costa", "cityID": 654232, "cityName": "Orinda" } }
+{ "create_at": datetime("2015-11-24T10:00:05.000Z"), "id": 669213915641978881, "text": "Well eff my life then https://t.co/rVNX0mTYGq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 415208998, "name": "↠ mandy ↞", "screen_name": "thecommonmandy", "lang": "en", "location": "null", "create_at": date("2011-11-17"), "description": "|| #HailSouthern || Psalm 23:4 ||", "followers_count": 1660, "friends_count": 930, "statues_count": 97847 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Evans, GA", "id": "01af006c3a91a348", "name": "Evans", "place_type": "city", "bounding_box": rectangle("-82.228501,33.479096 -82.098825,33.593268") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13073, "countyName": "Columbia", "cityID": 1328044, "cityName": "Evans" } }
+{ "create_at": datetime("2015-11-24T10:00:05.000Z"), "id": 669213915948052480, "text": "Can't wait to see you either ! https://t.co/pBq893twEX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 348304604, "name": "Rielly Weathers➰", "screen_name": "Stormyyyleee", "lang": "en", "location": "West Columbia, SC", "create_at": date("2011-08-03"), "description": "Rielly Summer Weathers // Its never too late to start over. // South Carolina // WKHS // Breakin' necks & Killin' egos// 8•0•3//", "followers_count": 716, "friends_count": 848, "statues_count": 6697 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Red Bank, SC", "id": "96ef9310fd1d1cb8", "name": "Red Bank", "place_type": "city", "bounding_box": rectangle("-81.30108,33.874722 -81.152072,33.966779") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45063, "countyName": "Lexington", "cityID": 4559110, "cityName": "Red Bank" } }
+{ "create_at": datetime("2015-11-24T10:00:05.000Z"), "id": 669213916287799297, "text": "She cries at the end of the video when she starts talking about her mom having cancer https://t.co/VDoCTIoeXD", "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": 1185, "friends_count": 578, "statues_count": 165456 }, "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("2015-11-24T10:00:05.000Z"), "id": 669213916573147136, "text": "Temp: 75.2°F | Humidity: 39% | Wind: N @ 0.0 mph | Barometer: 30.23 in | Dewpoint: 48.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": 163273 }, "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("2015-11-24T10:00:05.000Z"), "id": 669213916581376001, "text": "@___kwill you gotta slow down lol you move too fast fool", "in_reply_to_status": 669213629284118529, "in_reply_to_user": 100898773, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 100898773 }}, "user": { "id": 333290949, "name": "☎️", "screen_name": "HUNCHOZovaHOES", "lang": "en", "location": "Where you scared to go, Texas", "create_at": date("2011-07-11"), "description": "im ground breakin and im only takin baby steps $VO... Free my family #Leroy #Ambitious #AvaSkye 20", "followers_count": 2249, "friends_count": 794, "statues_count": 73429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Neches, TX", "id": "445c61729545f1bd", "name": "Port Neches", "place_type": "city", "bounding_box": rectangle("-93.982751,29.930821 -93.918394,30.00561") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48245, "countyName": "Jefferson", "cityID": 4858940, "cityName": "Port Neches" } }
+{ "create_at": datetime("2015-11-24T10:00:05.000Z"), "id": 669213916589854720, "text": "You know if a girl fucked, bc her hair be all over her head ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 276232243, "name": "AK✨", "screen_name": "xSHETHE_B0MB", "lang": "en", "location": "Chester, SC❤️", "create_at": date("2011-04-02"), "description": "19. snapchat. asia_1013 .", "followers_count": 1956, "friends_count": 1890, "statues_count": 29375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chester, SC", "id": "fc3d3d1697ab1ede", "name": "Chester", "place_type": "city", "bounding_box": rectangle("-81.242759,34.683155 -81.189781,34.730359") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45023, "countyName": "Chester", "cityID": 4514095, "cityName": "Chester" } }
+{ "create_at": datetime("2015-11-24T10:00:05.000Z"), "id": 669213916707164160, "text": "http:https://t.co/13ZAibL7PR\n\nListening to this podcast now.....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 326821994, "name": "LawKansas Bob Hiller", "screen_name": "rrh8", "lang": "en", "location": "Kansas, USA", "create_at": date("2011-06-30"), "description": "Sports / music fanatic. Attorney, mentor, writer & consultant. Northwestern U & KU Law School. Avid fan of Jayhawks, Bears, Bulls, Cubs, Blackhawks", "followers_count": 6873, "friends_count": 6437, "statues_count": 12348 }, "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("2015-11-24T10:00:05.000Z"), "id": 669213916782796805, "text": "Just got in about to be rocked ...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3044763960, "name": "BLT ✨", "screen_name": "_breekyy", "lang": "en", "location": "null", "create_at": date("2015-02-26"), "description": "back to this twitter shit .. NEW PAGE ! Dms closed .....", "followers_count": 591, "friends_count": 511, "statues_count": 12325 }, "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-24T10:00:05.000Z"), "id": 669213916937981952, "text": "I can't find any pants hel p", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 605591249, "name": "lil fulmer", "screen_name": "_thelionqueen_", "lang": "en", "location": "null", "create_at": date("2012-06-11"), "description": "#Bernie2016", "followers_count": 515, "friends_count": 271, "statues_count": 49166 }, "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("2015-11-24T10:00:05.000Z"), "id": 669213916938006528, "text": "But y'all aren't ready. �� ��: @afeulner44 @ Bala Cynwyd Philly https://t.co/MyWcGvqCJS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.22887936,40.00018116"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 261444431 }}, "user": { "id": 2285045699, "name": "Kohn", "screen_name": "_Kohn42", "lang": "en", "location": "Elkins Park", "create_at": date("2014-01-15"), "description": "|| #MelodyLane || @_ThePioneerz || Business: mattkohn42@gmail.com", "followers_count": 343, "friends_count": 460, "statues_count": 2039 }, "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-24T10:00:05.000Z"), "id": 669213916950458368, "text": "Playing Battle Creek today.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.233279,48.051143"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 519545250, "name": "Punk Lewis", "screen_name": "PunkLewisGolf", "lang": "en", "location": "Mountlake Terrace, WA USA ", "create_at": date("2012-03-09"), "description": "Just a boy chasing his dream. Wounded but not out. I still have one mission left. To play on the PGA Tour. #punklewisgolf #roadtothetour2016 #WWP Alumni", "followers_count": 975, "friends_count": 1346, "statues_count": 6534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Battle Creek Golf Course", "id": "07d9f33587086001", "name": "Battle Creek Golf Course", "place_type": "poi", "bounding_box": rectangle("-122.23327909999999,48.0511429 -122.233279,48.051143") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5343955, "cityName": "Marysville" } }
+{ "create_at": datetime("2015-11-24T10:00:05.000Z"), "id": 669213916988317700, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1012079107, "name": "jacqueline", "screen_name": "jaccc_hollis", "lang": "en", "location": "Leland, NC", "create_at": date("2012-12-14"), "description": "null", "followers_count": 485, "friends_count": 186, "statues_count": 3350 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Leland, NC", "id": "d38fb24505dfdd32", "name": "Leland", "place_type": "city", "bounding_box": rectangle("-78.078442,34.203879 -77.98719,34.275451") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37019, "countyName": "Brunswick", "cityID": 3737680, "cityName": "Leland" } }
+{ "create_at": datetime("2015-11-24T10:00:05.000Z"), "id": 669213917072199680, "text": "Fetty Wap tho. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 116074571, "name": "❤ MITAM ❤", "screen_name": "LiamAF93", "lang": "en", "location": "up 1D's ass", "create_at": date("2010-02-20"), "description": "Krystal. 21. \nRendezvous.\nLiam is my #1.\n(Not so) Secret Harry girl. \nNiall and Louis are my loves.", "followers_count": 1414, "friends_count": 1255, "statues_count": 13193 }, "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("2015-11-24T10:00:05.000Z"), "id": 669213917139177472, "text": "Europe just discovered that if the US stays home, they dont have a military option.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17885604, "name": "varifrank", "screen_name": "varifrank", "lang": "en", "location": "38.7758228 -121.2994642", "create_at": date("2008-12-04"), "description": "Looking for a skeptic? Well, you found one.", "followers_count": 446, "friends_count": 920, "statues_count": 30606 }, "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("2015-11-24T10:00:05.000Z"), "id": 669213917361508353, "text": "Last Christmas ���� https://t.co/fyakcnwhvN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "et", "is_retweet": false, "user": { "id": 2646370440, "name": "m i r a n d a .", "screen_name": "miranddaaaa99", "lang": "en", "location": "null", "create_at": date("2014-07-14"), "description": "It's all in the details.\nc/o 2016.", "followers_count": 233, "friends_count": 194, "statues_count": 10586 }, "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-24T10:00:05.000Z"), "id": 669213917365854210, "text": "#TheftByShoplifting at 5269 International Dr. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4513754,28.4671053"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TheftByShoplifting", "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": 265, "friends_count": 1, "statues_count": 64916 }, "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("2015-11-24T10:00:06.000Z"), "id": 669213917885808644, "text": "Partly cloudy/wind this afternoon, high 65 (18 C). Low 54 (12 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712512, "name": "AbileneWeather", "screen_name": "AbileneWeather", "lang": "en", "location": "Abilene, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Abilene, Texas", "followers_count": 915, "friends_count": 93, "statues_count": 8016 }, "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("2015-11-24T10:00:06.000Z"), "id": 669213917894148096, "text": "Can I just sleep in baes bed 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": 233391954, "name": "nicolenava.", "screen_name": "nicoleiscooool", "lang": "en", "location": "null", "create_at": date("2011-01-02"), "description": "bo$$y! ✨ insta: nicoleiscoooool", "followers_count": 1487, "friends_count": 713, "statues_count": 29478 }, "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-24T10:00:06.000Z"), "id": 669213917952937988, "text": "hmmm, seem jealous", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1355272627, "name": "allie", "screen_name": "alhudge", "lang": "en", "location": "null", "create_at": date("2013-04-15"), "description": "Magnolia, Texas", "followers_count": 282, "friends_count": 103, "statues_count": 798 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stagecoach, TX", "id": "0162afbe0e5cbfab", "name": "Stagecoach", "place_type": "city", "bounding_box": rectangle("-95.804079,30.102712 -95.682507,30.157468") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48339, "countyName": "Montgomery", "cityID": 4869932, "cityName": "Stagecoach" } }
+{ "create_at": datetime("2015-11-24T10:00:06.000Z"), "id": 669213917982277632, "text": "Interested in a #Retail #job near #Orlando, FL? This could be a great fit: https://t.co/mMRaxP94OB #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2064287,28.5536135"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "Orlando", "Hiring", "CareerArc" }}, "user": { "id": 28620786, "name": "TMJ-ORL Retail Jobs", "screen_name": "tmj_orl_retail", "lang": "en", "location": "Orlando, FL", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Orlando, FL. Need help? Tweet us at @CareerArc!", "followers_count": 377, "friends_count": 310, "statues_count": 923 }, "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("2015-11-24T10:00:06.000Z"), "id": 669213918074679296, "text": "Catch our Holiday special right now on @templetv !! Guest stars: @CoryW4de @Bex965 @Theskorpionshow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 222498748, 1157712085, 185297787, 90283517 }}, "user": { "id": 2200520574, "name": "Temple Talk", "screen_name": "Temple_Talk", "lang": "en", "location": "TUTV Philadelphia, PA", "create_at": date("2013-11-17"), "description": "Philadelphia's Entertainment Talk Show on @TempleTV! ⭐️Hosted By: @Luke_Frey @marissagiletto @Backover_there @JohnnaBooth_ ⭐️Email: Templetalktv@gmail.com", "followers_count": 249, "friends_count": 273, "statues_count": 495 }, "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-24T10:00:06.000Z"), "id": 669213918141657089, "text": "Look at what was in my @timehop! @Joscelyn97 sounds like something you would say�� https://t.co/HMtk6XOE0d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 436143123, 49866217 }}, "user": { "id": 293763991, "name": "Sarah Lukaszewicz", "screen_name": "sarah_lynn47", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2011-05-05"), "description": "Point Park University '19\n Forensic Science Major", "followers_count": 270, "friends_count": 622, "statues_count": 7872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carnegie, PA", "id": "926c2b6b7aac446c", "name": "Carnegie", "place_type": "city", "bounding_box": rectangle("-80.099975,40.398563 -80.067374,40.420078") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4211336, "cityName": "Carnegie" } }
+{ "create_at": datetime("2015-11-24T10:00:06.000Z"), "id": 669213918267469824, "text": "@beardedbloke_uk I have to agree on the adidas. I'm a whore for those 3 stripes & design.", "in_reply_to_status": 669209262812487680, "in_reply_to_user": 264149091, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 264149091 }}, "user": { "id": 301596621, "name": "Drub O'Ryan", "screen_name": "Drubskin", "lang": "en", "location": "San Diego, CA", "create_at": date("2011-05-19"), "description": "Homo, Erotic. Over 25 years of unapologetic pervy pleasure through fine-art prints, books, greeting cards, comics & other items.", "followers_count": 1994, "friends_count": 266, "statues_count": 18197 }, "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-24T10:00:06.000Z"), "id": 669213918670123008, "text": "Kinda just wanna lay in bed the whole day", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4032226694, "name": "Sarah Z", "screen_name": "SarahShaniaZ", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-10-26"), "description": "UNLV│️indianbowl️ │only way is ⬆️│", "followers_count": 90, "friends_count": 104, "statues_count": 91 }, "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("2015-11-24T10:00:06.000Z"), "id": 669213919014207488, "text": "@CClothier21 @silas_jacobs you do...", "in_reply_to_status": 669213747697885184, "in_reply_to_user": 706228519, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 706228519, 628049210 }}, "user": { "id": 2740907365, "name": "Walter Ollie", "screen_name": "Wally2Ollie", "lang": "en", "location": "null", "create_at": date("2014-08-17"), "description": "Loras College 19'", "followers_count": 628, "friends_count": 168, "statues_count": 1966 }, "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("2015-11-24T10:00:06.000Z"), "id": 669213919081164800, "text": "Haven't seen my lover in 4months so you can imagine how excited I am 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": 3121298618, "name": "K✨", "screen_name": "kelseyfineasss", "lang": "en", "location": "Vallejo, CA", "create_at": date("2015-03-31"), "description": "married to @Carter_SFH ❤️", "followers_count": 492, "friends_count": 482, "statues_count": 4339 }, "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("2015-11-24T10:00:06.000Z"), "id": 669213919135690752, "text": "Happy birthday douche fuck @MchsZack https://t.co/ilbZknd7a8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1726666952 }}, "user": { "id": 629771885, "name": "#TeamBuckets11", "screen_name": "NClemmons12", "lang": "en", "location": "Channahon, IL", "create_at": date("2012-07-07"), "description": "RIP DAD", "followers_count": 567, "friends_count": 543, "statues_count": 8181 }, "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("2015-11-24T10:00:06.000Z"), "id": 669213919274135553, "text": "The richest niggas be having the smallest dicks. ��", "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": "call me muva", "screen_name": "MickieeMoscoto", "lang": "en", "location": "where the wild things are ", "create_at": date("2010-06-08"), "description": "a niggas dream & bitches nightmare. bookingmickiee@gmail.com", "followers_count": 76169, "friends_count": 57605, "statues_count": 83138 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn Center, MN", "id": "70e030d7c5f03769", "name": "Brooklyn Center", "place_type": "city", "bounding_box": rectangle("-93.351357,45.039888 -93.281148,45.087566") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2707948, "cityName": "Brooklyn Center" } }
+{ "create_at": datetime("2015-11-24T10:00:06.000Z"), "id": 669213919542714370, "text": "Looking for a game to watch tonight? Come check out UNW men's hoops at Macalaster. JV tips at 5/varsity at 7.\n#GratitudeandGrit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GratitudeandGrit" }}, "user": { "id": 2499593131, "name": "Coach Tim Grosz", "screen_name": "unwmbb", "lang": "en", "location": "St. Paul, Minn.", "create_at": date("2014-05-16"), "description": "2015 NCAA III Sweet Sixteen, 2011-2014 NCAA III National Tournament, 2010 NCCAA I National Champions, UMAC Champions 12 of the last 15 years.", "followers_count": 368, "friends_count": 158, "statues_count": 1083 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roseville, MN", "id": "419d7e5fdfa100b4", "name": "Roseville", "place_type": "city", "bounding_box": rectangle("-93.208095,44.991758 -93.10574,45.036415") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27123, "countyName": "Ramsey", "cityID": 2755852, "cityName": "Roseville" } }
+{ "create_at": datetime("2015-11-24T10:00:06.000Z"), "id": 669213919928561665, "text": "Who would win this family brawl?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 386769228, "name": "Ethan Müller", "screen_name": "EthanJuice", "lang": "en", "location": "Rochester, MI ", "create_at": date("2011-10-07"), "description": "I drove myself home on a broken foot // I favorite my own tweets // Umich '19", "followers_count": 643, "friends_count": 343, "statues_count": 9193 }, "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-24T10:00:06.000Z"), "id": 669213920150683648, "text": "@deserteastgolf I enjoyed our thread! Keep in touch!", "in_reply_to_status": 669146337145958406, "in_reply_to_user": 25040261, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25040261 }}, "user": { "id": 3408044567, "name": "Isaac Medina", "screen_name": "IsaacJMedina", "lang": "en", "location": "El Paso, TX", "create_at": date("2015-08-07"), "description": "Photographer + Creative Director at IMPHOTOGRAPHY, yearbook/newspaper advisor at Irvin High School, Pizza aficionado, UTEP Alumni 12'", "followers_count": 112, "friends_count": 264, "statues_count": 544 }, "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("2015-11-24T10:00:06.000Z"), "id": 669213920154927104, "text": "Anyone need an indoor keeper? Co-ed or girls or any team?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2874785425, "name": "Kaylani☄", "screen_name": "amandatrystine2", "lang": "en", "location": "deep in the heart of tx", "create_at": date("2014-10-24"), "description": "• Simply doing what's best for me • Ranger JC goalie⚽️ #20", "followers_count": 366, "friends_count": 627, "statues_count": 9075 }, "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("2015-11-24T10:00:06.000Z"), "id": 669213920335257600, "text": "@abookapart how do I know when to expect my new copy of @beep’s book to arrive? No shipping notification, tracking info, etc.", "in_reply_to_status": -1, "in_reply_to_user": 106418338, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 106418338, 12534 }}, "user": { "id": 13138192, "name": "Jarrod", "screen_name": "wubbels", "lang": "en", "location": "Lincoln, NE", "create_at": date("2008-02-05"), "description": "Growed up farm kid.", "followers_count": 96, "friends_count": 238, "statues_count": 3137 }, "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("2015-11-24T10:00:06.000Z"), "id": 669213920394002434, "text": "Join the Centura Health team! See our latest #Physician #job opening here: https://t.co/0rggH0q12o #CastleRock, CO #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.8560902,39.3722121"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Physician", "job", "CastleRock", "Hiring", "CareerArc" }}, "user": { "id": 21681745, "name": "Denver Health Jobs", "screen_name": "tmj_den_health", "lang": "en", "location": "Denver, CO", "create_at": date("2009-02-23"), "description": "Follow this account for geo-targeted Healthcare job tweets in Denver, CO. Need help? Tweet us at @CareerArc!", "followers_count": 615, "friends_count": 305, "statues_count": 880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Castle Rock, CO", "id": "0fd0097acd635907", "name": "Castle Rock", "place_type": "city", "bounding_box": rectangle("-104.92104,39.322269 -104.773048,39.451319") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8035, "countyName": "Douglas", "cityID": 812415, "cityName": "Castle Rock" } }
+{ "create_at": datetime("2015-11-24T10:00:06.000Z"), "id": 669213920628871169, "text": "GOD DAMN ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2340487255, "name": "ahlysa", "screen_name": "AhlysaWise", "lang": "en", "location": "null", "create_at": date("2014-02-12"), "description": "17 & cute / scorpio", "followers_count": 308, "friends_count": 168, "statues_count": 2933 }, "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("2015-11-24T10:00:06.000Z"), "id": 669213920805056512, "text": "There's no way ! This is live af https://t.co/iQOkYSlnN5", "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": "Curved , but not forgotten ...", "create_at": date("2010-12-04"), "description": "Stunt on me once and I'll flex on you forever . Based.", "followers_count": 1248, "friends_count": 680, "statues_count": 32257 }, "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("2015-11-24T10:00:06.000Z"), "id": 669213920872239104, "text": "Fair warning... I'm about to start watching Titanic. Quote tweets in coming ;)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3299796421, "name": "Philip B. Fiorucci", "screen_name": "benfiorucci12", "lang": "en", "location": "Warsaw, IN", "create_at": date("2015-07-28"), "description": "Just a guy living in Indiana who loves his friends.", "followers_count": 24, "friends_count": 108, "statues_count": 1294 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warsaw, IN", "id": "9a4adc08e83b58df", "name": "Warsaw", "place_type": "city", "bounding_box": rectangle("-85.944986,41.190359 -85.765358,41.304979") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18085, "countyName": "Kosciusko", "cityID": 1880306, "cityName": "Warsaw" } }
+{ "create_at": datetime("2015-11-24T10:00:06.000Z"), "id": 669213920872255489, "text": "Girls are actually 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": 1480878060, "name": "mikemastaK", "screen_name": "mikeykrakowski", "lang": "en", "location": "Dallas Pennsylvania", "create_at": date("2013-06-03"), "description": "i do not own this name", "followers_count": 141, "friends_count": 150, "statues_count": 608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shavertown, PA", "id": "0b12c2212d92a2d5", "name": "Shavertown", "place_type": "city", "bounding_box": rectangle("-76.025396,41.278082 -75.905418,41.384029") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42079, "countyName": "Luzerne", "cityID": 4269832, "cityName": "Shavertown" } }
+{ "create_at": datetime("2015-11-24T10:00:06.000Z"), "id": 669213921018908672, "text": "Mf steady tryna piss 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": 579194707, "name": "• Queen E •", "screen_name": "Esme_BooBoo13", "lang": "en", "location": "null", "create_at": date("2012-05-13"), "description": "September 24 2015", "followers_count": 323, "friends_count": 323, "statues_count": 5423 }, "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-24T10:00:06.000Z"), "id": 669213921023238144, "text": "when you see something is due at 12 don't assume midnight like me cause now I'm fucked ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 322221778, "name": "Stevie Struhar", "screen_name": "steviejash3", "lang": "en", "location": "dover/akron oh", "create_at": date("2011-06-22"), "description": "pizza & fifth harmony control my existence", "followers_count": 362, "friends_count": 310, "statues_count": 22843 }, "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("2015-11-24T10:00:06.000Z"), "id": 669213921102860288, "text": "Pissed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1578848946, "name": "Almighty6", "screen_name": "chasewebster6_", "lang": "en", "location": "Odessa, Texas", "create_at": date("2013-07-08"), "description": "MaddyG is my babe and she is freaking awesome. I love her and football. DC. Odessa High Sophmore. oooh yah yah.. DC4 soon", "followers_count": 818, "friends_count": 645, "statues_count": 9496 }, "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("2015-11-24T10:00:06.000Z"), "id": 669213921148956672, "text": "LeBron and Steph were both born in Akron, OH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388364264, "name": "Evan Stites-Clayton", "screen_name": "the_esc", "lang": "en", "location": "null", "create_at": date("2011-10-10"), "description": "Co-Founder @teespring, Lucid Dreaming, and Scheming. ...and then there is using everything...", "followers_count": 961, "friends_count": 114, "statues_count": 599 }, "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("2015-11-24T10:00:06.000Z"), "id": 669213921199448064, "text": "@theyenvy_nee https://t.co/79L7OIxmzt", "in_reply_to_status": -1, "in_reply_to_user": 1176226992, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1176226992 }}, "user": { "id": 584293747, "name": "TAY⚡️", "screen_name": "TaylaGotDaJuice", "lang": "en", "location": "null", "create_at": date("2012-05-18"), "description": "IG- @__lultayy | The Offical Taylor™⭐️", "followers_count": 665, "friends_count": 589, "statues_count": 18348 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chantilly, VA", "id": "353916ce76086be6", "name": "Chantilly", "place_type": "city", "bounding_box": rectangle("-77.485561,38.840172 -77.364749,38.917243") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5114744, "cityName": "Chantilly" } }
+{ "create_at": datetime("2015-11-24T10:00:06.000Z"), "id": 669213921228791808, "text": "#Obama looks so lost without his \nTeleprompter", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Obama" }}, "user": { "id": 1292373920, "name": "Pete Byron Dooley", "screen_name": "pbyrond", "lang": "en", "location": "Philadelphia, PA/Hollywood, FL", "create_at": date("2013-03-23"), "description": "I just live in South Florida my Heart belongs to Philadelphia. The Rolling Stones All My Children #ExileKimmy ✌️❤️", "followers_count": 1280, "friends_count": 577, "statues_count": 42066 }, "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("2015-11-24T10:00:06.000Z"), "id": 669213921253814273, "text": "I can't remember if I really sold it 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": 3149592764, "name": "Okasan Kattana✨", "screen_name": "0kasan_", "lang": "en", "location": "null", "create_at": date("2015-04-11"), "description": "17| Feminist | Black Activist | Desexualize the Female body| #Blacklivesmatter", "followers_count": 89, "friends_count": 101, "statues_count": 784 }, "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("2015-11-24T10:00:06.000Z"), "id": 669213921375485952, "text": "@marionieto6 you right ����", "in_reply_to_status": 669213384865243136, "in_reply_to_user": 537788938, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 537788938 }}, "user": { "id": 788013746, "name": "TakeOff", "screen_name": "_lil_free_", "lang": "en", "location": "Saved By Jesus Christ", "create_at": date("2012-08-28"), "description": "Texas Tech commit ⚾️ #GunsUp #WreckEm", "followers_count": 688, "friends_count": 989, "statues_count": 1329 }, "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("2015-11-24T10:00:06.000Z"), "id": 669213921409003520, "text": "Sunny this afternoon, high 70 (21 C). Low 50 (10 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712530, "name": "MidlandWeather", "screen_name": "MidlandWeather", "lang": "en", "location": "Midland/Odessa, TX", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Midland/Odessa, TX", "followers_count": 864, "friends_count": 93, "statues_count": 7999 }, "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("2015-11-24T10:00:07.000Z"), "id": 669213921958453248, "text": "@oj_walker ight bro, well if you would tell yo buddy or whoever else for me", "in_reply_to_status": 669212309416312832, "in_reply_to_user": 703691454, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 703691454 }}, "user": { "id": 1490437632, "name": "Easton Gordon", "screen_name": "EG3hunna", "lang": "en", "location": "Ardmore, Ok", "create_at": date("2013-06-07"), "description": "Keep my commandments and LIVE - Proverbs 7:2", "followers_count": 510, "friends_count": 461, "statues_count": 42294 }, "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("2015-11-24T10:00:07.000Z"), "id": 669213922029883392, "text": "@SpaceGhost_Domo already! Appreciate that bro!", "in_reply_to_status": 669213047702032387, "in_reply_to_user": 175299791, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 175299791 }}, "user": { "id": 124806029, "name": "C.j. Germany", "screen_name": "Geeez19", "lang": "en", "location": "East side", "create_at": date("2010-03-20"), "description": "go get it.", "followers_count": 1771, "friends_count": 900, "statues_count": 47734 }, "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("2015-11-24T10:00:07.000Z"), "id": 669213922109603841, "text": "@FoxNews @AndyTho10107697 Bo needs to be impeached on the grounds of insanity he is no longer capable of leading our Country!", "in_reply_to_status": 669205401011818496, "in_reply_to_user": 1367531, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1367531, 529548712 }}, "user": { "id": 2149325455, "name": "Sunflower", "screen_name": "OneSunflowerGrl", "lang": "en", "location": "null", "create_at": date("2013-10-22"), "description": "Love my Country the Constitution #2A #Guns #Wife of a Military vet! I have strong opinions on politics! love life ♡♡ ~(••)~ WIN BEN WIN", "followers_count": 613, "friends_count": 866, "statues_count": 3561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elgin, IL", "id": "7c4ae4537997a58f", "name": "Elgin", "place_type": "city", "bounding_box": rectangle("-88.431293,41.986293 -88.189602,42.093586") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17089, "countyName": "Kane", "cityID": 1723074, "cityName": "Elgin" } }
+{ "create_at": datetime("2015-11-24T10:00:07.000Z"), "id": 669213922361278464, "text": "We Gotta switch those Trojans for these @pineappleswirl https://t.co/f3sXsyHKYV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2571086015 }}, "user": { "id": 110566444, "name": "Kawla Velez", "screen_name": "KarlaaaVelez", "lang": "en", "location": "null", "create_at": date("2010-02-01"), "description": "SENSATIONAL", "followers_count": 1022, "friends_count": 570, "statues_count": 89731 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jasmine Estates, FL", "id": "00b6dd7160a5af36", "name": "Jasmine Estates", "place_type": "city", "bounding_box": rectangle("-82.715072,28.273282 -82.616751,28.326834") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12101, "countyName": "Pasco", "cityID": 1235350, "cityName": "Jasmine Estates" } }
+{ "create_at": datetime("2015-11-24T10:00:07.000Z"), "id": 669213922377895936, "text": "@brooklynROFL @cmacmitch YES we stocked up", "in_reply_to_status": 669212296091033600, "in_reply_to_user": 111241740, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 111241740, 101923674 }}, "user": { "id": 265770264, "name": "cornucopkeisha", "screen_name": "keisha__miller", "lang": "en", "location": "null", "create_at": date("2011-03-13"), "description": "can u not", "followers_count": 427, "friends_count": 134, "statues_count": 10939 }, "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("2015-11-24T10:00:07.000Z"), "id": 669213922453417984, "text": "ARRIVE'S A NEW YEAR...“LINCOLN'S PATH – feat. PUB NOISE SECOND MOVEMENT” from the album 'Skillzzen' https://t.co/wUSjmPchMA ...", "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": 606, "friends_count": 2004, "statues_count": 109127 }, "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("2015-11-24T10:00:07.000Z"), "id": 669213923317563392, "text": "Truuuuuu ������ https://t.co/C7mfhDR3iy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 475484820, "name": "Saniya Khatri", "screen_name": "SaniyaKhatri", "lang": "en", "location": "null", "create_at": date("2012-01-26"), "description": "Learned the depth of love and hate, but never did learn how to wait.", "followers_count": 145, "friends_count": 157, "statues_count": 1858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, MA", "id": "47aff34722fae115", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-71.202095,42.397429 -71.132174,42.436652") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2501640, "cityName": "Arlington" } }
+{ "create_at": datetime("2015-11-24T10:00:07.000Z"), "id": 669213923497897986, "text": "@bengoodfella exactly..as a kid I'd love it if Pats were mentioned in media; now, too many on both sides use every mention as a weapon.", "in_reply_to_status": 669213274379014145, "in_reply_to_user": 80951038, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 80951038 }}, "user": { "id": 238206616, "name": "Frank", "screen_name": "Effing_Frank", "lang": "en", "location": "Boston area", "create_at": date("2011-01-14"), "description": "Husband/Dad, working stiff, runner, lifter.", "followers_count": 85, "friends_count": 302, "statues_count": 2816 }, "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("2015-11-24T10:00:07.000Z"), "id": 669213923757961216, "text": "SAD that putin doing more to protect christians than @potus.(Winning his war tho) Poss. more isil dead by road rage than his airstrikes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1536791610 }}, "user": { "id": 2207157043, "name": "William Bowne", "screen_name": "Bill_Bowne", "lang": "en", "location": "looks like I'm at navy 2today", "create_at": date("2013-11-21"), "description": "some days are better than others libertarian- some posts are just for my amusement, others are just my mind wandering out through my fingertips", "followers_count": 176, "friends_count": 248, "statues_count": 2567 }, "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("2015-11-24T10:00:07.000Z"), "id": 669213923887833088, "text": "Temp: 41.9°F - Dew Point: 35.3° - Wind: 7.1 mph - Gust: 8.9 - Rain Today: 0.00in. - Pressure: 30.32in, - 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": 9605 }, "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("2015-11-24T10:00:07.000Z"), "id": 669213923967537152, "text": "@handymayhem @JosiahMcElroy Thank you, sir!", "in_reply_to_status": 669128688479444992, "in_reply_to_user": 24870433, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24870433, 552282663 }}, "user": { "id": 264935205, "name": "Nicki", "screen_name": "cheerio15", "lang": "en", "location": "United States", "create_at": date("2011-03-12"), "description": "True blue, all American southern gal that believes in God & his beloved Son, Jesus Christ. Time to kick some butt, America! #NRA,#DAR#TCOT#CUFI#NoRefugees", "followers_count": 1058, "friends_count": 426, "statues_count": 23946 }, "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("2015-11-24T10:00:07.000Z"), "id": 669213924194054144, "text": "Oomf is so naturally beautiful it's crazy ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3028366621, "name": "January 1st", "screen_name": "SeeversJacob", "lang": "en", "location": "Moore, OK", "create_at": date("2015-02-18"), "description": "I don't like bitches that eat soggy cereal", "followers_count": 1775, "friends_count": 999, "statues_count": 32643 }, "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-24T10:00:07.000Z"), "id": 669213924345053184, "text": "Omg the feels I haven't seen that movie in ages but that was the first vhs I've ever owned https://t.co/2mugyGZ0wg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 295636725, "name": "Dr. Keem.", "screen_name": "RebelWithNoCauz", "lang": "en", "location": "null", "create_at": date("2011-05-09"), "description": "#Virgo everything is everything", "followers_count": 1710, "friends_count": 821, "statues_count": 52917 }, "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("2015-11-24T10:00:07.000Z"), "id": 669213924609257472, "text": "This lady added me on Facebook & she posts the most awkward selfies & it's the funniest thing ever https://t.co/tN4n6Os0bB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 176904999, "name": "Legея", "screen_name": "1jzS13", "lang": "en", "location": "1jz S13 ", "create_at": date("2010-08-10"), "description": "| //ラメボーイズ", "followers_count": 1763, "friends_count": 919, "statues_count": 37457 }, "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("2015-11-24T10:00:07.000Z"), "id": 669213924747776000, "text": "@HoldenHorse ill always be your partner <3", "in_reply_to_status": 669213787979821056, "in_reply_to_user": 152391661, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 152391661 }}, "user": { "id": 65580536, "name": "~ Stylish Craboo ~", "screen_name": "FzzyTiger", "lang": "en", "location": "Chicago, Illinois", "create_at": date("2009-08-13"), "description": "Im a Tiger who lives in Chicagor. I go Rawr n stuff. http://www.furaffinity.net/user/nekomon/", "followers_count": 1523, "friends_count": 1062, "statues_count": 30832 }, "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-24T10:00:07.000Z"), "id": 669213924793839616, "text": "@Ericc_M17 wierd ��", "in_reply_to_status": 669213842753241088, "in_reply_to_user": 859218842, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user_mentions": {{ 859218842 }}, "user": { "id": 1450173979, "name": "J Blunt®", "screen_name": "paranoide_jimmy", "lang": "en", "location": "Chico, CA", "create_at": date("2013-05-22"), "description": "If you aint got no sauce then ya lost, but you can also get lost in the sauce ️| C.G. A.J. | #AquaTechWellDrilling #NorCal #LesGetIt #FREEWEEZY", "followers_count": 519, "friends_count": 444, "statues_count": 15890 }, "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("2015-11-24T10:00:07.000Z"), "id": 669213925389438976, "text": "@EIfOnTheShelf @RealWhiteDrake @Radestlex_", "in_reply_to_status": 658502944124465152, "in_reply_to_user": 2925807244, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2925807244, 2306239020, 294845208 }}, "user": { "id": 474593555, "name": "abriana", "screen_name": "kiki____xoxo", "lang": "en", "location": "null", "create_at": date("2012-01-25"), "description": "null", "followers_count": 121, "friends_count": 185, "statues_count": 1783 }, "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("2015-11-24T10:00:07.000Z"), "id": 669213925460676608, "text": "*high AF https://t.co/pJFJtAl8qS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 486935848, "name": "Doobie_Master", "screen_name": "Josh_Lyons23", "lang": "en", "location": "null", "create_at": date("2012-02-08"), "description": "#RipRayRay #RipB-Hob #RipTreyPBlood", "followers_count": 1176, "friends_count": 482, "statues_count": 136441 }, "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-24T10:00:07.000Z"), "id": 669213925641076736, "text": "Lowdon not looking back:\nhttps://t.co/Xxk0vjptyj https://t.co/8Uj5l7hnqc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54987639, "name": "RACER", "screen_name": "RACERmag", "lang": "en", "location": "California, USA", "create_at": date("2009-07-08"), "description": "Full RACER + http://RACER.com intel at http://info.racer.com", "followers_count": 27212, "friends_count": 15289, "statues_count": 36281 }, "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("2015-11-24T10:00:07.000Z"), "id": 669213925729259520, "text": "@LumberKings I'll be the first in line", "in_reply_to_status": 669213328959500288, "in_reply_to_user": 76368129, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 76368129 }}, "user": { "id": 22518132, "name": "Tom Kruse", "screen_name": "tommyk77", "lang": "en", "location": "Sterling Illinois USA", "create_at": date("2009-03-02"), "description": "Hi I'm Tom I like traveling,cooking,drinking good beer. I love my wife and also Chile I hope to move back there someday", "followers_count": 424, "friends_count": 1358, "statues_count": 12492 }, "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("2015-11-24T10:00:07.000Z"), "id": 669213925788024832, "text": "This communications class is probably my favorite class in my two years of college 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": 186968285, "name": "Santiago Archieri", "screen_name": "SantiArch8", "lang": "en", "location": "50% Miami Beach 50% FIU", "create_at": date("2010-09-04"), "description": "19. Listen to me on the radio or watch me on tv. | Intern with a bunch of sporty stuff | Used to be a bowler now I'm a cheerleader. FIU18", "followers_count": 572, "friends_count": 198, "statues_count": 16036 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University Park, FL", "id": "883ee3f98abb082b", "name": "University Park", "place_type": "city", "bounding_box": rectangle("-80.385477,25.732128 -80.351653,25.761752") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1273287, "cityName": "University Park" } }
+{ "create_at": datetime("2015-11-24T10:00:08.000Z"), "id": 669213925972512769, "text": "#ThanksgivingWithBlackFamilies When your 2 uncles trying to pick out the females who ain't family... https://t.co/h17EvEzdfN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ThanksgivingWithBlackFamilies" }}, "user": { "id": 28107258, "name": "Chad Radwell Jr.", "screen_name": "NoChillJerry", "lang": "en", "location": "Cleveland", "create_at": date("2009-04-01"), "description": "Draco dormiens nunquam titillandus.", "followers_count": 9571, "friends_count": 6805, "statues_count": 38697 }, "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("2015-11-24T10:00:08.000Z"), "id": 669213926643490816, "text": "@jcbaniqued YES PLEASE!!!!", "in_reply_to_status": 669213848352649216, "in_reply_to_user": 3130736360, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3130736360 }}, "user": { "id": 2842856149, "name": "Rashita Minaj", "screen_name": "rashitaaraee", "lang": "en", "location": "champagnepapi ", "create_at": date("2014-10-06"), "description": "ONIKA TANYA MARAJ❤️❤️❤️❤️", "followers_count": 214, "friends_count": 219, "statues_count": 3102 }, "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("2015-11-24T10:00:08.000Z"), "id": 669213926693863424, "text": "NOW ON #SALE! Longhorn 4Burn LP Grill #BlackFriday #Christmas #Texans #Gift #Houston #Texas https://t.co/SO34idAhcm https://t.co/c5sCHeir2n", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SALE", "BlackFriday", "Christmas", "Texans", "Gift", "Houston", "Texas" }}, "user": { "id": 2563616802, "name": "Champion True Value", "screen_name": "ChampTrueValue", "lang": "en", "location": "16720 Champion Forest Drive", "create_at": date("2014-06-12"), "description": "We offer tremendous amount of products in hardware, kitchens, baths, windows, paint, giftware, and much more! So stop by if you are in the area!", "followers_count": 105, "friends_count": 137, "statues_count": 628 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:08.000Z"), "id": 669213926794465280, "text": "@JackieMackk ugh!!!!!", "in_reply_to_status": 669213680198918145, "in_reply_to_user": 27708161, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 27708161 }}, "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": 170, "friends_count": 405, "statues_count": 13709 }, "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("2015-11-24T10:00:08.000Z"), "id": 669213927058771968, "text": "All eyes 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": 490633804, "name": "Kenzie", "screen_name": "kenzieedwards99", "lang": "en", "location": "Lumberton, TX", "create_at": date("2012-02-12"), "description": "junior @ lumberton hs || Kevin James", "followers_count": 918, "friends_count": 915, "statues_count": 8282 }, "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("2015-11-24T10:00:08.000Z"), "id": 669213927079862272, "text": "Help youth heal through music ♪ ♫ ♩ ♬ https://t.co/yUiqYbN6Zd @plummerhome @crowdrise #GivingTuesday #PlummerMusic https://t.co/iOXyxHozOx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GivingTuesday", "PlummerMusic" }}, "user_mentions": {{ 2523262549, 77404218 }}, "user": { "id": 12421062, "name": "Julia C. Campbell", "screen_name": "JuliaCSocial", "lang": "en", "location": "Beverly, MA", "create_at": date("2008-01-18"), "description": "I help nonprofits make better use of social media & digital marketing tools. Mom, RPCV, feminist. #onlinefundraising #nptech #sm4np", "followers_count": 8692, "friends_count": 7597, "statues_count": 31330 }, "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("2015-11-24T10:00:08.000Z"), "id": 669213927146958851, "text": "I really need to do something with 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": 2547225049, "name": "bren", "screen_name": "od_onbren", "lang": "en", "location": "Atlanta, GA", "create_at": date("2014-06-04"), "description": "hoelanta", "followers_count": 978, "friends_count": 873, "statues_count": 7679 }, "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("2015-11-24T10:00:08.000Z"), "id": 669213927268569088, "text": "A 2010 Ford Focus was just scanned near Lincoln, NE 68514 https://t.co/Pa268hmek6 #apple #vinny_says #ios", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.64,40.89"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "apple", "vinny_says", "ios" }}, "user": { "id": 2361533634, "name": "Vinny Scans", "screen_name": "Vinny_Scans", "lang": "en", "location": "null", "create_at": date("2014-02-25"), "description": "In the market for a #usedcar? Ever wonder what your #car is worth? @Vinny_Says can help.", "followers_count": 782, "friends_count": 1, "statues_count": 441178 }, "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("2015-11-24T10:00:08.000Z"), "id": 669213927310364672, "text": "when we get 9th place out of 9 teams. https://t.co/p07Rd21Wj5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2921271823, "name": "Sydney McGhee", "screen_name": "sydneymcghee_", "lang": "en", "location": "null", "create_at": date("2014-12-06"), "description": "platinum athletics titanium senior restricted 5 & senior 3 black ice", "followers_count": 178, "friends_count": 85, "statues_count": 1060 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wright City, MO", "id": "00efbb0b73ce7a0d", "name": "Wright City", "place_type": "city", "bounding_box": rectangle("-91.049696,38.80954 -90.992494,38.850347") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29219, "countyName": "Warren", "cityID": 2981124, "cityName": "Wright City" } }
+{ "create_at": datetime("2015-11-24T10:00:08.000Z"), "id": 669213927419592704, "text": "@pcosss @switchinglaness \n������ omg scirts signature @@!", "in_reply_to_status": 669213630563524608, "in_reply_to_user": 157238280, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 157238280, 384692170 }}, "user": { "id": 923710766, "name": "MD in AC", "screen_name": "Matty_si", "lang": "en", "location": "null", "create_at": date("2012-11-03"), "description": "Nooooooooo man", "followers_count": 122, "friends_count": 184, "statues_count": 4019 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hauppauge, NY", "id": "1b8e90a61abe2e5e", "name": "Hauppauge", "place_type": "city", "bounding_box": rectangle("-73.272479,40.801671 -73.147805,40.846067") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3632732, "cityName": "Hauppauge" } }
+{ "create_at": datetime("2015-11-24T10:00:08.000Z"), "id": 669213927956320257, "text": "@pastorjarredCBC yeah that's really offensive... get a grip, champ. @google", "in_reply_to_status": 669211428151275520, "in_reply_to_user": 41507942, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 41507942, 20536157 }}, "user": { "id": 1205946938, "name": "This Justin", "screen_name": "NoThanksJustin", "lang": "en", "location": "The United States of Space", "create_at": date("2013-02-21"), "description": "Made it through school without google.", "followers_count": 4553, "friends_count": 606, "statues_count": 43658 }, "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("2015-11-24T10:00:08.000Z"), "id": 669213928614948864, "text": "Adult life in one picture https://t.co/lBXVk0049E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2521526700, "name": "big lexy", "screen_name": "venchthebench", "lang": "en", "location": "Bloomington, IN", "create_at": date("2014-05-24"), "description": "suburbanite/free spirit/cynic/baker #feelthebernfeelthebassnectar", "followers_count": 194, "friends_count": 472, "statues_count": 1666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Western Springs, IL", "id": "bbd7331f334b0e04", "name": "Western Springs", "place_type": "city", "bounding_box": rectangle("-87.91828,41.774916 -87.888206,41.833886") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1780242, "cityName": "Western Springs" } }
+{ "create_at": datetime("2015-11-24T10:00:08.000Z"), "id": 669213928639954948, "text": "@aimeemay @dailycamera can't wait for you to join us and to talk to you. Make sure you follow us @PearlStreetFit on periscope.", "in_reply_to_status": 669212458679033857, "in_reply_to_user": 11808662, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 11808662, 14504258, 2404414610 }}, "user": { "id": 2404414610, "name": "pearlstreetFIT", "screen_name": "PearlStreetFit", "lang": "en", "location": "Denver, CO", "create_at": date("2014-03-22"), "description": "Strong. Confident. People. full body fat burning workouts #fitness #worldwideworkout #icaniwll #strongconfidentliving #periscope", "followers_count": 425, "friends_count": 322, "statues_count": 357 }, "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-24T10:00:08.000Z"), "id": 669213928707092480, "text": "@Josh_Lyons23 serious af", "in_reply_to_status": 669213806308945920, "in_reply_to_user": 486935848, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 486935848 }}, "user": { "id": 3997419442, "name": "Breyonce'", "screen_name": "Bree_banana", "lang": "en", "location": "null", "create_at": date("2015-10-19"), "description": "yea ho y is you here", "followers_count": 153, "friends_count": 79, "statues_count": 2041 }, "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-24T10:00:08.000Z"), "id": 669213928719818752, "text": "Ripley SW Limestone Co. Temp: 59.7°F Wind:2.2mph Pressure: 1008.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": 37, "friends_count": 33, "statues_count": 44667 }, "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("2015-11-24T10:00:08.000Z"), "id": 669213929160073216, "text": "@PizzaPartyBen gay", "in_reply_to_status": 669213867000528896, "in_reply_to_user": 272107377, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 272107377 }}, "user": { "id": 258398208, "name": "Feminine Shitposter", "screen_name": "CProcks_", "lang": "en", "location": "Twitter", "create_at": date("2011-02-27"), "description": "14/bi/fluid/furry. Gets offensive at times. Telegram is CProcksRBLX. ask for skype. Vent: @RBLX_CProcks (Friends only)(Pronouns: he/she) pref she", "followers_count": 194, "friends_count": 512, "statues_count": 7674 }, "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("2015-11-24T10:00:08.000Z"), "id": 669213929432875008, "text": "@daveweigel @blackrepublican The Republican party will split at it's convention and Hillary will win! Words of the Prophet!", "in_reply_to_status": 669188006184624128, "in_reply_to_user": 13524182, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 13524182, 18294521 }}, "user": { "id": 3293192638, "name": "Prophet Campbell", "screen_name": "Prophet28287822", "lang": "en", "location": "Boston, MA", "create_at": date("2015-05-21"), "description": "Urban Prophet, Educator, Theologian, Scholar, Reconciler, Peacemaker, Author", "followers_count": 10, "friends_count": 262, "statues_count": 389 }, "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("2015-11-24T10:00:08.000Z"), "id": 669213929747251200, "text": "@doubleddolan Maaaan you ain't lying!!", "in_reply_to_status": 669209959251312640, "in_reply_to_user": 384164681, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 384164681 }}, "user": { "id": 249790840, "name": "TriP GoD BooG", "screen_name": "BoogieThaPheen", "lang": "en", "location": "Omaha, Neblastya ", "create_at": date("2011-02-09"), "description": "✈️ 4⃣0⃣2⃣ @jpowe23 @kevyocho @ThatNiggaD_7", "followers_count": 595, "friends_count": 506, "statues_count": 4236 }, "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("2015-11-24T10:00:08.000Z"), "id": 669213930124767232, "text": "S/O to the four babies on this plane����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1429436755, "name": "Annie❥❁➳", "screen_name": "AnnieDaniellee", "lang": "en", "location": "Cali☼ ", "create_at": date("2013-05-14"), "description": "she's cold and she's cruel but she knows what she's doing // Lucas❤️", "followers_count": 235, "friends_count": 136, "statues_count": 5341 }, "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-24T10:00:09.000Z"), "id": 669213930187677696, "text": "My jacket makes me look like a burnt marshmallow. It's ok tho at least I'm warm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 307170853, "name": "Sumi~", "screen_name": "Sumi0494", "lang": "en", "location": "knoxville ", "create_at": date("2011-05-28"), "description": "ebad's sister", "followers_count": 893, "friends_count": 300, "statues_count": 20331 }, "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("2015-11-24T10:00:09.000Z"), "id": 669213930225471488, "text": "This was actually �� https://t.co/UYjAv0OCw0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 523522077, "name": "Dinko", "screen_name": "DinoTursic", "lang": "en", "location": "null", "create_at": date("2012-03-13"), "description": "Find what you love and let it kill you. \nBosnian. Skateboarding. 801.", "followers_count": 275, "friends_count": 223, "statues_count": 7390 }, "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("2015-11-24T10:00:09.000Z"), "id": 669213930229641216, "text": "AND I get my Christmas bonus ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 516718403, "name": "Alisa Obaya", "screen_name": "alisaobaya", "lang": "en", "location": "null", "create_at": date("2012-03-06"), "description": "22. San Antonio, Tejas.", "followers_count": 215, "friends_count": 184, "statues_count": 26369 }, "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-24T10:00:09.000Z"), "id": 669213930368036864, "text": "Truuuuuu https://t.co/VWGVIwBSb1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1332667434, "name": "nani", "screen_name": "Valerieeexo_", "lang": "en", "location": "clutch city , tx", "create_at": date("2013-04-06"), "description": "what do you mean ?", "followers_count": 1327, "friends_count": 1895, "statues_count": 33512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:09.000Z"), "id": 669213930842136576, "text": "2:30 Hurry Up ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467620484, "name": "Mrs. H❤️", "screen_name": "MeMyself_Chey", "lang": "en", "location": "North Philly", "create_at": date("2012-01-18"), "description": "March 25th❤️ K.M.A ❤️ FREE SHIZZ ‼️", "followers_count": 1168, "friends_count": 836, "statues_count": 62830 }, "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-24T10:00:09.000Z"), "id": 669213931135569920, "text": "@imBINGnotGOOGLE @blogsquiat bruh I'm single and I'm the last reply in all my dms", "in_reply_to_status": 669213395959324672, "in_reply_to_user": 97755288, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 97755288, 58394759 }}, "user": { "id": 91308306, "name": "Selfie", "screen_name": "Swishkun", "lang": "en", "location": "null", "create_at": date("2009-11-20"), "description": "@inamorimiyuu ain't my girl. that's my waifu", "followers_count": 1147, "friends_count": 1129, "statues_count": 159192 }, "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("2015-11-24T10:00:09.000Z"), "id": 669213931336916992, "text": "@OmgMikkyRae ��", "in_reply_to_status": 669209793278509056, "in_reply_to_user": 303035280, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 303035280 }}, "user": { "id": 237596131, "name": "Dezmond Scott", "screen_name": "DezScott", "lang": "en", "location": "Dallas, Tx", "create_at": date("2011-01-12"), "description": "TAMUC. Pursuit Of Happiness. A Lion doesn't concern himself with the opinions of sheep", "followers_count": 2002, "friends_count": 1854, "statues_count": 131926 }, "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("2015-11-24T10:00:09.000Z"), "id": 669213931458666496, "text": "But why!? ���������� https://t.co/BUks3O361p", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1473369876, "name": "Dinari Ellis", "screen_name": "doctor_dinari10", "lang": "en", "location": "null", "create_at": date("2013-05-31"), "description": "21, Senior and Corner at IC, Football is in my blood, Peoria High School Alumni ,Follow? @eresbellla❤️ Peoria,IL----Jacksonville,IL #ICBlueboys #DBU #NoFlyZone", "followers_count": 895, "friends_count": 859, "statues_count": 24146 }, "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("2015-11-24T10:00:09.000Z"), "id": 669213931555176448, "text": "@QuiseYuNasty ����", "in_reply_to_status": 669212323865726976, "in_reply_to_user": 158829926, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 158829926 }}, "user": { "id": 920311940, "name": "Dab Sensei™", "screen_name": "1YoungHaitian", "lang": "en", "location": "Haiti ➡ Atlanta, GA", "create_at": date("2012-11-01"), "description": "Kickin' Dab like FIFA | YRN | 19 & Dabbin | OooWwwEee! | ⚾", "followers_count": 1396, "friends_count": 2086, "statues_count": 31071 }, "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("2015-11-24T10:00:09.000Z"), "id": 669213932024954881, "text": "Just 6 days left to get the best price for the highschool winter retreat! Link in bio to sign up?… https://t.co/wuFregkUdN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.73129977,42.73577684"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1452929958, "name": "GF Youth", "screen_name": "gracefellyouth", "lang": "en", "location": "null", "create_at": date("2013-05-23"), "description": "Youth Ministry \nGrace Fellowship - Latham", "followers_count": 167, "friends_count": 21, "statues_count": 1818 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Latham, NY", "id": "00278f1677a13e8d", "name": "Latham", "place_type": "city", "bounding_box": rectangle("-73.832933,42.676375 -73.694717,42.801868") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36001, "countyName": "Albany" } }
+{ "create_at": datetime("2015-11-24T10:00:09.000Z"), "id": 669213932075270144, "text": "@DailyRapFacts Lol @ how that's my uncle on the tv��", "in_reply_to_status": 668929305225728001, "in_reply_to_user": 2297244138, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2297244138 }}, "user": { "id": 246518161, "name": "мєℓαиιє", "screen_name": "cabrejam21", "lang": "en", "location": "null", "create_at": date("2011-02-02"), "description": "Instagram-melaniecabreja", "followers_count": 371, "friends_count": 258, "statues_count": 7349 }, "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("2015-11-24T10:00:09.000Z"), "id": 669213932205142016, "text": "�������� my dad is that Uncle https://t.co/x02cCIzVb0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 321669861, "name": "One Love Welcho", "screen_name": "ChrisusClay", "lang": "en", "location": "In my own lane ", "create_at": date("2011-06-21"), "description": "Trying to reign forever SHSU 18'", "followers_count": 538, "friends_count": 485, "statues_count": 18809 }, "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("2015-11-24T10:00:09.000Z"), "id": 669213932385628161, "text": "https://t.co/3bUxhi7THO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1033396430, "name": "bambi.", "screen_name": "_liljass", "lang": "en", "location": "Huntington, WV", "create_at": date("2012-12-24"), "description": "So many years of education yet nobody ever taught us how to love ourselves & why it's so important.", "followers_count": 1832, "friends_count": 1169, "statues_count": 97145 }, "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("2015-11-24T10:00:09.000Z"), "id": 669213932582752256, "text": "I hate my life I hate my life I hate my life I hate my life I hate my life I hate my life I hate my life I hate my life I hate my life I hat", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3289399217, "name": "Taylor Proctor", "screen_name": "tayp12321", "lang": "en", "location": "Florida, USA", "create_at": date("2015-05-18"), "description": "East Coast FFA Federation President. Deltona Sr. FFA Vice President", "followers_count": 91, "friends_count": 185, "statues_count": 1048 }, "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("2015-11-24T10:00:09.000Z"), "id": 669213932884725760, "text": "Do you know how hard it is to put on and take off snow stuff of four and five year olds?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 447235182, "name": "mad$", "screen_name": "madifishh", "lang": "en", "location": "Milwaukee, WI", "create_at": date("2011-12-26"), "description": "I'm holding onto what I know, & what I know; I must let go. |-/", "followers_count": 276, "friends_count": 477, "statues_count": 17187 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlington, WI", "id": "a6ad1c54aee52c70", "name": "Burlington", "place_type": "city", "bounding_box": rectangle("-88.306198,42.647177 -88.226143,42.712427") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55101, "countyName": "Racine", "cityID": 5511200, "cityName": "Burlington" } }
+{ "create_at": datetime("2015-11-24T10:00:09.000Z"), "id": 669213933106892800, "text": "Wind 3.1 mph S. Barometer 1006.8 hPa, Falling. Temperature 90.3 °F. Rain today 0.01 in. Humidity 10%", "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": 91, "friends_count": 264, "statues_count": 135419 }, "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("2015-11-24T10:00:09.000Z"), "id": 669213933438242817, "text": "i think Monica Geller had the best glow up https://t.co/96TKBQqwht", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3307315074, "name": "Lils", "screen_name": "lilylivealittle", "lang": "en", "location": "orhs ", "create_at": date("2015-08-05"), "description": "7.1.15 Cynthia Green❤️ 18 basketball & track", "followers_count": 194, "friends_count": 164, "statues_count": 3983 }, "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("2015-11-24T10:00:09.000Z"), "id": 669213933581033473, "text": "Just like when whores that swear they live by the bible , but listen to their friends that they are entitled ... smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 273197880, "name": "Gary Brandon", "screen_name": "God_Of_Awsome92", "lang": "en", "location": "Georgetown IL", "create_at": date("2011-03-27"), "description": "Got my own style, My own views, and I'm just me. Be weird, Be different, Screw fitting in, Be nice to people,#NobleTruths \nsnapchat:gcosat32", "followers_count": 448, "friends_count": 399, "statues_count": 35956 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tilton, IL", "id": "01df42715f5d22b0", "name": "Tilton", "place_type": "city", "bounding_box": rectangle("-87.658522,40.065072 -87.595483,40.110157") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17183, "countyName": "Vermilion", "cityID": 1775276, "cityName": "Tilton" } }
+{ "create_at": datetime("2015-11-24T10:00:09.000Z"), "id": 669213933673123840, "text": "Perfect example of my last tweets lmfao https://t.co/JyAFj1aDu7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1626481597, "name": "ur fav african simon", "screen_name": "InjeraOverHoes", "lang": "en", "location": "Dallas,Tejas//NGHS '17", "create_at": date("2013-07-27"), "description": "Curly Haired Eritrean.", "followers_count": 1278, "friends_count": 926, "statues_count": 11055 }, "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("2015-11-24T10:00:09.000Z"), "id": 669213933874581504, "text": "representing JDSD with Alex. school next year should be a blast at Reynoldsburg and with this girl���� https://t.co/asNoMEPS8G", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.816678,39.942149"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1373567257, "name": "brenna♡lewis", "screen_name": "brenna_lewis", "lang": "en", "location": "Columbus, OH", "create_at": date("2013-04-22"), "description": "Forward for the Washington Capitals ;)", "followers_count": 516, "friends_count": 1122, "statues_count": 5468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reynoldsburg High School", "id": "07d9ca0151086006", "name": "Reynoldsburg High School", "place_type": "poi", "bounding_box": rectangle("-82.81667809999999,39.9421489 -82.816678,39.942149") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3966390, "cityName": "Reynoldsburg" } }
+{ "create_at": datetime("2015-11-24T10:00:09.000Z"), "id": 669213933916540928, "text": "HUGE thanks to everyone @NewYorkLiveTv for all the sweet thoughts and photos! ���� @LaurenScala4NY @michaelG4NY @Darlene4NY @katcreag4NY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 73129083, 16256478, 90878730, 90705616, 254108775 }}, "user": { "id": 68438273, "name": "Tumbador Chocolate", "screen_name": "TumbadorTweets", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2009-08-24"), "description": "High-end, private label chocolate company in New York City.", "followers_count": 543, "friends_count": 385, "statues_count": 1107 }, "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-24T10:00:09.000Z"), "id": 669213933975306240, "text": "Happy birthday beautifulbombshell see yall in 2 days.! @ Rinks Plaza https://t.co/MyHRDu5OOM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.54733083,39.24378123"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 104332352, "name": "Pete Murden Rozay", "screen_name": "The_Real_Murden", "lang": "en", "location": "5⃣1⃣3⃣", "create_at": date("2010-01-12"), "description": "God. Family. Respect. Money. Party. Remy. Follow me on iG x The_Real_Murden |Snapchat x Murden88 | Vine x Murden #BnDCasinos #FinalFridays", "followers_count": 1588, "friends_count": 1028, "statues_count": 155706 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Groesbeck, OH", "id": "5e88ab9b08a852b7", "name": "Groesbeck", "place_type": "city", "bounding_box": rectangle("-84.63413,39.219074 -84.471287,39.280186") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3932536, "cityName": "Groesbeck" } }
+{ "create_at": datetime("2015-11-24T10:00:09.000Z"), "id": 669213934084198400, "text": "I need a new job I swear told her from the beginning I don't wanna work 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": 1390436959, "name": "Miscellaneous Mob", "screen_name": "reyez_b", "lang": "en", "location": "Fairmont City, IL", "create_at": date("2013-04-29"), "description": "Mexican in a Subaru WRX. Add me on SnapChat and Instagram: reyezb", "followers_count": 414, "friends_count": 335, "statues_count": 37669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sauget, IL", "id": "de4a838931c5b1b1", "name": "Sauget", "place_type": "city", "bounding_box": rectangle("-90.201166,38.563304 -90.116244,38.607349") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17163, "countyName": "St. Clair", "cityID": 1767756, "cityName": "Sauget" } }
+{ "create_at": datetime("2015-11-24T10:00:09.000Z"), "id": 669213934260473858, "text": "@Kolie_World SHE WAS HITTING THAT SHIT THOUGH LMAO", "in_reply_to_status": 669213708774699008, "in_reply_to_user": 240444579, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 240444579 }}, "user": { "id": 236183661, "name": "Doctor Baddie", "screen_name": "badgalloui", "lang": "en", "location": "PHL✈️NJ", "create_at": date("2011-01-09"), "description": "Perfection is often the only enemy of my greatness. @Rihanna is my liiiiiiife ✨ #SHU18 #RIPJohnnyWyche", "followers_count": 620, "friends_count": 514, "statues_count": 46778 }, "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-24T10:00:09.000Z"), "id": 669213934348468224, "text": "Dawm I lost my beer pong streak to @doovaq �� it was like an hour game though! Best game I've ever played!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 633465867 }}, "user": { "id": 1948540321, "name": "Majin_bluedragon", "screen_name": "CesarCp75", "lang": "en", "location": "null", "create_at": date("2013-10-08"), "description": "create something you love! live for it breathe for it dream about it. and when you create the perfect time you will be successful. Malaise Co-Founder.", "followers_count": 348, "friends_count": 443, "statues_count": 6495 }, "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("2015-11-24T10:00:10.000Z"), "id": 669213934407122944, "text": "Wind 2.0 mph ESE. Barometer 30.215 in, Falling slowly. Temperature 62.3 °F. Rain today 0.00in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.11555556,29.6725"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25210383, "name": "New Braunfels WX", "screen_name": "jeffreycentexwx", "lang": "en", "location": "New Braunfels, TX", "create_at": date("2009-03-18"), "description": "Weather Station located on County Line Road in New Braunfels, TX.\r\n\r\nRealtime Weather @ http://www.nbtx-wx.com\r\nFollow me @ http://twitter.com/jeffreycentex", "followers_count": 368, "friends_count": 21, "statues_count": 156116 }, "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("2015-11-24T10:00:10.000Z"), "id": 669213934507855872, "text": "When God gives you even the slightest bit of joy you gotta take it and run 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": 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": 2354, "friends_count": 1732, "statues_count": 113552 }, "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("2015-11-24T10:00:10.000Z"), "id": 669213934575067137, "text": "@GLBC_Cleveland Christmas Ale just in time for the holidays... https://t.co/z1aasooBK1", "in_reply_to_status": -1, "in_reply_to_user": 18343148, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18343148 }}, "user": { "id": 2471797856, "name": "Remington Tavern", "screen_name": "RemingtonTavern", "lang": "en", "location": "North Tonawanda, NY", "create_at": date("2014-04-30"), "description": "Seafood Exchange / Raw Bar / Private Party Catering / Valet / Call for Reservations (716) 362-2802 Come enjoy our Menu, Daily Features & Happy Hour!", "followers_count": 617, "friends_count": 648, "statues_count": 538 }, "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("2015-11-24T10:00:10.000Z"), "id": 669213934621102080, "text": "Why You Shouldn't Escape Into Social Media To Avoid Holiday Stress https://t.co/W1qyztnHtB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.3718169,25.6891713"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 363944963, "name": "Adriana Serna", "screen_name": "IamAdrianaSerna", "lang": "en", "location": "Miami, FL", "create_at": date("2011-08-28"), "description": "Multilingual marketing, social media and communications consultant and speaker. Click here to LIKE me on Facebook http://on.fb.me/RbDYW2", "followers_count": 916, "friends_count": 963, "statues_count": 4230 }, "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("2015-11-24T10:00:10.000Z"), "id": 669213934717501440, "text": "@vgunvalson Maybe should make an effort & reach out to Cancer patients??", "in_reply_to_status": 669210417437270016, "in_reply_to_user": 27588709, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27588709 }}, "user": { "id": 2962108945, "name": "teri chamberlin", "screen_name": "teri_chamberlin", "lang": "en", "location": "Boeing. everett, seattle, wa", "create_at": date("2015-01-06"), "description": "null", "followers_count": 137, "friends_count": 165, "statues_count": 4373 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Silver Firs, WA", "id": "003314b86e295845", "name": "Silver Firs", "place_type": "city", "bounding_box": rectangle("-122.187769,47.834183 -122.111025,47.891348") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5364380, "cityName": "Silver Firs" } }
+{ "create_at": datetime("2015-11-24T10:00:10.000Z"), "id": 669213934763704320, "text": "��AHORA MARTES BEN Y COMPARTE��\n��ENTRADA Gratis Asta 1am\n��@jaudgetcrazy CANTANDO EN VIVO��\n��MUSIC BY… https://t.co/fNjRm8Tg2b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.6249924,40.7083893"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 1355525791, "name": "Anthony Moran", "screen_name": "WAVEYSHOTNY", "lang": "en", "location": "waveyLand entrepreneur ", "create_at": date("2013-04-15"), "description": "CEO:waveyshotny/waveyent \nPROMOTER/PHOTOGRAPHER\nYOUNGER ENTREPRENEUR!!", "followers_count": 464, "friends_count": 567, "statues_count": 1511 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hempstead, NY", "id": "10d2f6857b8ac152", "name": "Hempstead", "place_type": "city", "bounding_box": rectangle("-73.643328,40.684538 -73.598486,40.72134") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3633139, "cityName": "Hempstead" } }
+{ "create_at": datetime("2015-11-24T10:00:10.000Z"), "id": 669213934923161600, "text": "OH EVEN FUCKING BETTER. You are 10 shut up and know your place. Bitch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 713327269, "name": "Dαииу✯", "screen_name": "danny_hanratty", "lang": "en", "location": "null", "create_at": date("2012-07-23"), "description": "Snapchat: dannyhanrattyxo ✨", "followers_count": 249, "friends_count": 339, "statues_count": 1286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kings Park, NY", "id": "5891cef221153945", "name": "Kings Park", "place_type": "city", "bounding_box": rectangle("-73.279475,40.866222 -73.21189,40.909936") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3639672, "cityName": "Kings Park" } }
+{ "create_at": datetime("2015-11-24T10:00:10.000Z"), "id": 669213935124418561, "text": "Practicing popcorn words during workshop. https://t.co/E6UyIKhZ9C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1409263393, "name": "Ellie Brown", "screen_name": "mrsbrownkinder", "lang": "en", "location": "null", "create_at": date("2013-05-06"), "description": "null", "followers_count": 64, "friends_count": 30, "statues_count": 55 }, "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("2015-11-24T10:00:10.000Z"), "id": 669213935145517056, "text": "Se espera que oleada de cubanos aumente los robos y las violaciones en C Rica", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 514999068, "name": "Alex Louisville", "screen_name": "alexlouisville", "lang": "en", "location": "New Albany, IN", "create_at": date("2012-03-04"), "description": "Policy, Laws, Immigration and music", "followers_count": 706, "friends_count": 2040, "statues_count": 1725 }, "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-24T10:00:10.000Z"), "id": 669213935355224064, "text": "Thursday night we making movies happy bday to the homie KD we going in!!!! @ Southgate Barber Shop https://t.co/OaVA0KwL1z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.61480081,35.75474472"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 284805916, "name": "Dj 919", "screen_name": "919Dj", "lang": "en", "location": "For booking call 919-279-0915 ", "create_at": date("2011-04-19"), "description": "Instagram dj919", "followers_count": 1854, "friends_count": 927, "statues_count": 10822 }, "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("2015-11-24T10:00:10.000Z"), "id": 669213935372009473, "text": "@Conceal @lxcaldreamer definitely", "in_reply_to_status": 669213527538798592, "in_reply_to_user": 3644974693, "favorite_count": 0, "coordinate": point("-75.94076,40.346172"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3644974693, 2884929017 }}, "user": { "id": 1634312190, "name": "Nov 27", "screen_name": "6vx", "lang": "en", "location": "Reading, PA", "create_at": date("2013-07-30"), "description": "Dont Take Me Serious.", "followers_count": 2586, "friends_count": 167, "statues_count": 73335 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aprils' III", "id": "07d9efe04ac88000", "name": "Aprils' III", "place_type": "poi", "bounding_box": rectangle("-75.94076009999999,40.3461719 -75.94076,40.346172") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42011, "countyName": "Berks", "cityID": 4263624, "cityName": "Reading" } }
+{ "create_at": datetime("2015-11-24T10:00:10.000Z"), "id": 669213935568973824, "text": "\"What are your grades looking like?\" https://t.co/diqRLNDB1k", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2309335239, "name": "Lawrence", "screen_name": "LarryBoy922", "lang": "en", "location": "304", "create_at": date("2014-01-28"), "description": "Fresh Prince of Morehead", "followers_count": 769, "friends_count": 644, "statues_count": 13161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morehead, KY", "id": "fdbee96c7a1800b8", "name": "Morehead", "place_type": "city", "bounding_box": rectangle("-83.474889,38.148113 -83.39478,38.215006") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21205, "countyName": "Rowan", "cityID": 2153418, "cityName": "Morehead" } }
+{ "create_at": datetime("2015-11-24T10:00:10.000Z"), "id": 669213935690579968, "text": "@Moz will do. Keep up the good work!", "in_reply_to_status": 669213745617342465, "in_reply_to_user": 15651700, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15651700 }}, "user": { "id": 3331546606, "name": "August Noble", "screen_name": "DigitalQuinn", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-06-17"), "description": "Millennial Marketer | Get Better Everyday #ContentMarketing #SEO #Millennials", "followers_count": 166, "friends_count": 146, "statues_count": 237 }, "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("2015-11-24T10:00:10.000Z"), "id": 669213935724269568, "text": "This will always make me happy https://t.co/3bDzlewwbz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 85169138, "name": "hypothetically unlit", "screen_name": "serdric", "lang": "en", "location": "null", "create_at": date("2009-10-25"), "description": "bad at life//USF//bad sports fan//bad with people", "followers_count": 630, "friends_count": 1241, "statues_count": 75097 }, "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("2015-11-24T10:00:10.000Z"), "id": 669213935753539585, "text": "@Shae_Frost80 put that thing in reverse and roll out!!! ��", "in_reply_to_status": 669213791284932609, "in_reply_to_user": 1198748323, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1198748323 }}, "user": { "id": 2319500011, "name": "JodiLynn.", "screen_name": "JodiLynn66", "lang": "en", "location": "null", "create_at": date("2014-01-30"), "description": "(: :)", "followers_count": 565, "friends_count": 473, "statues_count": 21656 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Breaux Bridge, LA", "id": "566de916698ce1f1", "name": "Breaux Bridge", "place_type": "city", "bounding_box": rectangle("-91.963297,30.220192 -91.809486,30.319226") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22099, "countyName": "St. Martin", "cityID": 2209340, "cityName": "Breaux Bridge" } }
+{ "create_at": datetime("2015-11-24T10:00:10.000Z"), "id": 669213935979986946, "text": "mhm https://t.co/dleQJ3j4iI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1226384509, "name": "Ꭶ", "screen_name": "sandywang_", "lang": "en", "location": "htx", "create_at": date("2013-02-27"), "description": "whatever happens happens", "followers_count": 371, "friends_count": 262, "statues_count": 9979 }, "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-24T10:00:10.000Z"), "id": 669213936193961984, "text": "Afternoon showers this afternoon, high 73 (23 C). Low 66 (19 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712593, "name": "CorpusChristiWX", "screen_name": "CorpusChristiWX", "lang": "en", "location": "Corpus Christi, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Corpus Christi, Texas", "followers_count": 1005, "friends_count": 93, "statues_count": 7991 }, "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("2015-11-24T10:00:10.000Z"), "id": 669213936315719682, "text": "@Trinidad_Lani lmao the walking dead be off the shits son", "in_reply_to_status": 669212023331385346, "in_reply_to_user": 98973082, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 98973082 }}, "user": { "id": 253403652, "name": "Lord Baelish", "screen_name": "pour_MYcaptain", "lang": "en", "location": "Milky Way", "create_at": date("2011-02-16"), "description": "expand your horizons ✌", "followers_count": 383, "friends_count": 316, "statues_count": 23148 }, "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("2015-11-24T10:00:10.000Z"), "id": 669213936428826624, "text": "Footage as a unit of measure??", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 104315595, "name": "Jorge De Los Santos", "screen_name": "Ame1916", "lang": "en", "location": "Houston, Texas", "create_at": date("2010-01-12"), "description": "Q-vo!!", "followers_count": 33, "friends_count": 230, "statues_count": 6165 }, "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("2015-11-24T10:00:10.000Z"), "id": 669213936974041089, "text": "Celebrating in style with the @PANDORA_NA adcampaign in @voguemagazine 's latest issue! https://t.co/btDXJclYuj https://t.co/Y5wS6pgTsF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 273957418, 136361303 }}, "user": { "id": 32496437, "name": "CHRISELLE LIM", "screen_name": "CHRISELLEtweets", "lang": "en", "location": "Los Angeles", "create_at": date("2009-04-17"), "description": "• Stylist, editor, creative mind & soul • French Bulldog enthusiast • wifey • new mommy to #chloevictoriachen •", "followers_count": 45327, "friends_count": 586, "statues_count": 29089 }, "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("2015-11-24T10:00:10.000Z"), "id": 669213937225723905, "text": "#thisisrena can you write a song about pepperoni pizza", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thisisrena" }}, "user": { "id": 3168026600, "name": "asshh", "screen_name": "rejectsqueen", "lang": "en", "location": "rowyso j.21 | bae area", "create_at": date("2015-04-22"), "description": "i am bryana holly trash", "followers_count": 271, "friends_count": 212, "statues_count": 3495 }, "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("2015-11-24T10:00:10.000Z"), "id": 669213937456447488, "text": "Slow moving maintenance vehicle, right lane closed in #Detroit on M-39 SB from 8 Mile Rd to Pinecrest Dr #traffic https://t.co/93KUmrUJbm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.21961,42.44416"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Detroit", "traffic" }}, "user": { "id": 249409119, "name": "TTWN Detroit", "screen_name": "TotalTrafficDET", "lang": "en", "location": "Detroit, MI", "create_at": date("2011-02-08"), "description": "Stuck in a slowdown? Give us a call on the 24 hour tipline 248-356-7550. We'll say your name on the air!", "followers_count": 259, "friends_count": 34, "statues_count": 43005 }, "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-24T10:00:10.000Z"), "id": 669213937754202112, "text": "I want so much more out of life instead of just working I just don't know what route to take", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2789242629, "name": "ky$", "screen_name": "k_eazyy13", "lang": "en", "location": "null", "create_at": date("2014-09-28"), "description": "don't let your dreams be dreams", "followers_count": 97, "friends_count": 97, "statues_count": 4414 }, "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("2015-11-24T10:00:10.000Z"), "id": 669213937808879616, "text": "IM HOME & HAPPY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1586637332, "name": "alex", "screen_name": "alexfischh", "lang": "en", "location": "null", "create_at": date("2013-07-11"), "description": "hello", "followers_count": 235, "friends_count": 170, "statues_count": 4451 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin Square, NY", "id": "4bfbb13bd35cf5e0", "name": "Franklin Square", "place_type": "city", "bounding_box": rectangle("-73.692411,40.682829 -73.661034,40.717994") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3627309, "cityName": "Franklin Square" } }
+{ "create_at": datetime("2015-11-24T10:00:10.000Z"), "id": 669213938026876928, "text": "Obama calls on EU to finally implement agreement to share passenger information to look for returning foreign… https://t.co/dyAfnJ9zbO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.6155346,39.30190319"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 66179557, "name": "Baltimore Press", "screen_name": "BaltimoreCP", "lang": "en", "location": "Baltimore, MD", "create_at": date("2009-08-16"), "description": "Latest news from Baltimore. Updates are frequent.", "followers_count": 2857, "friends_count": 37, "statues_count": 186908 }, "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-24T10:00:10.000Z"), "id": 669213938299465728, "text": "Nice! https://t.co/88fkYf4epg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2305365672, "name": "Abigail Pelzer", "screen_name": "AbigailPelzer", "lang": "en", "location": "Newton, Iowa", "create_at": date("2014-01-22"), "description": "Newton Daily News. Beloved wife & mother. Go Hawks.", "followers_count": 262, "friends_count": 633, "statues_count": 1505 }, "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("2015-11-24T10:00:10.000Z"), "id": 669213938400145409, "text": "Thank you @K_A_B for the Waste Works Grant awarded to our PTA to better promote recycling this year! @rountreerr https://t.co/8Z4cWxCF5o", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23458033, 771614966 }}, "user": { "id": 1608279158, "name": "Rountree PTA", "screen_name": "PTARountree", "lang": "en", "location": "Allen, TX", "create_at": date("2013-07-20"), "description": "null", "followers_count": 117, "friends_count": 90, "statues_count": 315 }, "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("2015-11-24T10:00:10.000Z"), "id": 669213938454663168, "text": "Catch me in the parking lot jamming to @BlkStoneCherry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20007424 }}, "user": { "id": 3252338130, "name": "Jive Turkey", "screen_name": "nativemamba1", "lang": "en", "location": "Tom Brady's house, OK", "create_at": date("2015-06-21"), "description": "if you want to chat or hang out click the link http://farmersonly.com", "followers_count": 442, "friends_count": 407, "statues_count": 3261 }, "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("2015-11-24T10:00:10.000Z"), "id": 669213938517577728, "text": "Which one of my followers go to MST ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 558311802, "name": "FUCK MICHIGAN", "screen_name": "MekhiMoore_", "lang": "en", "location": "Buffalo, NY ", "create_at": date("2012-04-19"), "description": "C/O 2016 |Capricorn ♑ #LakersNation #ChargersNation⚡️ #BuckeyeNation", "followers_count": 1175, "friends_count": 743, "statues_count": 21729 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213938677112833, "text": "Refurbishing this heart shaped Vintage Compact with Holiday shades. $65 #paducah #shoplocal… https://t.co/E0Ixp5Y8Z4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.5946121,37.0869598"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "paducah", "shoplocal" }}, "user": { "id": 1571412739, "name": "FACE Makeup Artistry", "screen_name": "facemakeupart", "lang": "en", "location": "125 Kentucky Avenue", "create_at": date("2013-07-05"), "description": "Makeup Artist inspired by the past, present, & future. Dedicated to using minimal products in dreamy shades & colors to create custom looks!", "followers_count": 641, "friends_count": 208, "statues_count": 2863 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213938719006720, "text": "Tell him bring another zip", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 244663691, "name": "Just Ja$", "screen_name": "theprincess_jas", "lang": "en", "location": "Everywhere you wish you were", "create_at": date("2011-01-29"), "description": "Ghanaian Princess", "followers_count": 543, "friends_count": 307, "statues_count": 17008 }, "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-24T10:00:11.000Z"), "id": 669213938769244160, "text": "I seriously need this dog in my life https://t.co/tKlGJzVMes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1094262078, "name": "Mari➡️EMCF16", "screen_name": "20MarissaRuiz12", "lang": "en", "location": "Corpus Christi, TX", "create_at": date("2013-01-15"), "description": "I like to lift heavy things and rave. @EuphoriaMusFest is my home ΣΦΕ Sweetheart ΑΓΔ #Basshead #goodvibetribe #TAMUCC #TexasEDMFam #fitfam IG:rissyy143", "followers_count": 829, "friends_count": 783, "statues_count": 26563 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213938870013957, "text": "@juuuulllia �������� all day everyday. https://t.co/eMHUcuIr1D", "in_reply_to_status": -1, "in_reply_to_user": 1013304601, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1013304601 }}, "user": { "id": 372970109, "name": "Janelle .", "screen_name": "bdxnell", "lang": "en", "location": "ATL⚡️229", "create_at": date("2011-09-13"), "description": "just a funny girl with an honest tongue & call me Nell.", "followers_count": 1559, "friends_count": 745, "statues_count": 46856 }, "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-24T10:00:11.000Z"), "id": 669213939088142336, "text": "So low omg #stance #scrapedaily #lowlife https://t.co/JH34gPdePV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "stance", "scrapedaily", "lowlife" }}, "user": { "id": 96650054, "name": "linvillle", "screen_name": "NatalieLinville", "lang": "en", "location": "Sunshine State ☀️", "create_at": date("2009-12-13"), "description": "07.19.15❤️ #bluelivesmatter fly high ryder❤️ @Braves may suck but I love em ⚾️", "followers_count": 702, "friends_count": 362, "statues_count": 14477 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213939113177088, "text": "cant wait for the weeknd @Hola_shellyXO @damndavidd ugh gives me chills!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2685895448, 328305902 }}, "user": { "id": 2172995268, "name": "huny bunz", "screen_name": "biglexi", "lang": "en", "location": "null", "create_at": date("2013-11-03"), "description": "a$$ like jlo angel wit the halo", "followers_count": 188, "friends_count": 119, "statues_count": 8204 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213939247411200, "text": "Feels like Friday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1852930537, "name": "R e e z o ⚡️", "screen_name": "eatmyretros", "lang": "en", "location": "@jaayd0llaaz was here ❗️", "create_at": date("2013-09-10"), "description": "Dreamchasing ✈️ Blessed ✨", "followers_count": 217, "friends_count": 131, "statues_count": 4400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:11.000Z"), "id": 669213939310333952, "text": "My mom said I was getting too skinny so I ate 4 donuts. Hope she's happy cuz 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": 2544456433, "name": "❀Holly❀", "screen_name": "DebenhamHolly", "lang": "en", "location": "null", "create_at": date("2014-06-03"), "description": "Give your brain as much attention as your hair and you'll be a thousand times better off", "followers_count": 206, "friends_count": 142, "statues_count": 605 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213939570515968, "text": "@lyyyyyyyyynn Happy Birthday! ☺️��", "in_reply_to_status": -1, "in_reply_to_user": 3004775649, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3004775649 }}, "user": { "id": 3316433347, "name": "Jordyn ❤️", "screen_name": "JMarie96_", "lang": "en", "location": "DET | BG", "create_at": date("2015-08-15"), "description": "A small girl with big dreams. 18. BGSU Twirler. #bgsu18", "followers_count": 180, "friends_count": 186, "statues_count": 403 }, "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-24T10:00:11.000Z"), "id": 669213939587268608, "text": "https://t.co/ftX0noMSBs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2552703848, "name": "Anthony", "screen_name": "__Kessel", "lang": "en", "location": "null", "create_at": date("2014-06-07"), "description": "#Pens // #Juve // Owner of @StatsNupdates // @coupeNYR //@McDonaghs_Ego // Phil Kessel is a Pittsburgh Penguin //", "followers_count": 1611, "friends_count": 283, "statues_count": 35129 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213939834728449, "text": "@ArlenaAmaro :( difícil situación my Dear!", "in_reply_to_status": 669212834740350977, "in_reply_to_user": 133789902, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 133789902 }}, "user": { "id": 65211826, "name": "VANESSA HAUC", "screen_name": "vanessahauc", "lang": "en", "location": "USA", "create_at": date("2009-08-12"), "description": "Mother, Immigrant, Telemundo Network Emmy Award winning Journalist, Green at Heart- God is my Strength! \nhttp://www.sachamama.org #LatinosPorLaTierra", "followers_count": 23749, "friends_count": 1177, "statues_count": 9467 }, "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-24T10:00:11.000Z"), "id": 669213939876655104, "text": "reason number 740281591959 why we're bestfriends https://t.co/7NJiPYol8I", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 989999334, "name": "briana lee", "screen_name": "shearer_13", "lang": "en", "location": "Florida, USA", "create_at": date("2012-12-04"), "description": "{Child of the King}", "followers_count": 455, "friends_count": 439, "statues_count": 4197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Live Oak, FL", "id": "0c299b40190868f6", "name": "Live Oak", "place_type": "city", "bounding_box": rectangle("-83.012792,30.275062 -82.96204,30.31192") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12121, "countyName": "Suwannee", "cityID": 1240875, "cityName": "Live Oak" } }
+{ "create_at": datetime("2015-11-24T10:00:11.000Z"), "id": 669213940107386884, "text": "@kennethhow2 @Kevinfor7 those are definitely better options!! They are undeniably longer and span multiple wards. I'd support!", "in_reply_to_status": 669213578445111296, "in_reply_to_user": 3317837682, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3317837682, 339283438 }}, "user": { "id": 515038894, "name": "Eric B. Chavous", "screen_name": "DrChavous", "lang": "en", "location": "Pittsburgh via Washington DC", "create_at": date("2012-03-04"), "description": "JD/MBA UPitt '15 | Howard University Alum | Former D1 Athlete | Sports & Entertainment Law Enthusiast.", "followers_count": 351, "friends_count": 366, "statues_count": 3309 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213940166078464, "text": "https://t.co/PzWk32mWnb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3709722135, "name": "Caity", "screen_name": "__Caitypie", "lang": "en", "location": "Tampa, FL", "create_at": date("2015-09-19"), "description": "CFES | 21 | Lightning⚡️| Hot mess express | Sass", "followers_count": 274, "friends_count": 268, "statues_count": 2402 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213940174356480, "text": "@thebrycewilliam selfie", "in_reply_to_status": 669207704188006400, "in_reply_to_user": 794908129, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 794908129 }}, "user": { "id": 3583881498, "name": "Alexandria Renee", "screen_name": "alexandrirenee", "lang": "en", "location": "null", "create_at": date("2015-09-16"), "description": "null", "followers_count": 39, "friends_count": 53, "statues_count": 430 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213940220563457, "text": "❤️❤️❤️❤️❤️❤️❤️❤️❤️❤️❤️❤️❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 103675432, "name": "lyss", "screen_name": "haiminnie", "lang": "en", "location": "null", "create_at": date("2010-01-10"), "description": "RIP JAV", "followers_count": 1964, "friends_count": 853, "statues_count": 50123 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213940300316673, "text": "@sam_flessel omg �������� yeah definitely not!!!", "in_reply_to_status": 669211693944295425, "in_reply_to_user": 2756661019, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2756661019 }}, "user": { "id": 727986422, "name": "summer locking", "screen_name": "SunnyLocking", "lang": "en", "location": "Buffalo, NY", "create_at": date("2012-07-31"), "description": "Buffalo ,NY 20 years Instagram:Summerlocking #EDCNY2015", "followers_count": 1148, "friends_count": 545, "statues_count": 16166 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hamburg, NY", "id": "0032b71e19053652", "name": "Hamburg", "place_type": "city", "bounding_box": rectangle("-78.916774,42.696345 -78.783331,42.789724") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3631643, "cityName": "Hamburg" } }
+{ "create_at": datetime("2015-11-24T10:00:11.000Z"), "id": 669213940547645440, "text": "Accident in #Fairfax on I-66 EB between VA-123 and Vienna Metro Station, jammed back to US-50, delay of 9 mins #DCtraffic", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.30894,38.86895"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Fairfax", "DCtraffic" }}, "user": { "id": 62875538, "name": "TTWN DC", "screen_name": "TotalTrafficDC", "lang": "en", "location": "Washington DC", "create_at": date("2009-08-04"), "description": "Traffic Tip Line 1.877.TTN.JAMS", "followers_count": 618, "friends_count": 136, "statues_count": 46049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakton, VA", "id": "015a1efa44b8123a", "name": "Oakton", "place_type": "city", "bounding_box": rectangle("-77.395515,38.863569 -77.253414,38.930322") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5158472, "cityName": "Oakton" } }
+{ "create_at": datetime("2015-11-24T10:00:11.000Z"), "id": 669213940891516928, "text": "Watch the lakers beat the Warriors tonight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1173753901, "name": "RAyShAuN", "screen_name": "WavyRJ", "lang": "en", "location": "IE,", "create_at": date("2013-02-12"), "description": "@kiannaaamariee ❤️", "followers_count": 1456, "friends_count": 1315, "statues_count": 42745 }, "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-24T10:00:11.000Z"), "id": 669213940891693056, "text": "@allyyyjane ��", "in_reply_to_status": 669198591433412609, "in_reply_to_user": 195964729, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 195964729 }}, "user": { "id": 151707313, "name": "annalaura maranville", "screen_name": "HatyPerry", "lang": "en", "location": "louisville, ky", "create_at": date("2010-06-03"), "description": "vegan | pretty ok to be around", "followers_count": 462, "friends_count": 556, "statues_count": 13675 }, "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-24T10:00:11.000Z"), "id": 669213941676056576, "text": "Wind 3.8 mph WNW. Barometer 30.388 in, Rising. Temperature 34.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": 6, "friends_count": 43, "statues_count": 4610 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213941780897792, "text": "My $AGN and $RTRX long positions are pulling the train - one that was almost derailed after yesterday's mauling...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 245069853, "name": "Zachary Prensky", "screen_name": "Zackfoot", "lang": "en", "location": "null", "create_at": date("2011-01-30"), "description": "Special Situations are my stock in trade. Follow me as I place binary bets on takeovers, frauds & transformative corporate events - and hopefully profitable!!", "followers_count": 2502, "friends_count": 262, "statues_count": 8641 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213941873123328, "text": "Yes. https://t.co/jzZDNwS7bT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 64040678, "name": "harrys knee caps", "screen_name": "nocontrolnyaz", "lang": "en", "location": "NYC-MIA", "create_at": date("2009-08-08"), "description": "Louis has high notes", "followers_count": 629, "friends_count": 192, "statues_count": 36707 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213941902372864, "text": "Wind 6.0 mph S. Barometer 29.784 in, Steady. Temperature 42.4 °F. Rain today 0.16 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": 191, "friends_count": 58, "statues_count": 228528 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213942066061312, "text": "@nicholsnelson interesting", "in_reply_to_status": 669196042525663232, "in_reply_to_user": 576647651, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 576647651 }}, "user": { "id": 23514851, "name": "Derek Chartt Scott", "screen_name": "ddritzenhein", "lang": "en", "location": "Waukesha, WI", "create_at": date("2009-03-09"), "description": "The Hub Of Distance Running & Honorary Member Of Brooks Beast Pro Distance Running Group & Melbourne Track Club", "followers_count": 2579, "friends_count": 1167, "statues_count": 275481 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213942103707649, "text": "Obama says ISIL cannot defeat us on battlefield but we cannot succumb to fear we did not waver after Ft Hood Boston… https://t.co/dyAfnJ9zbO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.6155346,39.30190319"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 66179557, "name": "Baltimore Press", "screen_name": "BaltimoreCP", "lang": "en", "location": "Baltimore, MD", "create_at": date("2009-08-16"), "description": "Latest news from Baltimore. Updates are frequent.", "followers_count": 2857, "friends_count": 37, "statues_count": 186909 }, "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-24T10:00:11.000Z"), "id": 669213942175019008, "text": "Wind 2.1 mph W. Barometer 30.275 in, Falling. Temperature 61.6 °F. Rain today 0.00 in. Humidity 39%", "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": 22, "statues_count": 52017 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213942212784128, "text": "so ready to get my nails & toes done tomorrow.", "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❤️. with August Alsina. ", "create_at": date("2011-11-21"), "description": "#CosmetologyStudent.", "followers_count": 1699, "friends_count": 1018, "statues_count": 191588 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213942414221312, "text": "Order @LastDragonPizza on your iPhone using the Homemade Food app by Grazer Labs, inc.\nStop rummaging… https://t.co/msMVweNdno", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.85786574,40.57380576"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2329390400 }}, "user": { "id": 2329390400, "name": "LAST DRAGON PIZZA", "screen_name": "LASTDRAGONPIZZA", "lang": "en", "location": "Arverne, NY", "create_at": date("2014-02-05"), "description": "#PizzaQueen Serving friends in #ROCKAWAY #NYC on WEDNESDAYS+FRIDAYS+SATURDAYS. Info: 917-780-2570. #Pizza Follow on FB & INSTAGRAM! https://youtu.be/m0OM1rm7koE", "followers_count": 368, "friends_count": 730, "statues_count": 1965 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213942451970048, "text": "@Ljusha7 @62Roses @Ibnelkayeem @loyal_cupid @drouchemed58 @ranasilvia63 @Asian_Poetry @zbmkhan @GCHDutta @almohid https://t.co/wMnHrytl8y", "in_reply_to_status": 669213884583161856, "in_reply_to_user": 48875310, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2965652843, 1409420528, 2963445490, 17308392, 2264139201, 2750799059, 1256913534, 3010331497, 1721848128, 140078994 }}, "user": { "id": 48875310, "name": "Alhaj Nasir Uddin", "screen_name": "MrNaassir", "lang": "en", "location": "New Yark City.New York. U.S.A.", "create_at": date("2009-06-19"), "description": "I am from Pakistan & U.S. Citizen. Love Photography, Flowers, Birds, Scenery & Nature.Please Type MrNaassir in Google & See my 16000 pictures Thanks. NO DM PLZ.", "followers_count": 5431, "friends_count": 2345, "statues_count": 134115 }, "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("2015-11-24T10:00:11.000Z"), "id": 669213942707650560, "text": "Obama: 'I want you to know that we will do everything in our power to defend our nation' https://t.co/OzaOBTaZWf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.03733586,38.90181532"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 81913437, "name": "Washington Press", "screen_name": "WashingtonCP", "lang": "en", "location": "Washington, DC", "create_at": date("2009-10-12"), "description": "Latest news from Washington, DC. Updates are frequent. For local news addicts.", "followers_count": 1406, "friends_count": 12, "statues_count": 140512 }, "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("2015-11-24T10:00:12.000Z"), "id": 669213942888165376, "text": "Johnny Cantgetright Manziel", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 198704549, "name": "L E W ", "screen_name": "IsThatJohnLEW_", "lang": "en", "location": "Dayton, Ohio", "create_at": date("2010-10-04"), "description": "Marketing Major. Tiffin University Student-Athlete C/o 18 #GoDragons You Can Either Be A Lion In The Jungle, Or A Lion In The Zoo.", "followers_count": 603, "friends_count": 853, "statues_count": 18828 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tiffin, OH", "id": "7161e3a5f968ce38", "name": "Tiffin", "place_type": "city", "bounding_box": rectangle("-83.224564,41.086231 -83.145003,41.143338") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39147, "countyName": "Seneca", "cityID": 3976778, "cityName": "Tiffin" } }
+{ "create_at": datetime("2015-11-24T10:00:12.000Z"), "id": 669213943122948096, "text": "@chase_keye15 thanks dude! ��", "in_reply_to_status": 668943527049498624, "in_reply_to_user": 442261065, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 442261065 }}, "user": { "id": 344508120, "name": "Tanner Robideaux", "screen_name": "TRobideaux", "lang": "en", "location": "Duluth, MN", "create_at": date("2011-07-28"), "description": "2128 Jump Street.", "followers_count": 830, "friends_count": 409, "statues_count": 3187 }, "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("2015-11-24T10:00:12.000Z"), "id": 669213943181643776, "text": "@meliapimentel no mames yo no pasó el nivel 30 de candy jajajaja", "in_reply_to_status": 669170406016716800, "in_reply_to_user": 81450975, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 81450975 }}, "user": { "id": 425353136, "name": "Andrea S", "screen_name": "AndyEckstein", "lang": "es", "location": "11/06/14 ♡ JDA", "create_at": date("2011-11-30"), "description": "Vivía y ante todo hacía vivir", "followers_count": 582, "friends_count": 338, "statues_count": 18633 }, "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("2015-11-24T10:00:12.000Z"), "id": 669213943328473088, "text": "New episode of #Gotham then rec ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Gotham" }}, "user": { "id": 32137018, "name": "ĆrysonTiller", "screen_name": "JetJohnson92", "lang": "en", "location": "Denton, TX", "create_at": date("2009-04-16"), "description": "Dorian Johnson #UNT", "followers_count": 1930, "friends_count": 981, "statues_count": 85291 }, "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-24T10:00:12.000Z"), "id": 669213943491985411, "text": "Got a package and it was a 3 course meal with a chocolate cake sent from my parents since im staying here for Thanksgiving I love 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": 220821331, "name": "Ali", "screen_name": "aligroholski", "lang": "en", "location": "Connecticut, USA", "create_at": date("2010-11-28"), "description": "It's pronounced Allie, not Ol-li. My mom was trying to be different [Kappa Delta]", "followers_count": 521, "friends_count": 374, "statues_count": 11951 }, "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("2015-11-24T10:00:12.000Z"), "id": 669213943609565184, "text": "Mean mug mode activated cause all these black people think they really doing something ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 961680884, "name": "SEXY BALDHEAD", "screen_name": "PeaceNdSerenity", "lang": "en", "location": "Cincinnati, OH ", "create_at": date("2012-11-20"), "description": "NKU'19 | R.I.P Lee Lee | snapchat: slim_serenity | BLESSED", "followers_count": 1773, "friends_count": 1910, "statues_count": 21555 }, "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("2015-11-24T10:00:12.000Z"), "id": 669213943626227712, "text": "Mostly sunny this afternoon, high 63 (17 C). Low 50 (10 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712706, "name": "TulsaWX", "screen_name": "TulsaWX", "lang": "en", "location": "Tulsa, Oklahoma", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Tulsa, Oklahoma", "followers_count": 510, "friends_count": 93, "statues_count": 8053 }, "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("2015-11-24T10:00:12.000Z"), "id": 669213943806730240, "text": "@Leone_SN @MrDanZak Elon should show a bit of humility until Falcon 9 is back in regular operation", "in_reply_to_status": 669208324341002240, "in_reply_to_user": 314549077, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 314549077, 66267332 }}, "user": { "id": 38279291, "name": "Doug Mohney", "screen_name": "DougonIPComm", "lang": "en", "location": "DC area", "create_at": date("2009-05-06"), "description": "An independent voice on VoIP, telecom, IP comm, and HD voice.", "followers_count": 1934, "friends_count": 86, "statues_count": 13746 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franconia, VA", "id": "7ca522a59e592638", "name": "Franconia", "place_type": "city", "bounding_box": rectangle("-77.184452,38.739182 -77.119991,38.798553") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5129552, "cityName": "Franconia" } }
+{ "create_at": datetime("2015-11-24T10:00:12.000Z"), "id": 669213943890489344, "text": "JUST IN: Pres. Obama on shoot down of Russian jet: \"Turkey has a right to defend its territory and its airspace.\" https://t.co/dyAfnJ9zbO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.6155346,39.30190319"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 66179557, "name": "Baltimore Press", "screen_name": "BaltimoreCP", "lang": "en", "location": "Baltimore, MD", "create_at": date("2009-08-16"), "description": "Latest news from Baltimore. Updates are frequent.", "followers_count": 2857, "friends_count": 37, "statues_count": 186910 }, "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-24T10:00:12.000Z"), "id": 669213943911567360, "text": "FREE BULL ❗️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1184310733, "name": "bèllà.", "screen_name": "prettysamoneee", "lang": "en", "location": "null", "create_at": date("2013-02-15"), "description": "unbothered.. R.I.P. Aries ❤️", "followers_count": 1190, "friends_count": 1292, "statues_count": 27785 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valley, AL", "id": "dd5d995aeda1ccf8", "name": "Valley", "place_type": "city", "bounding_box": rectangle("-85.224667,32.752675 -85.140283,32.856282") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1017, "countyName": "Chambers", "cityID": 178204, "cityName": "Valley" } }
+{ "create_at": datetime("2015-11-24T10:00:12.000Z"), "id": 669213944054202368, "text": "From our office to you: Wishing you all a joyous Thanksgiving filled with the blessings of friends and family! https://t.co/RfxEMDSxe4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2801985503, "name": "SBU Bookstore", "screen_name": "SBUBookstore", "lang": "en", "location": "Stony Brook, New York", "create_at": date("2014-10-03"), "description": "Official Twitter account of the Stony Brook University Bookstore. This profile is maintained by the Bookstore Liaison's Office.", "followers_count": 256, "friends_count": 236, "statues_count": 683 }, "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("2015-11-24T10:00:12.000Z"), "id": 669213944133758976, "text": "@997now u guys are so sweet ����������", "in_reply_to_status": 669209821581742081, "in_reply_to_user": 14992840, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14992840 }}, "user": { "id": 2661692972, "name": "shan // 8 days", "screen_name": "jaureguihotline", "lang": "en", "location": "San Francisco & The Badlands", "create_at": date("2014-07-19"), "description": "i can't think of a good bio at the moment", "followers_count": 1158, "friends_count": 181, "statues_count": 38562 }, "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-24T10:00:12.000Z"), "id": 669213944301535232, "text": "\"Excuse me sir. That's a water cup, you're getting Sprite.\" \n\n\"Oh wow, you can read.\" *walks off*\n\n-Cousin ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 466782271, "name": "Stevenson Prescott", "screen_name": "prescott_steven", "lang": "en", "location": "Jackson, TN", "create_at": date("2012-01-17"), "description": "Potential Google employee by day, Starbucks addict by night -@ashkanology #civictech, web and software developer. iOS, Android. Taking on new projects.", "followers_count": 550, "friends_count": 356, "statues_count": 3944 }, "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("2015-11-24T10:00:12.000Z"), "id": 669213944523956224, "text": "''Olha p frente Lóren'' TO MORRENDO JSJD", "in_reply_to_status": 669213508928675841, "in_reply_to_user": 2490262019, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 2490262019, "name": "Viviane", "screen_name": "standupcim5h", "lang": "pt", "location": "America Latina /Brazil", "create_at": date("2014-04-17"), "description": "My problems are not your problems", "followers_count": 9069, "friends_count": 9867, "statues_count": 57544 }, "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("2015-11-24T10:00:12.000Z"), "id": 669213944716746752, "text": "@__hellowillow yes mam!", "in_reply_to_status": 669182006077485057, "in_reply_to_user": 1654646012, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1654646012 }}, "user": { "id": 213428924, "name": "Anthony Meehan", "screen_name": "banjeebambi", "lang": "en", "location": "Texas, USA", "create_at": date("2010-11-08"), "description": "Texas Tech University | Apparel Design and Manufacturing | 20 ans | River Viiperi is fashion inspo", "followers_count": 1559, "friends_count": 872, "statues_count": 20856 }, "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("2015-11-24T10:00:12.000Z"), "id": 669213944817455104, "text": "���� 12/19 https://t.co/Hf14XFiGRe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 266415567, "name": "Gat Bol", "screen_name": "GatBol_", "lang": "en", "location": "U of Portland", "create_at": date("2011-03-14"), "description": "Rest in Peace PJ", "followers_count": 689, "friends_count": 482, "statues_count": 7251 }, "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("2015-11-24T10:00:12.000Z"), "id": 669213945023086596, "text": "A look North. [Cam 2] on Tuesday, November 24, 2015 @ 1:00:05 PM #CarolinaWx #ClaytonNC https://t.co/5V07JNVhN0", "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": 263, "friends_count": 34, "statues_count": 698374 }, "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.32187,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37101, "countyName": "Johnston" } }
+{ "create_at": datetime("2015-11-24T10:00:12.000Z"), "id": 669213945211822080, "text": "@melaninporn @schrutegirl @oldxsoull @courtzeus @_mcdaniels no just 9", "in_reply_to_status": 669213694656536576, "in_reply_to_user": 3226743216, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3226743216, 987716526, 489211649, 3053087689, 66749695 }}, "user": { "id": 366973798, "name": "C A N E L O", "screen_name": "DarianOIivares", "lang": "en", "location": "null", "create_at": date("2011-09-02"), "description": "The future is born, put the past in a casket. #CowboysNation", "followers_count": 488, "friends_count": 345, "statues_count": 12191 }, "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("2015-11-24T10:00:12.000Z"), "id": 669213945261981696, "text": "Cranberry sauce ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 47151236, "name": "❕Shemmy❕", "screen_name": "LittleBigShemmy", "lang": "en", "location": "West Tyler ✈️San Antonio", "create_at": date("2009-06-14"), "description": "Overdo$e of $auce", "followers_count": 3161, "friends_count": 1482, "statues_count": 145550 }, "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-24T10:00:12.000Z"), "id": 669213945282953216, "text": "Ladies day out! (at @Houlihans in Algonquin, IL) https://t.co/azcbMHZ51e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.33722889,42.15076751"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 31194444 }}, "user": { "id": 110829180, "name": "Victoria Prestia", "screen_name": "vickyp55", "lang": "en", "location": "Illinois", "create_at": date("2010-02-02"), "description": "I am a motivational speaker and topics include Working with difficult bosses and coworkers, and more.\r\n\r\nPlease e-mail me for more information.", "followers_count": 1094, "friends_count": 2059, "statues_count": 10274 }, "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("2015-11-24T10:00:12.000Z"), "id": 669213945325084672, "text": "Should have skipped all of my classes today", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 345175396, "name": "ScHoolboy Stauf", "screen_name": "nick_stauf", "lang": "en", "location": "Pallet Town", "create_at": date("2011-07-29"), "description": "KK☕️ . . . . HHS '15 ➡️ OU '19.", "followers_count": 576, "friends_count": 369, "statues_count": 5749 }, "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("2015-11-24T10:00:12.000Z"), "id": 669213945643868160, "text": "Thank god for this shortened week of school and the half day tomorrow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 413439797, "name": "Felice", "screen_name": "MikeCeciarelli", "lang": "en", "location": "Miami, FL", "create_at": date("2011-11-15"), "description": "Every day should feel this good // Money is the motive // Manalapan", "followers_count": 654, "friends_count": 483, "statues_count": 7474 }, "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("2015-11-24T10:00:12.000Z"), "id": 669213945836601344, "text": "Max went to All Saints but said he's an atheist", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2854951911, "name": "Taylor Smith", "screen_name": "Taylor_rain21", "lang": "en", "location": "null", "create_at": date("2014-11-01"), "description": "NY", "followers_count": 257, "friends_count": 221, "statues_count": 2457 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corning, NY", "id": "dea740edc22e507b", "name": "Corning", "place_type": "city", "bounding_box": rectangle("-77.077935,42.130906 -77.025832,42.177372") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36101, "countyName": "Steuben", "cityID": 3618256, "cityName": "Corning" } }
+{ "create_at": datetime("2015-11-24T10:00:12.000Z"), "id": 669213945891172353, "text": "I'm watching @ModernBaseball interviews and I just want to squish them all. They're so adorable!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 436187029 }}, "user": { "id": 4184475790, "name": "carpe.natem", "screen_name": "Briremmen", "lang": "en", "location": "miami, FL", "create_at": date("2015-11-13"), "description": "19", "followers_count": 55, "friends_count": 158, "statues_count": 90 }, "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("2015-11-24T10:00:12.000Z"), "id": 669213945912274944, "text": "@travelskills @airfarewatchdog Cebu #itsmorefuninthePhilippines", "in_reply_to_status": 669212158144573440, "in_reply_to_user": 1633008624, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "itsmorefuninthePhilippines" }}, "user_mentions": {{ 1633008624, 10937622 }}, "user": { "id": 152885460, "name": "jun Q", "screen_name": "JJQui", "lang": "en", "location": "NY", "create_at": date("2010-06-06"), "description": "i am what i am", "followers_count": 40, "friends_count": 254, "statues_count": 1382 }, "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("2015-11-24T10:00:12.000Z"), "id": 669213946004578304, "text": "2015 DFHCA 1st Team All-State: Erin Coverdale, Cape Henlopen", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user": { "id": 733491744, "name": "Spenser Tilus", "screen_name": "STilus47ABC", "lang": "en", "location": "Salisbury, Maryland", "create_at": date("2012-08-02"), "description": "They call me shirt pants. I rock J's. KCCO. I cover sports for 47ABC.", "followers_count": 1887, "friends_count": 611, "statues_count": 21772 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salisbury, MD", "id": "00caf39d503a84e9", "name": "Salisbury", "place_type": "city", "bounding_box": rectangle("-75.71412,38.28923 -75.487032,38.431613") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24045, "countyName": "Wicomico", "cityID": 2469925, "cityName": "Salisbury" } }
+{ "create_at": datetime("2015-11-24T10:00:12.000Z"), "id": 669213946029600768, "text": "Obama has a hard on for Assad, and is gonna drag the whole world down if he stays with this shit. He is blowing a HUGE opportunity w Russia.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2866359840, "name": "Your Cousin", "screen_name": "Cousin818", "lang": "en", "location": "818", "create_at": date("2014-10-19"), "description": "null", "followers_count": 3924, "friends_count": 1332, "statues_count": 7228 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-24T10:00:12.000Z"), "id": 669213946218434560, "text": "“Hadoop is a distributed system for counting words” https://t.co/i9Et2XKEbI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30950495, "name": "Kevin Chen", "screen_name": "kevinchen", "lang": "en", "location": "New York, NY", "create_at": date("2009-04-13"), "description": "Software developer and accidental machinist. Curious about compilers, big infra, and security. Previously at @Yelp.", "followers_count": 497, "friends_count": 292, "statues_count": 30860 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morningside Heights, Manhattan", "id": "6497298c68c6fe28", "name": "Morningside Heights", "place_type": "neighborhood", "bounding_box": rectangle("-73.976289,40.801889 -73.948014,40.823257") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2015-11-24T10:00:12.000Z"), "id": 669213946658856960, "text": "https://t.co/bOwTj6FfAO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 28753438, "name": "jimm parsons", "screen_name": "southsidejimm", "lang": "en", "location": "North Olmsted", "create_at": date("2009-04-03"), "description": "ProSupps Demo Rep save 10% with free shipping use code Jimm10 at http://www.prosupps.com", "followers_count": 1072, "friends_count": 1652, "statues_count": 978 }, "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("2015-11-24T10:00:12.000Z"), "id": 669213946901999616, "text": "From far behind... https://t.co/Ptw7Mt31RI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 76803949, "name": "Ted H. S. Hong", "screen_name": "THSHong", "lang": "en", "location": "Hilo, HI", "create_at": date("2009-09-23"), "description": "Running Dog Capitalist; Counter-revolutionary; wanna be Member of the Trilateral Commission (who knew it was real?); be firm, fly low and stay cool", "followers_count": 2261, "friends_count": 2352, "statues_count": 31879 }, "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("2015-11-24T10:00:13.000Z"), "id": 669213947061542912, "text": "The only reason that I have. I remorse for talking about her like this is because she's a disruptive student and she's mean ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1606489849, "name": "Slim", "screen_name": "aaronaharris", "lang": "en", "location": "Bakin cookies wit yo Gram-Gram", "create_at": date("2013-07-19"), "description": "Now Faith Is The Subtance Of Things Hoped For, The Evidence Of Things Not Seen. - Hebrews 11:1 #MSU18 ✳️ #Actor", "followers_count": 256, "friends_count": 243, "statues_count": 6799 }, "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("2015-11-24T10:00:13.000Z"), "id": 669213947073949696, "text": "Partly cloudy/wind this afternoon, high 61 (16 C). Low 54 (12 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712679, "name": "OKCWX", "screen_name": "OKCWX", "lang": "en", "location": "Oklahoma City, Oklahoma", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Oklahoma City, Oklahoma", "followers_count": 175, "friends_count": 93, "statues_count": 8080 }, "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-24T10:00:13.000Z"), "id": 669213947497611265, "text": "After Thanksgiving (: https://t.co/vRKoBmt6Ex", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3044006181, "name": "Jenelle Monge", "screen_name": "Jmonge2000", "lang": "en", "location": "null", "create_at": date("2015-02-17"), "description": "Always and forever 5.28.15 Eleazar S. Figueroa", "followers_count": 224, "friends_count": 252, "statues_count": 5169 }, "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("2015-11-24T10:00:13.000Z"), "id": 669213947505971200, "text": "Join the UPS team! See our latest #Transportation #job opening here: https://t.co/2AzKKbfcgY #LOUISVILLE, KY #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.7102437,38.1804513"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Transportation", "job", "LOUISVILLE", "Hiring", "CareerArc" }}, "user": { "id": 23502894, "name": "TMJ-SDF Transp. Jobs", "screen_name": "tmj_sdf_transp", "lang": "en", "location": "Louisville, KY", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Transportation job tweets in Louisville, KY from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 318, "friends_count": 291, "statues_count": 104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynnview, KY", "id": "5495f2884d783a43", "name": "Lynnview", "place_type": "city", "bounding_box": rectangle("-85.716887,38.175022 -85.705676,38.183837") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2148648, "cityName": "Lynnview" } }
+{ "create_at": datetime("2015-11-24T10:00:13.000Z"), "id": 669213947539488768, "text": "Get 20% off your @christophestogo food with the code TONEZONE..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 211530882 }}, "user": { "id": 623087205, "name": "Tony kyler", "screen_name": "Tonezonefitness", "lang": "en", "location": "atlanta metro area", "create_at": date("2012-06-30"), "description": "A difference maker in life.. (http://twiends.com/tonezonefitness)", "followers_count": 603, "friends_count": 1992, "statues_count": 2718 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Duluth, GA", "id": "ed71ac127b7ac5a2", "name": "Duluth", "place_type": "city", "bounding_box": rectangle("-84.184657,33.965953 -84.083328,34.050723") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13135, "countyName": "Gwinnett", "cityID": 1324600, "cityName": "Duluth" } }
+{ "create_at": datetime("2015-11-24T10:00:13.000Z"), "id": 669213947887665152, "text": "Accident, shoulder blocked in #Bakersfield on Hwy 43 SB at Hwy 58 #traffic https://t.co/j8BuvIQBBw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.2521,35.39809"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Bakersfield", "traffic" }}, "user": { "id": 930753054, "name": "TTN Bakersfield", "screen_name": "TotalTrafficBFL", "lang": "en", "location": "Bakersfield, CA", "create_at": date("2012-11-06"), "description": "Traffic Information for Bakersfield, CA.", "followers_count": 76, "friends_count": 5, "statues_count": 1949 }, "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": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2015-11-24T10:00:13.000Z"), "id": 669213948068110340, "text": "Only at Amazon you see a whole bird fly in the bathroom and take a shit in the sink��mmmmmk", "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": "snapchat- x-kayya", "screen_name": "Callmegunz__", "lang": "en", "location": "null", "create_at": date("2012-12-12"), "description": "If I'm single, so is your girl ...", "followers_count": 5547, "friends_count": 4550, "statues_count": 115819 }, "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("2015-11-24T10:00:13.000Z"), "id": 669213948080558080, "text": "I'm playing tonight. The good lord gone blessed me with the strength", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 235809796, "name": "Slim Jesus", "screen_name": "EasyMoneyMal", "lang": "en", "location": "on my way...", "create_at": date("2011-01-08"), "description": "Huston-Tillotson MBB #5... I'm on the pedal all year", "followers_count": 1925, "friends_count": 832, "statues_count": 108104 }, "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("2015-11-24T10:00:13.000Z"), "id": 669213948080742404, "text": "considering the fact we've taken tests on like half the chapters in this book... I might need to go ahead and start reading it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 503500171, "name": "Makayla.", "screen_name": "makaylanorrell", "lang": "en", "location": "probz in PFHS student section", "create_at": date("2012-02-25"), "description": "happy holla days. #PFMFT", "followers_count": 922, "friends_count": 354, "statues_count": 19370 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pigeon Forge, TN", "id": "5b4321b324f07e60", "name": "Pigeon Forge", "place_type": "city", "bounding_box": rectangle("-83.624306,35.768969 -83.530892,35.82953") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47155, "countyName": "Sevier", "cityID": 4758080, "cityName": "Pigeon Forge" } }
+{ "create_at": datetime("2015-11-24T10:00:13.000Z"), "id": 669213948353232896, "text": "Happy birthday baby girl enjoy your day. �� @AlissaHatesYou", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 862129320 }}, "user": { "id": 2700529234, "name": "adreanaamariee", "screen_name": "adreanaam_", "lang": "en", "location": "null", "create_at": date("2014-07-11"), "description": "I'm a brat", "followers_count": 649, "friends_count": 454, "statues_count": 16643 }, "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("2015-11-24T10:00:13.000Z"), "id": 669213948462415872, "text": "I just want to open a cereal, toast and coffee restaurant.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 74661351, "name": "⚜Brandywine⚜", "screen_name": "Peanut14389", "lang": "en", "location": "outside the box ", "create_at": date("2009-09-15"), "description": "sleep deprived college student who is fueled by coffee while desperately trying to get my life together. #drinkallofthecoffee #takeallofthechances #dontquit", "followers_count": 110, "friends_count": 432, "statues_count": 1141 }, "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("2015-11-24T10:00:13.000Z"), "id": 669213948537909248, "text": "How perfect is pizza though? ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325873631, "name": "Austin.", "screen_name": "Austin__Odom", "lang": "en", "location": "Baxley, GA", "create_at": date("2011-06-28"), "description": "Absolutely Awkward - Future World Traveler - coffee is life", "followers_count": 299, "friends_count": 410, "statues_count": 2426 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baxley, GA", "id": "4fb2adc301fa8568", "name": "Baxley", "place_type": "city", "bounding_box": rectangle("-82.372235,31.733889 -82.312054,31.811706") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13001, "countyName": "Appling", "cityID": 1306016, "cityName": "Baxley" } }
+{ "create_at": datetime("2015-11-24T10:00:13.000Z"), "id": 669213948680536064, "text": "TX State is pretty, but I thought my friends were prettier ❤️ @ Texas State University https://t.co/RbGGiXLXbh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.94051,29.88887941"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1442678882, "name": "m.killa ©", "screen_name": "MALLORYTATMAN", "lang": "en", "location": "TX ", "create_at": date("2013-05-19"), "description": "Jesus, Tswift, cheer, Grey's Anatomy & @haydensanders15 ❤️", "followers_count": 963, "friends_count": 734, "statues_count": 27974 }, "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("2015-11-24T10:00:13.000Z"), "id": 669213948781154305, "text": "I'm in class and on Twitter instead of doing work are you proud? @RenaLovelis #thisisrena", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thisisrena" }}, "user_mentions": {{ 305916419 }}, "user": { "id": 976658401, "name": "❌M E L I S S A❌", "screen_name": "MeliG5SOS", "lang": "en", "location": "Arkansas, USA", "create_at": date("2012-11-28"), "description": "I like a band and they do weird shit sometimes @5SOS❤️Desperate for Michael Clifford❤️Hey Violet+Casey follow✌️SC: meliigonzalez IG: _melissa_clifford_", "followers_count": 6166, "friends_count": 4368, "statues_count": 96025 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rogers, AR", "id": "a9e535bc5a50aba9", "name": "Rogers", "place_type": "city", "bounding_box": rectangle("-94.225208,36.265124 -94.070837,36.406101") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton", "cityID": 560410, "cityName": "Rogers" } }
+{ "create_at": datetime("2015-11-24T10:00:13.000Z"), "id": 669213948806168576, "text": "@marilynmanson Thank u! U've done it again TPE is EVERYTHING! Uniquely sexy, imaginative, refined, brilliant! & ur vocals r fkng stunning!", "in_reply_to_status": -1, "in_reply_to_user": 28206360, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28206360 }}, "user": { "id": 426247204, "name": "Devereau Chumrau", "screen_name": "devereauchumrau", "lang": "en", "location": "Los Angeles", "create_at": date("2011-12-01"), "description": "Actor. Artist. Happy Person. :)", "followers_count": 397, "friends_count": 335, "statues_count": 2640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-24T10:00:13.000Z"), "id": 669213948919451649, "text": "@EdenIrgens I would have that in an instant...\n@RustyRiderGirl @geetawalia", "in_reply_to_status": 669205522638245889, "in_reply_to_user": 27114449, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27114449, 15459073, 547035828 }}, "user": { "id": 321553655, "name": "Wynn Lare", "screen_name": "RippleCreekNW", "lang": "en", "location": "Coeur D'Alene Northwest USA ", "create_at": date("2011-06-21"), "description": "...[kind] I like NICE smart & redunculous; I am 50% nerd...PG•PG13...maker & designer copywriter WAS a builder a baker & a tech maker ... LIVE & LET LIVE", "followers_count": 1159, "friends_count": 1412, "statues_count": 33098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coeur d'Alene, ID", "id": "da6f5e66f73b8f7f", "name": "Coeur d'Alene", "place_type": "city", "bounding_box": rectangle("-116.855401,47.65878 -116.732088,47.744794") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16055, "countyName": "Kootenai", "cityID": 1616750, "cityName": "Coeur d'Alene" } }
+{ "create_at": datetime("2015-11-24T10:00:13.000Z"), "id": 669213949062217728, "text": "Babygirl you're getting played, like the girls you always talk about ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.23503409,32.83267358"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1072785920, "name": "Woah_There", "screen_name": "jazzyfresh_15", "lang": "en", "location": "North Richland Hills, Texas ", "create_at": date("2013-01-08"), "description": "Ig: Jazzyfresh16", "followers_count": 462, "friends_count": 291, "statues_count": 8546 }, "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("2015-11-24T10:00:13.000Z"), "id": 669213949137588225, "text": "Serious question. How are u late to your hair appt, when u live with your stylist �� seriously all u have 2 do is walk in the living room ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 322936624, "name": "Slim.", "screen_name": "trapgvddess", "lang": "en", "location": "dtx✈️ca", "create_at": date("2011-06-23"), "description": "late bloomer. I ain't trippin✨| #RIPLucky ⚜|#LongLiveMonty", "followers_count": 693, "friends_count": 746, "statues_count": 15562 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-24T10:00:13.000Z"), "id": 669213949452279808, "text": "Choppa knock the weight out a nigga like slim fast", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2336759444, "name": "SauceMan CDub™", "screen_name": "BallISCassius", "lang": "en", "location": " DAL TX ✈️ OH", "create_at": date("2014-02-10"), "description": "Money to be made i got my cleats on #CSU19. NFLBound. #CentralState #LegendaryCDub CSU football commit Cassius Williams", "followers_count": 535, "friends_count": 368, "statues_count": 28307 }, "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("2015-11-24T10:00:13.000Z"), "id": 669213949687042052, "text": "I need to get to Houston NOW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 236226579, "name": "God's Grace ", "screen_name": "GiannaDeVan", "lang": "en", "location": "null", "create_at": date("2011-01-09"), "description": "utsa17'", "followers_count": 509, "friends_count": 490, "statues_count": 14589 }, "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-24T10:00:13.000Z"), "id": 669213950010068993, "text": "@daniellel33x0 now I'm going to be going to a empty suite since everyone left ����", "in_reply_to_status": 669193138087534593, "in_reply_to_user": 259496192, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 259496192 }}, "user": { "id": 371429252, "name": "Amy", "screen_name": "ammy_volker", "lang": "en", "location": "null", "create_at": date("2011-09-10"), "description": "18 | NJ | FDU Devils Dance Team'19", "followers_count": 526, "friends_count": 457, "statues_count": 27155 }, "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("2015-11-24T10:00:13.000Z"), "id": 669213950345478144, "text": "I Still Laugh At That Mf Video @KayeB_Beautiful �������������� Drunk Asses .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 896325584 }}, "user": { "id": 233090529, "name": "R.I.P GRANNY", "screen_name": "LilBrian_iTsAgo", "lang": "en", "location": "jȏʟıєṭ | ċһıċѧɢȏ", "create_at": date("2011-01-01"), "description": "arтιѕт, ѕongwrιтer | LΣΩ ♌️ | 8⃣1⃣7⃣ | ѕnapcнaт | ιnѕтagraм: @LILBRIAN_ITSAGO #lвe #nғl #ιтѕago #dyнмn |corιnтнιanѕ 5:7 aѕv", "followers_count": 1134, "friends_count": 980, "statues_count": 16966 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Preston Heights, IL", "id": "8df4294fff20f553", "name": "Preston Heights", "place_type": "city", "bounding_box": rectangle("-88.088236,41.480031 -88.060955,41.502664") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1761860, "cityName": "Preston Heights" } }
+{ "create_at": datetime("2015-11-24T10:00:13.000Z"), "id": 669213950429364224, "text": "Cloudy this afternoon, high 69 (21 C). Low 61 (16 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712488, "name": "SanAntonioWX", "screen_name": "SanAntonioWX", "lang": "en", "location": "San Antonio, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for San Antonio, Texas", "followers_count": 245, "friends_count": 93, "statues_count": 8070 }, "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-24T10:00:13.000Z"), "id": 669213950454599680, "text": "@Parisha_xo #MTVStars 5 seconds of summer", "in_reply_to_status": -1, "in_reply_to_user": 1066185402, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MTVStars" }}, "user_mentions": {{ 1066185402 }}, "user": { "id": 2936041511, "name": "CAITLIN", "screen_name": "_penguin_hemmo", "lang": "en", "location": "New Smyrna Beach, FL", "create_at": date("2014-12-21"), "description": "|| Derping since 2012//just that one girl obsessed with 4 dorks in a band// SLFL JULY 20th||", "followers_count": 275, "friends_count": 166, "statues_count": 4541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Smyrna Beach, FL", "id": "df82825ad778ab15", "name": "New Smyrna Beach", "place_type": "city", "bounding_box": rectangle("-80.986841,28.987795 -80.867436,29.083799") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1248625, "cityName": "New Smyrna Beach" } }
+{ "create_at": datetime("2015-11-24T10:00:13.000Z"), "id": 669213950496473088, "text": "home bound����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1283582492, "name": "Rachael Ziegler", "screen_name": "rachael_ziegler", "lang": "en", "location": "University of Arizona", "create_at": date("2013-03-20"), "description": "dogs are people too", "followers_count": 165, "friends_count": 161, "statues_count": 2326 }, "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-24T10:00:13.000Z"), "id": 669213950551027712, "text": "Oh Whale ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3897429432, "name": "Alice", "screen_name": "AlisonNunez27", "lang": "en", "location": "Pasadena", "create_at": date("2015-10-14"), "description": "Nothing Last Forever.", "followers_count": 129, "friends_count": 117, "statues_count": 1968 }, "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("2015-11-24T10:00:13.000Z"), "id": 669213950572105728, "text": "Could be going to the Michigan State - Penn State game this weekend, instead I'll be going to Grand Valleys game. #dedication", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "dedication" }}, "user": { "id": 309880837, "name": "Hayden⚓️", "screen_name": "SAnn_Hayden", "lang": "en", "location": "Allendale, Michigan", "create_at": date("2011-06-02"), "description": "GV | nittany lions | pittsburgh | sports managment |", "followers_count": 388, "friends_count": 469, "statues_count": 11926 }, "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("2015-11-24T10:00:13.000Z"), "id": 669213950664306688, "text": "@dswebsme Here's a look at my reports. :) https://t.co/6KgNyiVoLx", "in_reply_to_status": 669213184713220096, "in_reply_to_user": 2814081001, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2814081001 }}, "user": { "id": 15815424, "name": "Kristi Hines", "screen_name": "kikolani", "lang": "en", "location": "Peoria, AZ", "create_at": date("2008-08-11"), "description": "Freelance writer and business blogger. HubSpot Inbound Marketing Certified and Google Analytics Qualified. Find out how I can help you at http://kristihines.com", "followers_count": 65278, "friends_count": 6670, "statues_count": 26935 }, "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-24T10:00:13.000Z"), "id": 669213950999941120, "text": "@NinoGuz @1Drought2017 me either ��", "in_reply_to_status": 669210215380815872, "in_reply_to_user": 614144271, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 614144271, 544714368 }}, "user": { "id": 15094638, "name": "brianna", "screen_name": "omgxbrianna", "lang": "en", "location": "chicago, il ", "create_at": date("2008-06-12"), "description": "22 // current nursing student // future RN ☺️", "followers_count": 350, "friends_count": 115, "statues_count": 24470 }, "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("2015-11-24T10:00:13.000Z"), "id": 669213951045992448, "text": "#Tb ���� https://t.co/20T1sehFIT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Tb" }}, "user": { "id": 2438882345, "name": "DBANDz$ ♋️", "screen_name": "DBANDz_7", "lang": "en", "location": "Woods FL, 305 Born 954 Living", "create_at": date("2014-03-26"), "description": "19 | Bettering Myself | Just Want To Accomplish My Goals | #Wrestler | Never Lose Hope, Always Have Faith | #InGodITrust | MoneyMotivated | A.T.M MAFIA | #2k16", "followers_count": 531, "friends_count": 251, "statues_count": 8213 }, "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("2015-11-24T10:00:13.000Z"), "id": 669213951092092928, "text": "@KerkmanMatt pretty early to be on tinder Matt?", "in_reply_to_status": 669101424119189504, "in_reply_to_user": 415010119, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 415010119 }}, "user": { "id": 348012007, "name": "Aaron Janak", "screen_name": "ayyyjanyyy", "lang": "en", "location": "null", "create_at": date("2011-08-03"), "description": "null", "followers_count": 191, "friends_count": 184, "statues_count": 2728 }, "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("2015-11-24T10:00:14.000Z"), "id": 669213951566069761, "text": "Can you recommend anyone for this #Retail #job? https://t.co/uXnqMSrynl #Dallas #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.8002704,32.9418531"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "Dallas", "Hiring" }}, "user": { "id": 2798254742, "name": "Chico's Careers", "screen_name": "ChicosCareers", "lang": "en", "location": "null", "create_at": date("2014-09-08"), "description": "Chico’s: Fabulous looks. Conversation-starting jewelry. The most amazing service. For 30 years, we’ve helped millions of women look as great as they feel.", "followers_count": 145, "friends_count": 0, "statues_count": 14415 }, "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-24T10:00:14.000Z"), "id": 669213951599579137, "text": "This #Retail #job might be a great fit for you: Keyholder, J.Crew Retail, Highland Village - https://t.co/SL2s7roENV #Houston, TX #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.4515725,29.740582"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "Houston", "Hiring" }}, "user": { "id": 28482546, "name": "TMJ-HOU Retail Jobs", "screen_name": "tmj_hou_retail", "lang": "en", "location": "Houston, TX", "create_at": date("2009-04-02"), "description": "Follow this account for geo-targeted Retail job tweets in Houston, TX. Need help? Tweet us at @CareerArc!", "followers_count": 433, "friends_count": 318, "statues_count": 1560 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:14.000Z"), "id": 669213951708798977, "text": "#TransformationTuesday: Check out this gorgeous caramel tone to the right! ➡️\n#caramelhairdontcare… https://t.co/8ztuYzcD2N", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.6918016,40.7932981"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TransformationTuesday", "caramelhairdontcare" }}, "user": { "id": 31221564, "name": "nuBest salon and spa", "screen_name": "nuBestsalon", "lang": "en", "location": "manhasset, ny", "create_at": date("2009-04-14"), "description": "nuBest is proud to be called upon by top magazine editors and many others as a resource for beauty trends, tips and techniques.", "followers_count": 3202, "friends_count": 541, "statues_count": 3267 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhasset, NY", "id": "a6953c8c4e525219", "name": "Manhasset", "place_type": "city", "bounding_box": rectangle("-73.715895,40.771234 -73.667433,40.803543") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3644897, "cityName": "Manhasset" } }
+{ "create_at": datetime("2015-11-24T10:00:14.000Z"), "id": 669213951847100417, "text": "@TrentaBitch https://t.co/9DhCJ9X7fA", "in_reply_to_status": 669213792342003712, "in_reply_to_user": 487256511, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 487256511 }}, "user": { "id": 61546390, "name": "️", "screen_name": "httpscottie", "lang": "en", "location": "Ypsilanti, MI", "create_at": date("2009-07-30"), "description": "Je vis là où le ciel se termine.", "followers_count": 7007, "friends_count": 388, "statues_count": 230692 }, "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.32554") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26161, "countyName": "Washtenaw", "cityID": 2689140, "cityName": "Ypsilanti" } }
+{ "create_at": datetime("2015-11-24T10:00:14.000Z"), "id": 669213952073592832, "text": "You are my fucking everything. @TheMattEspinosa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1906234147 }}, "user": { "id": 2289308869, "name": "nobody", "screen_name": "lovelyy_aileen", "lang": "en", "location": "Matthew Lee Espinosa", "create_at": date("2014-01-13"), "description": "I love you so much babe bye -Matthew. 10-23-15 ❤️❤️", "followers_count": 378, "friends_count": 347, "statues_count": 13818 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-24T10:00:14.000Z"), "id": 669213952073596928, "text": "@prettycambino exactly what I said.", "in_reply_to_status": 669213732652871680, "in_reply_to_user": 717997576, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 717997576 }}, "user": { "id": 2942520031, "name": "Slim Baby ♋", "screen_name": "ayeeyoslim", "lang": "en", "location": "Petty Blvd Bitch", "create_at": date("2014-12-24"), "description": "Your Favorite Slim Chick on Twitter. @OG_St0ner #02272011 Who's going to be the next dumb nigga or bitch in my mentions? I'll wait ..", "followers_count": 14419, "friends_count": 4220, "statues_count": 47785 }, "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-24T10:00:14.000Z"), "id": 669213952258117632, "text": "See our latest #Honolulu, HI #job and click to apply: Manager - Womens Center - https://t.co/a5t6R6y3qi #Healthcare #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8241911,21.2877723"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Honolulu", "job", "Healthcare", "Hiring", "CareerArc" }}, "user": { "id": 23526678, "name": "TMJ- HON Health Jobs", "screen_name": "tmj_hon_health", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Healthcare job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 402, "friends_count": 304, "statues_count": 181 }, "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("2015-11-24T10:00:14.000Z"), "id": 669213952434438146, "text": "@BaseheadHarley I'm DEAD!", "in_reply_to_status": 669213714239913989, "in_reply_to_user": 200985141, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 200985141 }}, "user": { "id": 595109759, "name": "jas.", "screen_name": "dazzleMEjazz_", "lang": "en", "location": "2 up 2 down.", "create_at": date("2012-05-30"), "description": "IG:dazzlingjas / senior '16", "followers_count": 1025, "friends_count": 712, "statues_count": 24430 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Boston, VA", "id": "33f91bef2fc69db5", "name": "South Boston", "place_type": "city", "bounding_box": rectangle("-78.951141,36.693374 -78.872754,36.746024") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51083, "countyName": "Halifax", "cityID": 5173712, "cityName": "South Boston" } }
+{ "create_at": datetime("2015-11-24T10:00:14.000Z"), "id": 669213952442691584, "text": "@devcapps fuck I keep thinking this is Alaunah", "in_reply_to_status": 669197237289811968, "in_reply_to_user": 3629981413, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3629981413 }}, "user": { "id": 824153820, "name": "Sandra Michelle", "screen_name": "sanndramichelle", "lang": "en", "location": "||NAC|| SFASU ", "create_at": date("2012-09-14"), "description": "••Just a cupcake in a muffin world••", "followers_count": 896, "friends_count": 723, "statues_count": 9717 }, "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("2015-11-24T10:00:14.000Z"), "id": 669213952526561280, "text": "Y'all don't be cute, just naked.\nY'all don't be cute, just naked.\nY'all don't be cute, just naked.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 156131886, "name": "God's Child ⚓️", "screen_name": "meishaNOTmyesha", "lang": "en", "location": "null", "create_at": date("2010-06-15"), "description": "Romans 8:18 | I love God & Trap Music.", "followers_count": 2348, "friends_count": 1516, "statues_count": 87080 }, "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("2015-11-24T10:00:14.000Z"), "id": 669213952547627008, "text": "@TheRapologist @thetrevmaster both of your men in one vid��", "in_reply_to_status": 668831345901182977, "in_reply_to_user": 2985300865, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2985300865, 601517805 }}, "user": { "id": 727396927, "name": "B", "screen_name": "BethOverfelt", "lang": "en", "location": "Mount Airy, NC", "create_at": date("2012-07-30"), "description": "hey shawty, what it do, what's happenin?", "followers_count": 599, "friends_count": 507, "statues_count": 6962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Airy, NC", "id": "46b149b6aa3775a2", "name": "Mount Airy", "place_type": "city", "bounding_box": rectangle("-80.671877,36.464892 -80.579574,36.560353") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37171, "countyName": "Surry", "cityID": 3744800, "cityName": "Mount Airy" } }
+{ "create_at": datetime("2015-11-24T10:00:14.000Z"), "id": 669213952577052672, "text": "Chilling in New Orleans!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 532790782, "name": "Ellison Holley", "screen_name": "RealEHolley59", "lang": "en", "location": "Denver ", "create_at": date("2012-03-21"), "description": "Memphis To Denver SnapChat: ellison.holley", "followers_count": 290, "friends_count": 638, "statues_count": 3506 }, "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("2015-11-24T10:00:14.000Z"), "id": 669213952652496897, "text": "@Colombiana_Gal @Zona_Buckeye @justinbieber lol", "in_reply_to_status": 669213778525884417, "in_reply_to_user": 163717434, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 163717434, 1257329953, 27260086 }}, "user": { "id": 794824172, "name": "JeremyinAkron™", "screen_name": "JeremyinAkron", "lang": "en", "location": "Copley", "create_at": date("2012-08-31"), "description": "null", "followers_count": 2678, "friends_count": 2315, "statues_count": 62503 }, "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("2015-11-24T10:00:14.000Z"), "id": 669213952832839683, "text": "\"Permit place\" https://t.co/sGDeJL71r0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2579238731, "name": "Jonathan", "screen_name": "Jonathon1210", "lang": "en", "location": "null", "create_at": date("2014-06-02"), "description": "null", "followers_count": 299, "friends_count": 47, "statues_count": 9644 }, "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("2015-11-24T10:00:14.000Z"), "id": 669213953038225409, "text": "literally just got patted down like a criminal, yelled at for smuggling more than 3 ounce of tooth paste through security ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1027239426, "name": "Rachel Anne", "screen_name": "RaccCitty", "lang": "en", "location": "BAYLOR", "create_at": date("2012-12-21"), "description": "Baylor 19 |snapchat: rachellecreedon| New Jersey will always be home. bring me starbucks and we can date.", "followers_count": 382, "friends_count": 665, "statues_count": 4765 }, "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("2015-11-24T10:00:14.000Z"), "id": 669213953508020224, "text": "@yo_coiley @Ammer916 @collegefession @clarkyolsen", "in_reply_to_status": 667991120178089984, "in_reply_to_user": 209157111, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 209157111, 1957822291, 536649400, 512823783 }}, "user": { "id": 2960233598, "name": "Trice G", "screen_name": "tricegortonn", "lang": "en", "location": "null", "create_at": date("2015-01-05"), "description": "null", "followers_count": 228, "friends_count": 251, "statues_count": 418 }, "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("2015-11-24T10:00:14.000Z"), "id": 669213953575116800, "text": "Cloudy this afternoon, high 64 (18 C). Low 57 (14 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712452, "name": "FortWorthWX", "screen_name": "FortWorthWX", "lang": "en", "location": "Fort Worth, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Fort Worth, Texas", "followers_count": 208, "friends_count": 93, "statues_count": 8057 }, "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("2015-11-24T10:00:14.000Z"), "id": 669213953621409794, "text": "@MsDakkota \nDakkota Vs Sarah Brooke, Great Job Sunday\nhttps://t.co/CZtvUgHDAs https://t.co/JkpSRMzu4x", "in_reply_to_status": -1, "in_reply_to_user": 1409228166, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1409228166 }}, "user": { "id": 1680921481, "name": "Greg Alan", "screen_name": "GregAlan3", "lang": "en", "location": "Illinois, USA", "create_at": date("2013-08-18"), "description": "Producer", "followers_count": 95, "friends_count": 24, "statues_count": 104 }, "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-24T10:00:14.000Z"), "id": 669213953843707905, "text": "I want some soup && a sandwich ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155732149, "name": "Ayo!", "screen_name": "sayAYEE", "lang": "en", "location": ".....", "create_at": date("2010-06-14"), "description": "@CarpéAye", "followers_count": 822, "friends_count": 584, "statues_count": 10960 }, "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("2015-11-24T10:00:14.000Z"), "id": 669213953927548928, "text": "You love me but you dont know 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": 79564404, "name": "Gabyton'", "screen_name": "GabycardonaPR", "lang": "en", "location": "Florida", "create_at": date("2009-10-03"), "description": "Total, yo vivo chilling |Guayanilla, PR | sc;gaby_cardona", "followers_count": 561, "friends_count": 722, "statues_count": 19669 }, "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("2015-11-24T10:00:14.000Z"), "id": 669213953969422336, "text": "We're #hiring! Read about our latest #job opening here: Seeking Substitute Teachers for Corydon Schools! - https://t.co/YjjlMq4VNW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.1219155,38.2120121"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job" }}, "user": { "id": 21838240, "name": "TMJ-SDF Educ. Jobs", "screen_name": "tmj_sdf_edu", "lang": "en", "location": "Louisville, KY", "create_at": date("2009-02-24"), "description": "Follow this account for geo-targeted Education/Teaching job tweets in Louisville, KY from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 278, "friends_count": 234, "statues_count": 30 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corydon, IN", "id": "8427920cc8176b7c", "name": "Corydon", "place_type": "city", "bounding_box": rectangle("-86.155205,38.196842 -86.105924,38.24823") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18061, "countyName": "Harrison", "cityID": 1815256, "cityName": "Corydon" } }
+{ "create_at": datetime("2015-11-24T10:00:14.000Z"), "id": 669213954065829889, "text": "If you're a professional in #LAHAINA, HI, check out this #job: https://t.co/VwFfmdNqVE #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-156.6817594,20.8866199"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LAHAINA", "job", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 38082277, "name": "TMJ-MAU Jobs", "screen_name": "tmj_mau_jobs", "lang": "en", "location": "Maui, HI", "create_at": date("2009-05-05"), "description": "Follow this account for geo-targeted Other job tweets in Maui, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 546, "friends_count": 535, "statues_count": 15 }, "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("2015-11-24T10:00:14.000Z"), "id": 669213954116329472, "text": "Just passed Judy Genshaft and I feel like I passed a celebrity #starstruck ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "starstruck" }}, "user": { "id": 1228096814, "name": "Alexis Branch", "screen_name": "alexissummr", "lang": "en", "location": "null", "create_at": date("2013-02-28"), "description": "I don't really know.", "followers_count": 157, "friends_count": 202, "statues_count": 396 }, "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("2015-11-24T10:00:14.000Z"), "id": 669213954363826176, "text": "I can be a lady in the streets in a dress with her hair tied up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 72289101, "name": "Jay", "screen_name": "JaayyyB", "lang": "en", "location": "‏‏ما هو الماضي لا يموت", "create_at": date("2009-09-07"), "description": "null", "followers_count": 544, "friends_count": 216, "statues_count": 25940 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westlake, OH", "id": "abcede5a546a90a3", "name": "Westlake", "place_type": "city", "bounding_box": rectangle("-81.970255,41.41104 -81.872904,41.478131") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3983622, "cityName": "Westlake" } }
+{ "create_at": datetime("2015-11-24T10:00:14.000Z"), "id": 669213954552369152, "text": "@BearCov When she locked up kilgrave and thought she would get him to admit to hopes murders, like she doesn't know the law? She's a PI?!?", "in_reply_to_status": 669213407669673984, "in_reply_to_user": 174714925, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 174714925 }}, "user": { "id": 345496263, "name": "Jak Knight", "screen_name": "ItsJak_Knight", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-07-30"), "description": "Comedian. Thank you for everything Kanye.", "followers_count": 1562, "friends_count": 410, "statues_count": 8390 }, "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("2015-11-24T10:00:14.000Z"), "id": 669213954615279616, "text": "Happy Birthday @madsaustin !!! �� Hope you have an awesome day!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3599632694 }}, "user": { "id": 112619360, "name": "Daniel Koeth", "screen_name": "DKoeth", "lang": "en", "location": "Tempe, Arizona", "create_at": date("2010-02-08"), "description": "i am exciting and delicious", "followers_count": 292, "friends_count": 350, "statues_count": 4548 }, "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("2015-11-24T10:00:14.000Z"), "id": 669213954875375616, "text": "https://t.co/I5DDBlzH09", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3105349218, "name": "s h e f a l i", "screen_name": "shefalijey", "lang": "en", "location": "null", "create_at": date("2015-03-23"), "description": "livin'", "followers_count": 97, "friends_count": 99, "statues_count": 613 }, "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-24T10:00:14.000Z"), "id": 669213955089375232, "text": "@caltarquinio vamo mesmo, to na biblioteca pública to apaixonada", "in_reply_to_status": 669191246448082944, "in_reply_to_user": 2301925747, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user_mentions": {{ 2301925747 }}, "user": { "id": 1112294618, "name": "luíza", "screen_name": "namelesslui", "lang": "en", "location": "null", "create_at": date("2013-01-22"), "description": "there's too many young angels on the southside", "followers_count": 584, "friends_count": 299, "statues_count": 62518 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213955408183296, "text": "@Emili6no You're welcome cousin", "in_reply_to_status": 669213804245336064, "in_reply_to_user": 1375990536, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1375990536 }}, "user": { "id": 2781600482, "name": "Benny Romo III", "screen_name": "iii_romo", "lang": "en", "location": "French Camp, CA", "create_at": date("2014-08-30"), "description": "I'm 19 years old I like the Los Angeles Lakers I'm the leader of the Squad", "followers_count": 232, "friends_count": 840, "statues_count": 6610 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213955487760384, "text": "bring it on will forever be one of my fav movies", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 490642780, "name": "ƙɑтɛƖყɴ ɢɑuтʀɛɑu×", "screen_name": "katelyngotro_", "lang": "en", "location": "Walker, LA", "create_at": date("2012-02-12"), "description": "do to others as you would have them do to you. luke 6:31 :)", "followers_count": 163, "friends_count": 212, "statues_count": 3727 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213955500306433, "text": "Yoooo OMG!!! RT to save a life https://t.co/CnmMpKrNCa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 516658543, "name": "Abu JR", "screen_name": "twerkitonjr", "lang": "en", "location": "Colombia", "create_at": date("2012-03-06"), "description": "if you creepin might as well hit my DMs", "followers_count": 466, "friends_count": 385, "statues_count": 33607 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213955697459200, "text": "Ima be so high on thanksgiving", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2505956576, "name": "Troy Wayne", "screen_name": "YoTroytroy", "lang": "en", "location": "Red Oak, TX", "create_at": date("2014-05-18"), "description": "whiteboytroy Dallas #FutureHive Cowboys nation. Hokage", "followers_count": 229, "friends_count": 163, "statues_count": 6377 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213955886198784, "text": "Everytime I hear Obama say ISIL, I feel like he wants to say assholes lol.... You have my permission to curse Mr. President ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325522571, "name": "Mushu", "screen_name": "MrMujuDaBeast94", "lang": "en", "location": "null", "create_at": date("2011-06-28"), "description": "PSN ID: muju_da_beast94", "followers_count": 322, "friends_count": 1635, "statues_count": 6205 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213955902996480, "text": "[Tue Nov 24th, 12:00pm] Temperature: 68.9 F, 20.5 C; Humidity: 41.7%; Light: 16.89%. https://t.co/UI5qWnQ21d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 269430133, "name": "Shoemaker Sandbox", "screen_name": "shoe_sandbox", "lang": "en", "location": "Earth (for now). ", "create_at": date("2011-03-20"), "description": "I am a bot. Developer account for @shoemaker.", "followers_count": 7, "friends_count": 1, "statues_count": 23498 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213956079226882, "text": "Rain, steel, trucks, cappuccinos. @ Special Xtra https://t.co/np5ix6FVlB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.39894759,37.7877601"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 44993, "name": "Rebecca Sinclair", "screen_name": "sinclair", "lang": "en", "location": "San Francisco", "create_at": date("2006-12-06"), "description": "Currently http://HonestDesign.com @thatshonest Formerly @Airbnb @IDEO @Microsoft @Amazon Make things people need. Make things people love.", "followers_count": 1340, "friends_count": 1040, "statues_count": 1101 }, "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-24T10:00:15.000Z"), "id": 669213956179804161, "text": "she really makes me angry �� .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2999178991, "name": "feminist✨ ...", "screen_name": "naiyamariah", "lang": "en", "location": "w/ paige & emma .", "create_at": date("2015-01-28"), "description": "null", "followers_count": 2131, "friends_count": 755, "statues_count": 58915 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213956276228097, "text": "I need this in my life. https://t.co/6PwcE7RZhL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15719243, "name": "Corina Alfaro", "screen_name": "piccolo1979", "lang": "en", "location": "Atwater, CA", "create_at": date("2008-08-03"), "description": "null", "followers_count": 101, "friends_count": 120, "statues_count": 7409 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213956330754049, "text": "I'm tired, I'm sick and I just want to be home cuddled up next to my my fireplace with my puppy ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 457880555, "name": "Kåÿłâ Mīddłêkãüff", "screen_name": "KaylaMiddle", "lang": "en", "location": "null", "create_at": date("2012-01-07"), "description": "null", "followers_count": 397, "friends_count": 202, "statues_count": 2287 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213956330885127, "text": "@Linbei_ they be the most ignorant ones!!!", "in_reply_to_status": 669210966651858944, "in_reply_to_user": 416167711, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 416167711 }}, "user": { "id": 2465516757, "name": "LowKeyHighStrung", "screen_name": "IamaJarrod", "lang": "en", "location": "Everywhere, USA", "create_at": date("2014-04-06"), "description": "I buy books I'll never read but never brought a drink I didn't finish. Creative Genius, Universal Sex Symbol, All Around Good Guy", "followers_count": 64, "friends_count": 81, "statues_count": 298 }, "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-24T10:00:15.000Z"), "id": 669213956343398400, "text": "#CareerArc #Retail #Job alert: Retail Sales Associate | OfficeMax | #WALPOLE, MA https://t.co/wFoY5pE4Qx #Veterans #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.2017114,42.1555693"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "CareerArc", "Retail", "Job", "WALPOLE", "Veterans", "Jobs", "Hiring" }}, "user": { "id": 59950708, "name": "TMJ-MA Retail Jobs", "screen_name": "tmj_ma_retail", "lang": "en", "location": "Massachusetts", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in Massachusetts Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 410, "friends_count": 310, "statues_count": 781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walpole, MA", "id": "81732523636732fe", "name": "Walpole", "place_type": "city", "bounding_box": rectangle("-71.303783,42.088125 -71.194022,42.204216") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25021, "countyName": "Norfolk", "cityID": 2572460, "cityName": "Walpole" } }
+{ "create_at": datetime("2015-11-24T10:00:15.000Z"), "id": 669213956494503936, "text": "@politico @RealBenCarson to facts and really not the media tells us but what our eyes see", "in_reply_to_status": 669212652195983360, "in_reply_to_user": 9300262, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 9300262, 1180379185 }}, "user": { "id": 366636488, "name": "GIL DUPUY", "screen_name": "DUPUY77", "lang": "en", "location": "Miami", "create_at": date("2011-09-02"), "description": "\\", "followers_count": 95, "friends_count": 293, "statues_count": 1192 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213956582588416, "text": "Tomorrow we have the honor of playing our favorite Honky Tonk ever- the one and only Broken Spoke!!!!… https://t.co/X7TDwQWl7P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.7851791,30.2409096"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21529846, "name": "Cameron Thomas Duddy", "screen_name": "cameronduddy", "lang": "en", "location": "Dripping Springs ", "create_at": date("2009-02-21"), "description": "null", "followers_count": 1306, "friends_count": 344, "statues_count": 142 }, "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-24T10:00:15.000Z"), "id": 669213956700045314, "text": "@Gutlery @triggzyyy fuck you Doug", "in_reply_to_status": 669207743274852353, "in_reply_to_user": 1276097088, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1276097088, 2722916150 }}, "user": { "id": 2298993378, "name": "James #OASF", "screen_name": "FearsMelt", "lang": "en", "location": "IL", "create_at": date("2014-01-18"), "description": "Stay True. BO3 DON. Silver Surfer. @Johnny1Bullet covered me in a prime", "followers_count": 544, "friends_count": 90, "statues_count": 15547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roselle, IL", "id": "5e80ee08ea361155", "name": "Roselle", "place_type": "city", "bounding_box": rectangle("-88.124996,41.949557 -88.031515,42.000647") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1765806, "cityName": "Roselle" } }
+{ "create_at": datetime("2015-11-24T10:00:15.000Z"), "id": 669213957085876224, "text": "how to respond to a girl who politely says she's not interested #plsdontmessageme https://t.co/V8nFGAZook", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "plsdontmessageme" }}, "user": { "id": 3006796308, "name": "Choe", "screen_name": "sydneychoe", "lang": "en", "location": "Cleveland State Softball #3 ❥", "create_at": date("2015-02-01"), "description": "fitness junkie, comm major, aspiring to change the world like Ellen Degeneres •• IG: @bodybychoe", "followers_count": 361, "friends_count": 339, "statues_count": 5597 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213957119451137, "text": "Right off the plane to the Philly projects!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24661128, "name": "Tayyib Ali", "screen_name": "TayyibAli", "lang": "en", "location": "United States", "create_at": date("2009-03-15"), "description": "booking: dashley@icmpartners.com", "followers_count": 19449, "friends_count": 3003, "statues_count": 26142 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Croydon, PA", "id": "b68a06eb5bd7e385", "name": "Croydon", "place_type": "city", "bounding_box": rectangle("-74.914887,40.073604 -74.872306,40.105456") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42017, "countyName": "Bucks", "cityID": 4217448, "cityName": "Croydon" } }
+{ "create_at": datetime("2015-11-24T10:00:15.000Z"), "id": 669213957572288512, "text": "#Honolulu, Hawaii #Retail #Job: Assistant Restaurant Manager at Nordstrom https://t.co/0q0ut7ICCm #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Honolulu", "Retail", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 28625039, "name": "TMJ-HON Retail Jobs", "screen_name": "tmj_hon_retail", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 373, "friends_count": 311, "statues_count": 441 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213958100926465, "text": "boys are dumb all the time\nboys are dumb all the time\nboys are dumb all the time\nboys are dumb all the time https://t.co/5lR0LE5Z4k", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3320240596, "name": "Em ❁", "screen_name": "ThatsSoEmileee", "lang": "en", "location": "O R L A N D O ❀", "create_at": date("2015-06-11"), "description": "I just wanna hear Him say well done my faithful servant. #weareyx IG:@ThatsSoEmileee", "followers_count": 121, "friends_count": 217, "statues_count": 6046 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213958306463745, "text": "@___Jarvis lmaoo they not that big", "in_reply_to_status": 669213546702610434, "in_reply_to_user": 180598844, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 180598844 }}, "user": { "id": 129954208, "name": "Lady Legasus.", "screen_name": "passoutPanda", "lang": "en", "location": "ATL HOE ✨", "create_at": date("2010-04-05"), "description": "Fuck them shoes .. fuck those socks with the belt on it .. fuck your gay ass fairy faggot accent .. #Classé #FreeWop .. ♊️ #RIPShawnSmith\nIG: passoutpanda", "followers_count": 2176, "friends_count": 889, "statues_count": 150524 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213958465699840, "text": "The forecast for tonight is clear and a low of 36°F. #NationalSardinesDay #Dbcooperday #CelebrateYourUniqueTalentDay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.5333,41.5167"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NationalSardinesDay", "Dbcooperday", "CelebrateYourUniqueTalentDay" }}, "user": { "id": 42666285, "name": "Quad Cities Weather", "screen_name": "qcweather", "lang": "en", "location": "Quad Cities", "create_at": date("2009-05-26"), "description": "Is it going to be warm, wet or cold in the Quad Cities today? You can also find us on Facebook.", "followers_count": 734, "friends_count": 0, "statues_count": 10402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rock Island Arsenal, IL", "id": "8f966c3da836a196", "name": "Rock Island Arsenal", "place_type": "city", "bounding_box": rectangle("-90.570223,41.509992 -90.511303,41.523635") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17161, "countyName": "Rock Island", "cityID": 1765104, "cityName": "Rock Island Arsenal" } }
+{ "create_at": datetime("2015-11-24T10:00:15.000Z"), "id": 669213958482427905, "text": "Can you recommend anyone for this #Healthcare #job? https://t.co/GcLJzTI72j #Dayton, OH #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.1916069,39.7589478"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "Dayton", "Hiring" }}, "user": { "id": 759953168, "name": "MENTOR Network Jobs", "screen_name": "JoinMENTORNetwk", "lang": "en", "location": "null", "create_at": date("2012-08-15"), "description": "null", "followers_count": 21, "friends_count": 0, "statues_count": 2335 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213958591516672, "text": "Can you recommend anyone for this #Labor #job? https://t.co/30rCIUqhCe #HILO, HI #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-155.09,19.7297222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Labor", "job", "HILO", "Hiring", "CareerArc" }}, "user": { "id": 118676555, "name": "TMJ-HI Labor Jobs", "screen_name": "tmj_HI_LABOR", "lang": "en", "location": "Hawaii", "create_at": date("2010-03-01"), "description": "Follow this account for geo-targeted General Labor job tweets in Hawaii Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 284, "friends_count": 279, "statues_count": 8 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213958608257025, "text": "I envy people with nice grandmothers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 498292491, "name": "Lee Lee", "screen_name": "keepinupwitleea", "lang": "en", "location": "Los Angeles, CA ", "create_at": date("2012-02-20"), "description": "Mellz ❤️", "followers_count": 2313, "friends_count": 1861, "statues_count": 102140 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213958633611265, "text": "@viicttoriiaaaaa not weird at all... Moes is bae", "in_reply_to_status": 669213852886818816, "in_reply_to_user": 183480353, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 183480353 }}, "user": { "id": 354063383, "name": "Halee Baumgras", "screen_name": "HaleeBaumgras", "lang": "en", "location": "roc", "create_at": date("2011-08-12"), "description": "null", "followers_count": 418, "friends_count": 383, "statues_count": 9497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsford, NY", "id": "01431a8402105db6", "name": "Pittsford", "place_type": "city", "bounding_box": rectangle("-77.580355,43.026793 -77.479304,43.124356") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3658354, "cityName": "Pittsford" } }
+{ "create_at": datetime("2015-11-24T10:00:15.000Z"), "id": 669213958675394560, "text": "@KariHallWeather and I was about to go outside and do errands. Will it be clear later?", "in_reply_to_status": 669213370575269888, "in_reply_to_user": 425280688, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 425280688 }}, "user": { "id": 20554825, "name": "SFSOMA75DEVILDOG", "screen_name": "SBUX75DEVILDOG", "lang": "en", "location": "Views do not represent anyone", "create_at": date("2009-02-10"), "description": "For all the things that never died, To make it through the night, Love will find you.", "followers_count": 855, "friends_count": 2015, "statues_count": 39142 }, "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-24T10:00:15.000Z"), "id": 669213958734237696, "text": "My nickname is international", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 462536814, "name": "Eric", "screen_name": "E_Z_E_G", "lang": "en", "location": "null", "create_at": date("2012-01-12"), "description": "Ash :)", "followers_count": 271, "friends_count": 507, "statues_count": 5160 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213958964842496, "text": "Alcohol Brand Ambassador Part Time - Advantage Sales and Marketing: (#Wadsworth, OH) https://t.co/OkBqJZfH8k #Sales #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.7298519,41.0256101"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Wadsworth", "Sales", "Job", "Jobs", "Hiring" }}, "user": { "id": 28720800, "name": "Advantage Careers", "screen_name": "asmcareers", "lang": "en", "location": "North America", "create_at": date("2009-04-03"), "description": "Advantage Sales and Marketing has a career for you! \nCome Join Our Team Today!", "followers_count": 1983, "friends_count": 1250, "statues_count": 28831 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213959040319492, "text": "Interested in a #Actuarial #job near #WoodlandHills, CA? This could be a great fit: https://t.co/9ZiR4eOmid #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.6059197,34.1683386"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Actuarial", "job", "WoodlandHills", "Hiring", "CareerArc" }}, "user": { "id": 23235188, "name": "LA Accounting Jobs", "screen_name": "tmj_lax_acct", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-03-07"), "description": "Follow this account for geo-targeted Accounting job tweets in Los Angeles, CA. Need help? Tweet us at @CareerArc!", "followers_count": 581, "friends_count": 297, "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("2015-11-24T10:00:15.000Z"), "id": 669213959279415296, "text": "@kerry_portie yes", "in_reply_to_status": 669213872092487680, "in_reply_to_user": 1191664634, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1191664634 }}, "user": { "id": 2313004519, "name": "kylie jenner", "screen_name": "_KatieAlbares", "lang": "en", "location": "null", "create_at": date("2014-01-26"), "description": "I LOVE MY BF YES", "followers_count": 183, "friends_count": 409, "statues_count": 1161 }, "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("2015-11-24T10:00:15.000Z"), "id": 669213959392788482, "text": "Jovenel Moise & Jude Celestin iront au 2e tour selon les résultats definitifs des elections du 25Octobre @cep_haiti https://t.co/x4qkX5KGXi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user_mentions": {{ 2203438896 }}, "user": { "id": 274670842, "name": "Gerdens Chery", "screen_name": "gerdenschery", "lang": "en", "location": "Haiti", "create_at": date("2011-03-30"), "description": "Head of Communications @RTNHaiti • Government of Republic of Haiti • Opinions and tweets are my own.", "followers_count": 728, "friends_count": 647, "statues_count": 15928 }, "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-24T10:00:15.000Z"), "id": 669213959438888960, "text": "Two days left ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 215485031, "name": "5-5 Chiefs", "screen_name": "Trvll_tvmo", "lang": "en", "location": "Atlanta, GA", "create_at": date("2010-11-13"), "description": "| Ig-swagggy.t| S t a y x W a v y|", "followers_count": 740, "friends_count": 593, "statues_count": 15007 }, "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("2015-11-24T10:00:16.000Z"), "id": 669213959636041728, "text": "This damn bus is 20 minutes late. Bullshit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1011002726, "name": "Sherretta R. Day", "screen_name": "SherrettaRDay", "lang": "en", "location": "null", "create_at": date("2012-12-14"), "description": "null", "followers_count": 7, "friends_count": 10, "statues_count": 280 }, "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("2015-11-24T10:00:16.000Z"), "id": 669213959669620736, "text": "Are you #Curious about #LatinLeche @ #RainbowPlayground #Queens #NYC Tuesdays 7pm-12am? Write or DM to tell you more #BBBH #BBBHNYC #BB33", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Curious", "LatinLeche", "RainbowPlayground", "Queens", "NYC", "BBBH", "BBBHNYC", "BB33" }}, "user": { "id": 2811605702, "name": "J. C. S. #RD", "screen_name": "BBRAW727", "lang": "en", "location": "New York, NY 10018", "create_at": date("2014-09-15"), "description": "#NYC VersBottom wants to get fucked & bred can also breed holes too #BBBH #teambottom #cumwhore #cumslut #poppers #gaysex #gangbang #rawbottom #BareBbackPledge", "followers_count": 1029, "friends_count": 721, "statues_count": 6828 }, "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("2015-11-24T10:00:16.000Z"), "id": 669213959774449670, "text": "@_Rocky2x_ appreciate that!", "in_reply_to_status": 669211896596307968, "in_reply_to_user": 606613151, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 606613151 }}, "user": { "id": 124806029, "name": "C.j. Germany", "screen_name": "Geeez19", "lang": "en", "location": "East side", "create_at": date("2010-03-20"), "description": "go get it.", "followers_count": 1771, "friends_count": 900, "statues_count": 47735 }, "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("2015-11-24T10:00:16.000Z"), "id": 669213959833063424, "text": "We will call the winner tomorrow morning! https://t.co/4rSpCiBLv9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 6484412, "name": "Rich DeMuro", "screen_name": "richdemuro", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2007-05-31"), "description": "Tech Reporter @KTLAMorningNews & syndicated on @TribuneTV stations nationwide. From NJ - love gadgets, travel, life & being a new dad (again!)", "followers_count": 24700, "friends_count": 4800, "statues_count": 18109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-24T10:00:16.000Z"), "id": 669213959891779584, "text": "#Lubbock, TX #Nursing #Job: Registered Nurse (RN)-Intermediate Critical Care Unit at UMC https://t.co/QMw9niRKou #nurse #criticalcare #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-101.8919288,33.5881366"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Lubbock", "Nursing", "Job", "nurse", "criticalcare", "Jobs" }}, "user": { "id": 2940007148, "name": "UMC Nursing Jobs", "screen_name": "UMCNursingJobs", "lang": "en", "location": "null", "create_at": date("2014-12-22"), "description": "UMC Health System is the leader in comprehensive healthcare delivery in West Texas and Eastern New Mexico. Follow us to learn about our Nursing Jobs!", "followers_count": 82, "friends_count": 279, "statues_count": 64 }, "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("2015-11-24T10:00:16.000Z"), "id": 669213959904501760, "text": "@Somecallmejon it must be a Phantom Pain, Johnny! (See what I did there)", "in_reply_to_status": 669213537689055232, "in_reply_to_user": 876935449, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 876935449 }}, "user": { "id": 1551132812, "name": "The Quiet Assassin", "screen_name": "TheDoucheKnight", "lang": "en", "location": "Kerwan | Helios", "create_at": date("2013-06-27"), "description": "I'm Eric. @xlostatC is lovely. I remember it becasue no one else can! @TheDemolisher53 is bro, bro. First Nico. Private: @knight_private. I upload Arkham Photos", "followers_count": 375, "friends_count": 750, "statues_count": 15240 }, "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("2015-11-24T10:00:16.000Z"), "id": 669213960013352962, "text": "We're #hiring! Click to apply: CAE in Cyber Operations Summer Intern Program - https://t.co/5nGfXa6j4C #internship #Honolulu, HI #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "internship", "Honolulu", "Job", "Jobs" }}, "user": { "id": 154601054, "name": "TMJ-HON Intern Jobs", "screen_name": "tmj_HON_intern", "lang": "en", "location": "Honolulu, HI", "create_at": date("2010-06-11"), "description": "Follow this account for geo-targeted Internships job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 262, "friends_count": 256, "statues_count": 3 }, "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("2015-11-24T10:00:16.000Z"), "id": 669213960097406976, "text": "Mon 146.970 on FT1DR (@ Lovely Nails in Virginia Beach, VA) https://t.co/OmOfYbUx30", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.1899972,36.77955853"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 56001919, "name": "Arthur", "screen_name": "n4trqjedi", "lang": "en", "location": "APRS N of Chesapeake, VA", "create_at": date("2009-07-11"), "description": "Electronics Technician and Amateur Radio Operator N4TRQ\r\nhttp://n4trq.wordpress.com", "followers_count": 419, "friends_count": 681, "statues_count": 46480 }, "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-24T10:00:16.000Z"), "id": 669213960147591168, "text": "It looks like a snow day for a lot of people today ������������������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2169833372, "name": "kami kutsunai", "screen_name": "kamikatsu15", "lang": "en", "location": "null", "create_at": date("2013-11-02"), "description": "b♥ wailuku ➳ klamath falls", "followers_count": 346, "friends_count": 296, "statues_count": 6107 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Klamath Falls, OR", "id": "a167da250f5ab0b9", "name": "Klamath Falls", "place_type": "city", "bounding_box": rectangle("-121.835805,42.170168 -121.727603,42.268381") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41035, "countyName": "Klamath", "cityID": 4139700, "cityName": "Klamath Falls" } }
+{ "create_at": datetime("2015-11-24T10:00:16.000Z"), "id": 669213960265166848, "text": "Genesis Healthc is hiring! Certified Nursi #jobs in CEDARBROOK Apply today https://t.co/q2gZ0ahnmF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.151628,40.085044"), "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": 165, "friends_count": 42, "statues_count": 68532 }, "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": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2015-11-24T10:00:16.000Z"), "id": 669213960298741760, "text": "I'm at QFC in Seattle, WA https://t.co/orQmKaWp82", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.29680061,47.66230923"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 45429835, "name": "Robert Paul Dobson", "screen_name": "Seattleb0b", "lang": "en", "location": "Seattle", "create_at": date("2009-06-07"), "description": "life is wonderful!", "followers_count": 41, "friends_count": 201, "statues_count": 823 }, "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-24T10:00:16.000Z"), "id": 669213960755875842, "text": "N4TRQ-8 Near Lovely Nails, Virginia Beach, VA https://t.co/Jebzp2ND2R #APRS 13b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.19033,36.7795"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "APRS" }}, "user": { "id": 56001919, "name": "Arthur", "screen_name": "n4trqjedi", "lang": "en", "location": "APRS N of Chesapeake, VA", "create_at": date("2009-07-11"), "description": "Electronics Technician and Amateur Radio Operator N4TRQ\r\nhttp://n4trq.wordpress.com", "followers_count": 419, "friends_count": 681, "statues_count": 46481 }, "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-24T10:00:16.000Z"), "id": 669213960822894593, "text": "https://t.co/BkuCtZOeeY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2716146803, "name": "DAVID CURRY", "screen_name": "davidlcurry1", "lang": "en", "location": "Planet Earth, Portland, Oregon", "create_at": date("2014-07-17"), "description": "THE KINGDOM OF GOD IS WITHIN YOU. Live life to its fullest! Share your LOVE wherever you go❤️Let's make Earth a better place to live/learn. Pics not mine.", "followers_count": 4023, "friends_count": 4206, "statues_count": 22673 }, "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("2015-11-24T10:00:16.000Z"), "id": 669213960839688193, "text": "@FarmFairyCrafts You're speaking our language! ������", "in_reply_to_status": 668986673452027906, "in_reply_to_user": 2680729040, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2680729040 }}, "user": { "id": 602373587, "name": "JustFruit Bars", "screen_name": "JustFruitBars", "lang": "en", "location": "North Bonneville, WA", "create_at": date("2012-06-07"), "description": "Made with natural fruit & nothing else, JustFruit™ bars are real fruit, but better: Convenient, with many delicious flavors. #GlutenFree #DairyFree #NonGMO", "followers_count": 2541, "friends_count": 3684, "statues_count": 3687 }, "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("2015-11-24T10:00:16.000Z"), "id": 669213961196208128, "text": "I don't know what to wear tonight ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 427733029, "name": "Yentihw☄", "screen_name": "dannofrench_", "lang": "en", "location": "With C❤️", "create_at": date("2011-12-03"), "description": "1⃣9⃣|self centered✨", "followers_count": 524, "friends_count": 600, "statues_count": 38225 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:16.000Z"), "id": 669213961364086785, "text": "Browns need to fire there entire front office.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325760506, "name": "Julian Cortes", "screen_name": "JulianCortes24", "lang": "en", "location": "Amherst, Ohio", "create_at": date("2011-06-28"), "description": "null", "followers_count": 840, "friends_count": 453, "statues_count": 36387 }, "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-24T10:00:16.000Z"), "id": 669213961393446914, "text": "Résumé: flobots - handlebars lyrics", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 709556695, "name": "YUNG Breeze", "screen_name": "AaronRies1", "lang": "en", "location": "cincinnati, OH", "create_at": date("2012-07-21"), "description": "my name is YUNG Breeze/Memorize your Memories/Im a 17 year old rap artist and I #skateeverydamnday!", "followers_count": 344, "friends_count": 211, "statues_count": 4599 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edgewood, KY", "id": "6e734e4ae762bec9", "name": "Edgewood", "place_type": "city", "bounding_box": rectangle("-84.591846,38.991867 -84.533385,39.029105") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21117, "countyName": "Kenton", "cityID": 2123932, "cityName": "Edgewood" } }
+{ "create_at": datetime("2015-11-24T10:00:16.000Z"), "id": 669213961527537664, "text": "Straight up �� https://t.co/QFFb3Miefl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 175558145, "name": "Dopeman", "screen_name": "donovan_khv", "lang": "en", "location": "Yay Area", "create_at": date("2010-08-06"), "description": "Class of '15 .. ,I DONT FUCK WITH NONE OF YOU NIGGAS an if it ain't about no money you gettin no attention #KHILL", "followers_count": 384, "friends_count": 518, "statues_count": 13774 }, "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("2015-11-24T10:00:16.000Z"), "id": 669213961833873408, "text": "Genesis Healthc is hiring! Personal Care A #jobs in CLARKS GREEN Apply today https://t.co/cWIwgnGFxH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.730478,41.458054"), "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": 165, "friends_count": 42, "statues_count": 68533 }, "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": 42069, "countyName": "Lackawanna" } }
+{ "create_at": datetime("2015-11-24T10:00:16.000Z"), "id": 669213961993220096, "text": "i made more popcorn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.1823403,42.8086976"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2882682793, "name": "Kittyyy( ^ω^ )", "screen_name": "KatPfeiffer1", "lang": "en", "location": "null", "create_at": date("2014-10-29"), "description": "*・゜゚・*:.。..。.:*・'(*゚▽゚*)'・*:.。. .。.:*・゜゚・* bring me the horizon", "followers_count": 643, "friends_count": 2062, "statues_count": 304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salem, NH", "id": "00ef8954160d7a3c", "name": "Salem", "place_type": "city", "bounding_box": rectangle("-71.284469,42.73759 -71.166666,42.852318") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25009, "countyName": "Essex", "cityID": 2529405, "cityName": "Haverhill" } }
+{ "create_at": datetime("2015-11-24T10:00:16.000Z"), "id": 669213962089566208, "text": "We're #hiring! Click to apply: Senior HCM Sales Consultant - https://t.co/4DQYNslNGk #Sales #Westborough, MA #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.6161777,42.2695387"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Sales", "Westborough", "Job", "Jobs", "CareerArc" }}, "user": { "id": 22038280, "name": "TMJ-MA-US Sales Jobs", "screen_name": "tmj_MA_sales", "lang": "en", "location": "Massachusetts", "create_at": date("2009-02-26"), "description": "Follow this account for geo-targeted Sales job tweets in Massachusetts Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 397, "friends_count": 308, "statues_count": 423 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westborough, MA", "id": "4ac6e2d5a5617918", "name": "Westborough", "place_type": "city", "bounding_box": rectangle("-71.677611,42.226598 -71.540828,42.303624") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25027, "countyName": "Worcester", "cityID": 2575050, "cityName": "Westborough" } }
+{ "create_at": datetime("2015-11-24T10:00:16.000Z"), "id": 669213962110525440, "text": "@DonnieWahlberg #TWUG have a great day stay warm. We did safety drills today. Gotany?", "in_reply_to_status": 669213631666520065, "in_reply_to_user": 24776235, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TWUG" }}, "user_mentions": {{ 24776235 }}, "user": { "id": 1521086168, "name": "kristin wagener❤", "screen_name": "bergganator", "lang": "en", "location": "null", "create_at": date("2013-06-15"), "description": "I love nkotb and bsb .i live, love and laugh, i am me, i stand up for what i believe, i got my follow donnie wahlberg & 1/25/14 & my danny wood 2/23/14 love", "followers_count": 1905, "friends_count": 2089, "statues_count": 33912 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Harrisonville, MO", "id": "5ff4d92096b59efd", "name": "Harrisonville", "place_type": "city", "bounding_box": rectangle("-94.37986,38.63171 -94.314529,38.677534") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29037, "countyName": "Cass", "cityID": 2930610, "cityName": "Harrisonville" } }
+{ "create_at": datetime("2015-11-24T10:00:16.000Z"), "id": 669213962618015745, "text": "When you q-tip your ears right after a shower https://t.co/7jgFnLu0je", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 433787533, "name": "☁️YoungVapeLawd☁️", "screen_name": "Jiggans_", "lang": "en", "location": "null", "create_at": date("2011-12-10"), "description": "caroline! wenatchee, washington state university hmu for kicks 509-860-8556", "followers_count": 592, "friends_count": 277, "statues_count": 8831 }, "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("2015-11-24T10:00:16.000Z"), "id": 669213962643353603, "text": "Hey, you call me up again just to break me like a promise. So casually cruel in the name of being honest.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3092599437, "name": "Tayoholics ❤️", "screen_name": "WorldToTaylor", "lang": "en", "location": "Chicago, IL", "create_at": date("2015-03-16"), "description": "I'm Ashley, Welcome To The World According To Taylor Swift Where You Can Get 24/7 Everything #TaylorSwift. I Follow Back. Fansite Owned by @ShadesOfTayGrey", "followers_count": 5595, "friends_count": 5540, "statues_count": 7498 }, "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-24T10:00:16.000Z"), "id": 669213962693689344, "text": "I did the digital da$h I f*cked the b*tch on the pa$$enger ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 64263094, "name": "$.L.$ TiZZie", "screen_name": "TiZZieGreaTesT", "lang": "en", "location": "Warner Robins, GA", "create_at": date("2009-08-09"), "description": "I'm just a 6'6 young leader outta Warner Robin$ follow me on IG http://instagram.com/tizzie_greatest #MiLk2 i$ out now link in the bio", "followers_count": 2498, "friends_count": 1761, "statues_count": 114797 }, "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("2015-11-24T10:00:16.000Z"), "id": 669213962710331392, "text": "you know it's real when he comments on your old insta posts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 610158320, "name": "Katherine Mannix", "screen_name": "KathyMannix", "lang": "en", "location": "null", "create_at": date("2012-06-16"), "description": "null", "followers_count": 148, "friends_count": 146, "statues_count": 436 }, "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-24T10:00:16.000Z"), "id": 669213962760687616, "text": "#laugh it's good for the #soul #attitude #major #fashion #hair #fun #because xo @ Gold's Gym https://t.co/UYLmzulsGp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.74775386,31.10587825"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "laugh", "soul", "attitude", "major", "fashion", "hair", "fun", "because" }}, "user": { "id": 20386541, "name": "Ted Gibson", "screen_name": "tedgibson", "lang": "en", "location": "NYC", "create_at": date("2009-02-08"), "description": "Beauty Is Individual,celebrity hairstylist,salon owner, product developer, snapchat tedgibsonbeauty", "followers_count": 154986, "friends_count": 5723, "statues_count": 48143 }, "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("2015-11-24T10:00:16.000Z"), "id": 669213962769076226, "text": "please deactivate bc you sound so stupid https://t.co/aWVXrAZjX2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 271690493, "name": "tca$h☹️", "screen_name": "idknatsirt", "lang": "en", "location": "null", "create_at": date("2011-03-24"), "description": "tview and hoh and iridocyclitis", "followers_count": 881, "friends_count": 687, "statues_count": 80576 }, "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("2015-11-24T10:00:16.000Z"), "id": 669213962827923456, "text": "Wtf oh fuck no https://t.co/VoLe2XMUx6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2147608774, "name": "Nugget", "screen_name": "eliza_darien007", "lang": "en", "location": "nature ", "create_at": date("2013-10-21"), "description": "let's trip, get a furry blanket and let go adventure.", "followers_count": 364, "friends_count": 385, "statues_count": 12675 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Canonsburg, PA", "id": "d64d438446aaf66a", "name": "Canonsburg", "place_type": "city", "bounding_box": rectangle("-80.206972,40.242046 -80.158673,40.29827") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42125, "countyName": "Washington", "cityID": 4211152, "cityName": "Canonsburg" } }
+{ "create_at": datetime("2015-11-24T10:00:16.000Z"), "id": 669213962999754752, "text": "On this day in 1434, the River Thames froze over. The freeze lasted until February of 1435.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 57501485, "name": "Paul Jimerson", "screen_name": "pauljimerson", "lang": "en", "location": "Santa Cruz, California", "create_at": date("2009-07-16"), "description": "Ocean Obsessive Blissorder", "followers_count": 5948, "friends_count": 5828, "statues_count": 133125 }, "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("2015-11-24T10:00:16.000Z"), "id": 669213963024859136, "text": "Partly cloudy this afternoon, high 65 (18 C). Low 56 (13 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712424, "name": "DallasWX", "screen_name": "DallasWX", "lang": "en", "location": "Dallas, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Dallas, Texas", "followers_count": 191, "friends_count": 93, "statues_count": 8034 }, "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-24T10:00:16.000Z"), "id": 669213963108814848, "text": "Can you recommend anyone for this #job? Medical Director, Clinical - https://t.co/A5I7eI6cA9 #Princeton, NJ #IT #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.6590472,40.3487181"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Princeton", "IT", "Hiring", "CareerArc" }}, "user": { "id": 21630910, "name": "NJ Software Dev Jobs", "screen_name": "tmj_njc_it", "lang": "en", "location": "Central NJ", "create_at": date("2009-02-22"), "description": "Follow this account for geo-targeted Software Dev. - General/IT job tweets in Central NJ from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 529, "friends_count": 345, "statues_count": 132 }, "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.325071 -74.619759,40.37435") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34021, "countyName": "Mercer", "cityID": 3460900, "cityName": "Princeton" } }
+{ "create_at": datetime("2015-11-24T10:00:16.000Z"), "id": 669213963125727232, "text": "Genesis Healthc is hiring! Registered Nurs #jobs in CEDARBROOK Apply today https://t.co/scrHgwNc1e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.151628,40.085044"), "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": 165, "friends_count": 42, "statues_count": 68534 }, "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": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2015-11-24T10:00:16.000Z"), "id": 669213963167637504, "text": "Thanksgiving lunch with my princesses!! Moments like these is what I'm… https://t.co/dtes5CU8mu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.3617096,36.628891"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 232934847, "name": "Omar Perez", "screen_name": "ohommie8", "lang": "en", "location": "USA", "create_at": date("2011-01-01"), "description": "null", "followers_count": 39, "friends_count": 106, "statues_count": 731 }, "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("2015-11-24T10:00:16.000Z"), "id": 669213963343626241, "text": "\"Costco aka short person hell\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 4214872396, "name": "Starr Y", "screen_name": "snalexhamilton", "lang": "en", "location": "your ass", "create_at": date("2015-11-17"), "description": "just a small town hamilton, living in a lonely, political wooooorld", "followers_count": 11, "friends_count": 45, "statues_count": 161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Royal Palm Beach, FL", "id": "a386795418308ec0", "name": "Royal Palm Beach", "place_type": "city", "bounding_box": rectangle("-80.257234,26.665515 -80.156201,26.736442") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1262100, "cityName": "Royal Palm Beach" } }
+{ "create_at": datetime("2015-11-24T10:00:17.000Z"), "id": 669213963729551360, "text": "So in other words we are all completely safe ���� https://t.co/SImPufWccj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2393075864, "name": "Lucas Lee", "screen_name": "L_timestwo", "lang": "en", "location": "UC Santa Cruz", "create_at": date("2014-03-16"), "description": "IG ltimestwo", "followers_count": 185, "friends_count": 226, "statues_count": 418 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213963746316288, "text": "I'M SO PISSED I'M GONNA FUCK SOMEONE UP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3348140687, "name": "ama", "screen_name": "aamandahhx", "lang": "en", "location": "null", "create_at": date("2015-06-27"), "description": "volleyball | ⛈", "followers_count": 252, "friends_count": 213, "statues_count": 236 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213963838619648, "text": "We're #hiring! Read about our latest #job opening here: Family Life Education, Adjunct, Part-Time, Temporary - https://t.co/9qxo0n3suB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.4733564,35.6619513"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job" }}, "user": { "id": 2209214250, "name": "Jobs at UCO", "screen_name": "JobsatUCO", "lang": "en", "location": "Edmond, OK", "create_at": date("2013-11-22"), "description": "The University of Central #Oklahoma is constantly on the lookout for the latest talent to add to the exceptional workforce we already have on campus.", "followers_count": 569, "friends_count": 223, "statues_count": 909 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213963876306944, "text": "Want to work at Hilton Worldwide? We're #hiring in #Honolulu, HI! Click for details: https://t.co/jt8lMaToaM #Hospitality #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.86,21.31"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Honolulu", "Hospitality", "Job", "Jobs" }}, "user": { "id": 23897767, "name": "TMJ-HON Hosp. Jobs", "screen_name": "tmj_hon_hrta", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-11"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 406, "friends_count": 292, "statues_count": 257 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213963960238080, "text": "This shit ain't going down like this again. NO repeat cycles here.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3266809291, "name": "LisaMarie", "screen_name": "MCMBCIII", "lang": "en", "location": "Arlington, TX", "create_at": date("2015-07-02"), "description": "God 1st #LongLiveCarlDarwin 1/12/15 the day my heart broke. #28", "followers_count": 239, "friends_count": 444, "statues_count": 8603 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213963977146369, "text": "Time for another tattoo or more.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 496514345, "name": "Evelyn Martinez", "screen_name": "EvelynMar_96", "lang": "en", "location": "null", "create_at": date("2012-02-18"), "description": "http://instagram.com/Evelynmar__", "followers_count": 1561, "friends_count": 217, "statues_count": 1707 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chelsea, MA", "id": "5f8c8fa6711d7ce6", "name": "Chelsea", "place_type": "city", "bounding_box": rectangle("-71.055294,42.384142 -71.009459,42.4146") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2513205, "cityName": "Chelsea" } }
+{ "create_at": datetime("2015-11-24T10:00:17.000Z"), "id": 669213963989684224, "text": "Make a Dash to The Secret Stash today, check out all our great products and awesome specials! Sign up… https://t.co/hm6Zy7DjBX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.8705233,38.84869027"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 269352322, "name": "The Secret Stash", "screen_name": "SecretStashMMJ", "lang": "en", "location": "Colorado Springs, CO", "create_at": date("2011-03-20"), "description": "null", "followers_count": 54, "friends_count": 1, "statues_count": 207 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213964061011969, "text": "I'll dunk on Ray again like it ain't shit to me but just a little two step gather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54416033, "name": "Mike Lowry", "screen_name": "ANREE_3000", "lang": "en", "location": "PS-118", "create_at": date("2009-07-06"), "description": "Living my dream #CollegeBaseballPlayer #FOE IG:Anree_3000⚾", "followers_count": 617, "friends_count": 259, "statues_count": 21512 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213964111253505, "text": "@gullett_james @MLive @umichfootball most are, but early on JH thanked BH for 'setting the table' for a lot of the success they may have", "in_reply_to_status": 669210684622770176, "in_reply_to_user": 3273930834, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3273930834, 12952842, 27902825 }}, "user": { "id": 1894825736, "name": "Mike Rapert", "screen_name": "BikerMike03flhr", "lang": "en", "location": "null", "create_at": date("2013-09-22"), "description": "Retired; Father of Two, Husband for 34 years; Former Machinist, Truck Driver, Medical Equipment Delivery", "followers_count": 91, "friends_count": 315, "statues_count": 3800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jackson, MI", "id": "06ab95cfe367eee7", "name": "Jackson", "place_type": "city", "bounding_box": rectangle("-84.493659,42.176663 -84.364192,42.276421") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26075, "countyName": "Jackson", "cityID": 2641420, "cityName": "Jackson" } }
+{ "create_at": datetime("2015-11-24T10:00:17.000Z"), "id": 669213964123955200, "text": "It's also the time of year when people get mad at me for that reason that I just say, \"well, it is almost Christmas time ����������������������������������.\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 704070930, "name": "kimmi hill", "screen_name": "kimmiTHAT", "lang": "en", "location": "prob @ omega, omega, or omega.", "create_at": date("2012-07-18"), "description": "there's a madness in us all.", "followers_count": 463, "friends_count": 622, "statues_count": 27126 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213964249800704, "text": "Genesis Healthc is hiring! Registered Nurs #jobs in PHILADELPHIA Apply today https://t.co/yXfcbaXlSe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.1644,39.9525"), "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": 165, "friends_count": 42, "statues_count": 68535 }, "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-24T10:00:17.000Z"), "id": 669213964308520960, "text": "https://t.co/R5xTye8dTE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 62100109, "name": "Arnie Taylor", "screen_name": "arnietaylor", "lang": "en", "location": "ÜT: 41.489156,-81.690582", "create_at": date("2009-08-01"), "description": "..helping others find the best entertainment value in the free world...and maybe an opinion or two every now and then... :)", "followers_count": 202, "friends_count": 405, "statues_count": 4934 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213964467703808, "text": "Former CBP officer in 'peeping' case arrested again for allegedly violating a protective order https://t.co/yP1ciJ5MBw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-106.4855885,31.76017093"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 73663373, "name": "El Paso CP", "screen_name": "ElPasoCP", "lang": "en", "location": "El Paso, TX", "create_at": date("2009-09-12"), "description": "El Paso City And Press", "followers_count": 1195, "friends_count": 5, "statues_count": 41095 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213964518096896, "text": "@JessSanchez990 you love me and you know it��❤️��", "in_reply_to_status": 669213849740947456, "in_reply_to_user": 2272577563, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2272577563 }}, "user": { "id": 1565106661, "name": "A$AP BRIG$", "screen_name": "brigs_jensen", "lang": "en", "location": "BBW", "create_at": date("2013-07-03"), "description": "I was an atheist until I realized I am a god - #kanye2020", "followers_count": 1006, "friends_count": 824, "statues_count": 7840 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213964526489600, "text": "@idcnhu good idea", "in_reply_to_status": 669213795173056512, "in_reply_to_user": 633289228, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 633289228 }}, "user": { "id": 1046729460, "name": "naomi", "screen_name": "NaomiYitna", "lang": "en", "location": "null", "create_at": date("2012-12-29"), "description": "null", "followers_count": 401, "friends_count": 175, "statues_count": 9767 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213964539052032, "text": "My fav movie �� https://t.co/fscyv4okZh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2439584227, "name": "leslie", "screen_name": "_lelyyyy_", "lang": "en", "location": "Austin, Tejas", "create_at": date("2014-04-11"), "description": "Doing whatever the hell makes me happy", "followers_count": 1378, "friends_count": 631, "statues_count": 36339 }, "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-24T10:00:17.000Z"), "id": 669213964539162625, "text": "I can't wait to go home for thanksgiving bc my brother turned my room into an art studio and I don't even have a bed anymore!!! #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": 35320379, "name": "Allie Britt", "screen_name": "thealliebritt", "lang": "en", "location": "by the water or woods, usa", "create_at": date("2009-04-25"), "description": "FAU||AΔΠ • just tryna go owls •", "followers_count": 696, "friends_count": 762, "statues_count": 12151 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213964732129281, "text": "I'll never learn,I'd pour kerosene on everything I love and watch it burn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 380622365, "name": "john ezzai", "screen_name": "JohnEzzai", "lang": "en", "location": "Philadelphia", "create_at": date("2011-09-26"), "description": "am John, friend, lover, son, student, scumbag.", "followers_count": 667, "friends_count": 616, "statues_count": 16212 }, "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-24T10:00:17.000Z"), "id": 669213964786491392, "text": "I enjoyed watching Bonanza on @insptv today", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.2676895,42.2812554"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 165832415 }}, "user": { "id": 4329679814, "name": "Hi Everyone :)", "screen_name": "_MLFan_", "lang": "en", "location": "United States", "create_at": date("2015-11-22"), "description": "null", "followers_count": 10, "friends_count": 148, "statues_count": 38 }, "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", "cityID": 2621020, "cityName": "Dearborn Heights" } }
+{ "create_at": datetime("2015-11-24T10:00:17.000Z"), "id": 669213964820025345, "text": "I was worrisome highschool", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325024183, "name": "Andrea Stanford", "screen_name": "_andreastanford", "lang": "en", "location": "Grambling, LA", "create_at": date("2011-06-27"), "description": "They call me Angie #Gramfam19 #GSU", "followers_count": 830, "friends_count": 215, "statues_count": 28719 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213964824219648, "text": "@politicspop No big deal! I make typos every day. ! @KillerMike @rachelmaddow @BernieSanders @larrygilmore", "in_reply_to_status": 669209030699704320, "in_reply_to_user": 3008999627, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3008999627, 21265120, 16302198, 216776631, 45576064 }}, "user": { "id": 379869831, "name": "Charles Munn,", "screen_name": "CharlesMunn1", "lang": "en", "location": "Becoming", "create_at": date("2011-09-25"), "description": "Democratic Socialist,painter&author. Mother was a Texan, father an ET.Tries to be kind&follows back all. Bernie Sanders, 2016!", "followers_count": 77710, "friends_count": 76066, "statues_count": 138569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orting, WA", "id": "2061d6e4f83d9ddf", "name": "Orting", "place_type": "city", "bounding_box": rectangle("-122.234941,47.063225 -122.15967,47.122039") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5352005, "cityName": "Orting" } }
+{ "create_at": datetime("2015-11-24T10:00:17.000Z"), "id": 669213965227065346, "text": "Wind 2.0 mph SE. Barometer 30.438 in, Falling slowly. Temperature 45.1 °F. Rain today 0.00 in. Humidity 58%", "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": 23299 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213965260447746, "text": "If I smell like cigarette smoke bc I needed to take a piss I'm gonna kill someone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3764877372, "name": "rye 「today」", "screen_name": "MikeyKudos", "lang": "en", "location": "Arizona, USA", "create_at": date("2015-10-02"), "description": "In spite of everything, I still believe people are really good at heart. | CdS '18 | other accounts: http://twitlonger.com/show/n_1snsv3r |", "followers_count": 127, "friends_count": 134, "statues_count": 5941 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213965482749952, "text": "Alot of you niggas is soft", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 844761523, "name": "真荚", "screen_name": "dreeewdreew", "lang": "en", "location": "focu$ed", "create_at": date("2012-09-24"), "description": "I know more pain than I know happiness.", "followers_count": 1458, "friends_count": 1206, "statues_count": 19196 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213965671510016, "text": "@e___cortes03 how about Christmas and new years?��", "in_reply_to_status": 669213777263517696, "in_reply_to_user": 1409208186, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1409208186 }}, "user": { "id": 2744209202, "name": "Emma Cortes", "screen_name": "Emmaa_Nina", "lang": "en", "location": "Hillsboro, OR", "create_at": date("2014-08-19"), "description": "CHC⚾❤", "followers_count": 173, "friends_count": 198, "statues_count": 1692 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213966493609985, "text": "Cloudy this afternoon, high 67 (19 C). Low 60 (16 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712472, "name": "AustinWX", "screen_name": "AustinWX", "lang": "en", "location": "Austin, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Austin, Texas", "followers_count": 1009, "friends_count": 1368, "statues_count": 7895 }, "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-24T10:00:17.000Z"), "id": 669213966522974208, "text": "Appropriately raining.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2297090690, "name": "Dom", "screen_name": "dominicjcar", "lang": "en", "location": "Cotati, CA", "create_at": date("2014-01-17"), "description": "Hell yeah brother", "followers_count": 330, "friends_count": 392, "statues_count": 1683 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213966690832385, "text": "#ThanksgivingWithBlackFamilies ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "ThanksgivingWithBlackFamilies" }}, "user": { "id": 58031191, "name": "♌️08.06.1996♌️1", "screen_name": "Nickname_ManMan", "lang": "en", "location": "Montgomery Village, MD ", "create_at": date("2009-07-18"), "description": "#MSU18 D1 IG|@ManMan101 Tracklife #trackNation ♌️#TeamLeo Psalm23 Phillipians4:13 Isaiah54:17 ΣΑΠ MSU Vprez MorganStateSga|Sophomore Class|Chief Of Staff 15-16", "followers_count": 1696, "friends_count": 2088, "statues_count": 96159 }, "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-24T10:00:17.000Z"), "id": 669213966757838849, "text": "Want to work at Sibley Memorial Hospital? We're #hiring in #Washington, DC! Click for details: https://t.co/YNfkr7kVzr #Healthcare #Job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.0363658,38.8951118"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Washington", "Healthcare", "Job" }}, "user": { "id": 22148215, "name": "TMJ-WAS Nursing Jobs", "screen_name": "tmj_dc_nursing", "lang": "en", "location": "Washington, DC", "create_at": date("2009-02-27"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in DC Metro. Need help? Tweet us at @CareerArc!", "followers_count": 371, "friends_count": 306, "statues_count": 505 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213966808317952, "text": "I like ZENIT Over 1.5 Team Goals... Odds⬇️⬇️⬇️ @ 1.68 \nIm just waiting!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2979140740, "name": "THE BET SOCIETY", "screen_name": "thebetsociety", "lang": "en", "location": "Miami, FL", "create_at": date("2015-01-14"), "description": "Beat the Odds, when the Odds are against you... Im not a TIPSTER, Im just SHARING my bets with the WORLD. Registered on: http://winafy.com", "followers_count": 3214, "friends_count": 295, "statues_count": 4783 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213966879584257, "text": "@DonnieWahlberg can I please get a Hello #TWUG never gotten a Hello before or anything besides #TWUG & #CUFFED ������ https://t.co/qbqNotEwA6", "in_reply_to_status": -1, "in_reply_to_user": 24776235, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TWUG", "TWUG", "CUFFED" }}, "user_mentions": {{ 24776235 }}, "user": { "id": 3668408955, "name": "Sammi Smith", "screen_name": "SammiSm1992", "lang": "en", "location": "West Virginia, USA ", "create_at": date("2015-09-15"), "description": "Jesus Christ comes first! #DonnieGirl #LOVE #loveeternal ❤️ #BlueBloods Followed by @DannyWood 10/30/15 Macky, Jojo, AaronCarter @JennyMcCarthy 11-19-15", "followers_count": 351, "friends_count": 685, "statues_count": 6645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Gay, WV", "id": "3489c9f090cc9b55", "name": "Fort Gay", "place_type": "city", "bounding_box": rectangle("-82.613834,38.112319 -82.582679,38.126423") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54099, "countyName": "Wayne", "cityID": 5428516, "cityName": "Fort Gay" } }
+{ "create_at": datetime("2015-11-24T10:00:17.000Z"), "id": 669213967009644545, "text": "I literally overthink e v e r y t h i n g ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 227712800, "name": "Katie Karger", "screen_name": "ItsThaatBlondie", "lang": "en", "location": "w/ casey probably", "create_at": date("2010-12-17"), "description": "Trust the vibes you get; energy never lies. USI '18", "followers_count": 437, "friends_count": 308, "statues_count": 12463 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213967437455360, "text": "A rare spa day... (@ Halina European Day Spa in Austin, TX) https://t.co/iAzlbEEpZI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.739442,30.328022"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14137772, "name": "Ashley McNamara", "screen_name": "ashleymcnamara", "lang": "en", "location": "Austin, TX", "create_at": date("2008-03-12"), "description": "I am 15% Unicorn, 25% Developer, 70% Sarcastic, 44% Mathematician, and 100% Awesome. If you're mathing right now, Refer to sarcastic. A Racker/Tweets are my own", "followers_count": 7709, "friends_count": 543, "statues_count": 18100 }, "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-24T10:00:17.000Z"), "id": 669213967705763840, "text": "@JayyNaee that's why I said that ��. How long you here?", "in_reply_to_status": 669213834855477249, "in_reply_to_user": 741643572, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 741643572 }}, "user": { "id": 3243642080, "name": "n e s e", "screen_name": "NeseWalters", "lang": "en", "location": "Fort Smith, AR", "create_at": date("2015-06-12"), "description": "college athlete // #staytuned", "followers_count": 381, "friends_count": 220, "statues_count": 3506 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213967814791168, "text": "@sarah_nicholass @stockwally YES", "in_reply_to_status": 669026506648129536, "in_reply_to_user": 2599086805, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2599086805, 2828855231 }}, "user": { "id": 908741882, "name": "ℓєχ", "screen_name": "lexialvey", "lang": "en", "location": "null", "create_at": date("2012-10-27"), "description": "http://PHHHOTO.COM/LEXIALVEY -------http://gofundme.com/9cabytus", "followers_count": 879, "friends_count": 640, "statues_count": 859 }, "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("2015-11-24T10:00:17.000Z"), "id": 669213967835906048, "text": ".@Mayhem said I have to tweet this for my vote to count in the #MayhemTweetOff. Go Florida! #MeltFSU https://t.co/S7aUpQINbT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MayhemTweetOff", "MeltFSU" }}, "user_mentions": {{ 1858823539 }}, "user": { "id": 1697352404, "name": "ben powers", "screen_name": "powersss_", "lang": "en", "location": "McCarthy", "create_at": date("2013-08-24"), "description": "†", "followers_count": 525, "friends_count": 471, "statues_count": 14273 }, "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-24T10:00:18.000Z"), "id": 669213967970013184, "text": "Hooping in the bloody 10s, gotta add some swag on the road!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2665595189, "name": "Marvelous", "screen_name": "MarcusMowens6", "lang": "en", "location": "null", "create_at": date("2014-07-02"), "description": "New twitter, Lamar17, #Xman", "followers_count": 428, "friends_count": 397, "statues_count": 8424 }, "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("2015-11-24T10:00:18.000Z"), "id": 669213967990976512, "text": "We're #hiring! Click to apply: Registered Nurse RN -FT, 7p-7a- Nursing & Rehab- Bashford - https://t.co/kFIyCPcssZ #Job #Louisville, KY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.7584557,38.2526647"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Job", "Louisville" }}, "user": { "id": 22160062, "name": "TMJ-SDF Nursing Jobs", "screen_name": "tmj_sdf_nursing", "lang": "en", "location": "Louisville, KY", "create_at": date("2009-02-27"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Louisville, KY from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 384, "friends_count": 302, "statues_count": 144 }, "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-24T10:00:18.000Z"), "id": 669213968028839936, "text": "�� https://t.co/SKo0z1G9h5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2916372886, "name": "princess micah", "screen_name": "Micah78619625", "lang": "en", "location": "im chuck bass", "create_at": date("2014-12-11"), "description": "be happy stay trippy", "followers_count": 449, "friends_count": 1410, "statues_count": 4750 }, "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("2015-11-24T10:00:18.000Z"), "id": 669213968188198912, "text": "If Cook had as many touches as McCaffrey he would have 2,427 yards from scrimmage", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 128791015, "name": "Sports and Law", "screen_name": "mercernole", "lang": "en", "location": "Tallahassee", "create_at": date("2010-04-01"), "description": "Husband/Father/NOLE Fan/Music Lover/Amateur Sociologist/Constitutionalist. Opinions are my own and not considered that of any other person/entity/agency.", "followers_count": 868, "friends_count": 1289, "statues_count": 7684 }, "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("2015-11-24T10:00:18.000Z"), "id": 669213968213372928, "text": "Someone went heavy on the cologne on this plane. #clevelandinternation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.838628,41.412644"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "clevelandinternation" }}, "user": { "id": 16369654, "name": "Ronnie", "screen_name": "nawthatstupid", "lang": "en", "location": "Chagrin Falls, OH", "create_at": date("2008-09-19"), "description": "But when you tell my story, and I hope somebody does. Remember me as something bigger than I was.", "followers_count": 462, "friends_count": 1280, "statues_count": 4917 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gate A3", "id": "07d9dd8530c88003", "name": "Gate A3", "place_type": "poi", "bounding_box": rectangle("-81.8386281,41.4126439 -81.838628,41.412644") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3909288, "cityName": "Brook Park" } }
+{ "create_at": datetime("2015-11-24T10:00:18.000Z"), "id": 669213968376950784, "text": "@Piteo_10 I'll admit it's a tad bit too sweet����", "in_reply_to_status": 669213829046267904, "in_reply_to_user": 349169857, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 349169857 }}, "user": { "id": 2590949035, "name": "MAMBA", "screen_name": "white_Mamb8", "lang": "en", "location": "null", "create_at": date("2014-06-26"), "description": "War Eagle! WHS sophmore. @angelbangel03 ❤️", "followers_count": 380, "friends_count": 315, "statues_count": 6264 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jasper, AL", "id": "fa4a0db0e23833d3", "name": "Jasper", "place_type": "city", "bounding_box": rectangle("-87.313548,33.806298 -87.212496,33.910309") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1127, "countyName": "Walker", "cityID": 138416, "cityName": "Jasper" } }
+{ "create_at": datetime("2015-11-24T10:00:18.000Z"), "id": 669213968393596928, "text": "#Nursing #Job alert: Nurse... | Cancer Treatment Centers of America | #Goodyear, AZ https://t.co/9BjZXuKINE #Jobs https://t.co/5un5xUfQ8d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.3596206,33.4528164"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "Job", "Goodyear", "Jobs" }}, "user": { "id": 128004921, "name": "CTCA Jobs", "screen_name": "CTCAJobs", "lang": "en", "location": "#AZ #GA #IL #OK #PA #FL", "create_at": date("2010-03-30"), "description": "Here you'll find our latest job opportunities across the U.S. Follow @CTCACareers to learn more & ask us questions! #jobs #careers #healthcare", "followers_count": 833, "friends_count": 484, "statues_count": 10778 }, "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("2015-11-24T10:00:18.000Z"), "id": 669213968406347776, "text": "I want this windbreaker from VC ���� https://t.co/FlZ2dqqzeQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 470555219, "name": "Mr.Høllywøød™", "screen_name": "Quinn__Fisher15", "lang": "en", "location": "Sandusky OH", "create_at": date("2012-01-21"), "description": "19 #NativeAmercain♓️ UpcomingFashionDesigner Fashion Is My Passion #Track SnapChat: iam_quin", "followers_count": 1572, "friends_count": 1851, "statues_count": 31974 }, "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("2015-11-24T10:00:18.000Z"), "id": 669213968410415106, "text": "Mixed signals.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 879284089, "name": "Savannah Jones", "screen_name": "savcx", "lang": "en", "location": "null", "create_at": date("2012-10-13"), "description": "•Live like there's no tomorrow• Fuck how it used to be. Keep your guard up, even the people you love can be heartless. ~ Seeing the light - Cody Johnson", "followers_count": 261, "friends_count": 146, "statues_count": 4515 }, "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("2015-11-24T10:00:18.000Z"), "id": 669213968448143361, "text": "Tornamesas listas para el mix de esta tarde at 12 pm Subeleeee @Latino1027 LIKE para un Saludo en Vivo Lunch Time Quick Mix Subeleeee!! ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 1431344666 }}, "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": 912, "friends_count": 1912, "statues_count": 3753 }, "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-24T10:00:18.000Z"), "id": 669213968490037248, "text": "should marco", "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": "jolly evan", "screen_name": "evantuaIIy", "lang": "en-gb", "location": "løs angeles", "create_at": date("2013-06-14"), "description": "j'suis un ananas", "followers_count": 2946, "friends_count": 172, "statues_count": 25714 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-24T10:00:18.000Z"), "id": 669213968515231744, "text": "Kevin Garnett ain't even better than Charles Barkley https://t.co/gL9ArHdPx5", "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": "Chum", "screen_name": "TheeGDawg", "lang": "en", "location": "Under₩ater, TX", "create_at": date("2013-10-08"), "description": "#WomenAreTheDevil شهيد من الملعونين", "followers_count": 1302, "friends_count": 901, "statues_count": 69420 }, "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("2015-11-24T10:00:18.000Z"), "id": 669213968603463681, "text": "Anyone wanna go shoping at the mall on Black Friday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1644552600, "name": "loud ass lasher", "screen_name": "rlasher107", "lang": "en", "location": "null", "create_at": date("2013-08-03"), "description": "hey snapchat rlasher13579", "followers_count": 152, "friends_count": 373, "statues_count": 972 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Auburn, NY", "id": "f8e6f4aa8e4393c0", "name": "Auburn", "place_type": "city", "bounding_box": rectangle("-76.614535,42.898722 -76.522688,42.997447") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36011, "countyName": "Cayuga", "cityID": 3603078, "cityName": "Auburn" } }
+{ "create_at": datetime("2015-11-24T10:00:18.000Z"), "id": 669213968666243076, "text": "@miss_burky @bayleeangel the final results are in", "in_reply_to_status": 668843587224928257, "in_reply_to_user": 2740801765, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2740801765, 19154241 }}, "user": { "id": 2740801765, "name": "shauna", "screen_name": "miss_burky", "lang": "es", "location": "chi • golden apple • dpu", "create_at": date("2014-08-17"), "description": "i love spinach pizza, educating the future of america, tea, josh groban, dogs, and greek yogurt boys • matthew 6:34", "followers_count": 235, "friends_count": 172, "statues_count": 7366 }, "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-24T10:00:18.000Z"), "id": 669213968804675584, "text": "So many shoes I want to buy ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 842210983, "name": "ferris", "screen_name": "500daysofallen", "lang": "en", "location": "null", "create_at": date("2012-09-23"), "description": "✨I'm the ghost with the most, babe.", "followers_count": 629, "friends_count": 108, "statues_count": 42332 }, "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("2015-11-24T10:00:18.000Z"), "id": 669213969052135425, "text": "I had a shitty lunch smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 224486625, "name": "kky", "screen_name": "Babyykwilliams_", "lang": "en", "location": "PNW☀️", "create_at": date("2010-12-08"), "description": "null", "followers_count": 1263, "friends_count": 524, "statues_count": 70024 }, "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("2015-11-24T10:00:18.000Z"), "id": 669213969064660993, "text": "#Repost @reloadpoolparty with repostapp\n・・・\nJoin us at the newest pool party on Saturdays,… https://t.co/48aJDDkKnH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.1341,25.813"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Repost" }}, "user_mentions": {{ 1598734952 }}, "user": { "id": 24911506, "name": "Dan Ribeiro", "screen_name": "Dannysobe", "lang": "en", "location": "South Beach", "create_at": date("2009-03-17"), "description": "Time traveler from a diferent galaxy", "followers_count": 807, "friends_count": 943, "statues_count": 4549 }, "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-24T10:00:18.000Z"), "id": 669213969207459840, "text": "From every single Walt Disney theme park, to every water park, to universal studios and the… https://t.co/Cbe7TTaFu3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.58111111,28.41861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 428680936, "name": "Shannon Farias", "screen_name": "ShannonFarias", "lang": "en", "location": "Canada", "create_at": date("2011-12-04"), "description": "It's not about the years in your life, it's about the life in your years | j.d❤️", "followers_count": 456, "friends_count": 389, "statues_count": 10378 }, "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("2015-11-24T10:00:18.000Z"), "id": 669213969358286849, "text": "Mostly cloudy this afternoon, high 71 (22 C). Low 60 (16 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712388, "name": "HoustonWX", "screen_name": "HoustonWX", "lang": "en", "location": "Houston, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Houston, Texas", "followers_count": 1737, "friends_count": 77, "statues_count": 7904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:18.000Z"), "id": 669213969526099969, "text": "my mom always talks shit in german and im pretty sure her and my gma are talking about their disappointment in 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": 1146700129, "name": "BAY", "screen_name": "bayleedochwat", "lang": "en", "location": "808 | 618", "create_at": date("2013-02-03"), "description": "null", "followers_count": 447, "friends_count": 389, "statues_count": 10592 }, "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("2015-11-24T10:00:18.000Z"), "id": 669213969580601344, "text": "God.. I need your help. 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": 452728487, "name": "CodyyBrower™", "screen_name": "CodyyyWilliam__", "lang": "en", "location": "null", "create_at": date("2012-01-01"), "description": "null", "followers_count": 507, "friends_count": 302, "statues_count": 5560 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Senatobia, MS", "id": "31fb0bb7a127a0f3", "name": "Senatobia", "place_type": "city", "bounding_box": rectangle("-90.003347,34.595668 -89.949621,34.641424") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28137, "countyName": "Tate", "cityID": 2866440, "cityName": "Senatobia" } }
+{ "create_at": datetime("2015-11-24T10:00:18.000Z"), "id": 669213969610055680, "text": "I LOVE GETTING COMPLIMENTS ON MY BROWS ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 620445312, "name": "♡paytyntusay♡", "screen_name": "paytusay", "lang": "en", "location": "null", "create_at": date("2012-06-27"), "description": "take a journey with me. festifam. #ripblake #PLUR c/o '16", "followers_count": 837, "friends_count": 622, "statues_count": 29192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Belton, MO", "id": "7235125f54ba7b87", "name": "Belton", "place_type": "city", "bounding_box": rectangle("-94.567317,38.790672 -94.490023,38.845062") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29037, "countyName": "Cass", "cityID": 2904384, "cityName": "Belton" } }
+{ "create_at": datetime("2015-11-24T10:00:18.000Z"), "id": 669213970667057152, "text": "@Twerkin_Perkins cause the pic is on the phone 6s at Best Buy lol", "in_reply_to_status": 669211689926025216, "in_reply_to_user": 396328639, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 396328639 }}, "user": { "id": 388461530, "name": "Abbey", "screen_name": "abbeyy_", "lang": "en", "location": "null", "create_at": date("2011-10-10"), "description": "rip Nathan. ♡", "followers_count": 642, "friends_count": 151, "statues_count": 10375 }, "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("2015-11-24T10:00:18.000Z"), "id": 669213971061325825, "text": "@brandon305beast shut up", "in_reply_to_status": 669209861473873920, "in_reply_to_user": 2376417682, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2376417682 }}, "user": { "id": 532707352, "name": "Luna", "screen_name": "lunaxgarcia", "lang": "en", "location": "Hogwarts", "create_at": date("2012-03-21"), "description": "I solemnly swear that I am up to no good", "followers_count": 699, "friends_count": 326, "statues_count": 51200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glenvar Heights, FL", "id": "f546db03397f6e09", "name": "Glenvar Heights", "place_type": "city", "bounding_box": rectangle("-80.334426,25.690519 -80.292923,25.733912") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1226100, "cityName": "Glenvar Heights" } }
+{ "create_at": datetime("2015-11-24T10:00:18.000Z"), "id": 669213971069669376, "text": "@jewpunk Exactly-- now it's out of your hands. Be free, young book! Be good and be bad! Behave and misbehave! See what happens! #grubinc", "in_reply_to_status": 669212951182749696, "in_reply_to_user": 14582914, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "grubinc" }}, "user_mentions": {{ 14582914 }}, "user": { "id": 118915803, "name": "Sonya Larson", "screen_name": "SonyaLarson", "lang": "en", "location": "Boston, MA", "create_at": date("2010-03-01"), "description": "Program Director of Grub Street. Writer. Break-dancer? Someday.", "followers_count": 1379, "friends_count": 884, "statues_count": 1130 }, "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("2015-11-24T10:00:18.000Z"), "id": 669213971170377729, "text": "@lybr3 @meldoyle9 @kimmieguy I'm not speaking for Christ. I'm just telling you what his word says.", "in_reply_to_status": 669210793695514625, "in_reply_to_user": 601076863, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 601076863, 73644811, 107033885 }}, "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": 20306, "friends_count": 16940, "statues_count": 85461 }, "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("2015-11-24T10:00:18.000Z"), "id": 669213971661070341, "text": "I'm at Entera Healthcare in Skokie, IL https://t.co/mgBBH4Z6We", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.71294,42.033349"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1069909158, "name": "Niknik", "screen_name": "nikkapatriciaa", "lang": "en", "location": "Chicago, IL", "create_at": date("2013-01-07"), "description": "Kathleen's ❣", "followers_count": 775, "friends_count": 640, "statues_count": 16900 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213972256546817, "text": "LCS!! We're going on a ski trip to Sulphur Friday since there's s(NO)w stopping us!! Wear all your ski gear!!! ☃❄️����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1835438480, "name": "Jillian Harpe", "screen_name": "jillianharpe", "lang": "en", "location": "null", "create_at": date("2013-09-08"), "description": "My outfit says preppy white girl but my playlist says emotional hoodrat", "followers_count": 785, "friends_count": 475, "statues_count": 16596 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213972403326976, "text": "Partly cloudy/wind this afternoon, high 67 (19 C). Low 54 (12 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575081, "name": "San Angelo Weather", "screen_name": "SanAngeloWX", "lang": "en", "location": "San Angelo, TX", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for San Angelo, TX", "followers_count": 1916, "friends_count": 92, "statues_count": 7794 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213972424294400, "text": "tonight is gonna be so great", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2386151006, "name": "lys", "screen_name": "awyssaab", "lang": "en", "location": "null", "create_at": date("2014-03-12"), "description": "it's all temporary", "followers_count": 376, "friends_count": 209, "statues_count": 1835 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213972453842944, "text": "buffalo chicken dip makes me o 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": 2153385855, "name": "Bri cabon", "screen_name": "cabonbri", "lang": "en", "location": "null", "create_at": date("2013-10-25"), "description": "insta: bricabon ; I like chipotle & coffee & I'm sometimes funny", "followers_count": 1095, "friends_count": 643, "statues_count": 4170 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tarpon Springs, FL", "id": "4ad609c5e10fc033", "name": "Tarpon Springs", "place_type": "city", "bounding_box": rectangle("-82.806995,28.118339 -82.718876,28.173326") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1271150, "cityName": "Tarpon Springs" } }
+{ "create_at": datetime("2015-11-24T10:00:19.000Z"), "id": 669213972604678145, "text": "We're #hiring! Read about our latest #job opening here: A&P Mechanics - https://t.co/28RAiA3H7C #Honolulu, HI #Engineering #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8689014,21.3644294"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Honolulu", "Engineering", "CareerArc" }}, "user": { "id": 22777294, "name": "TMJ-HON Labor Jobs", "screen_name": "tmj_hon_labor", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-04"), "description": "Follow this account for geo-targeted General Labor job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 299, "friends_count": 291, "statues_count": 3 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213972650815488, "text": "thank you so much, love you, beautiful!❤️ https://t.co/19dZzOq8N0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2590240164, "name": "ᴘɴᴍ", "screen_name": "paytonmoreaux", "lang": "en", "location": "null", "create_at": date("2014-06-26"), "description": "mi vida locaaaaaa", "followers_count": 557, "friends_count": 1011, "statues_count": 7610 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213972701163521, "text": "Smoking just to fit in smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2898842987, "name": "Javin Bowman", "screen_name": "__LegendaryOne", "lang": "en", "location": "Leisuretowne, NJ", "create_at": date("2014-11-29"), "description": "Live and Maintain #2RANGZ | RCBC'17", "followers_count": 760, "friends_count": 673, "statues_count": 5811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Leisuretowne, NJ", "id": "338e3ac8301cb2f1", "name": "Leisuretowne", "place_type": "city", "bounding_box": rectangle("-74.722109,39.880684 -74.679419,39.925738") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34005, "countyName": "Burlington", "cityID": 3439885, "cityName": "Leisuretowne" } }
+{ "create_at": datetime("2015-11-24T10:00:19.000Z"), "id": 669213972772610048, "text": "Glory to God! All nations...Russia needs everyone's help to fight Islamic terrorist in Syria! Send your warplanes to Syria...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.99460556,34.58878543"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 607011873, "name": "tammy glass", "screen_name": "tamilou007", "lang": "en", "location": "null", "create_at": date("2012-06-12"), "description": "null", "followers_count": 20, "friends_count": 2, "statues_count": 4707 }, "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": 1071, "countyName": "Jackson", "cityID": 168736, "cityName": "Scottsboro" } }
+{ "create_at": datetime("2015-11-24T10:00:19.000Z"), "id": 669213972860661760, "text": "@sierrasemko I NEED TO SEE YOU", "in_reply_to_status": 669213330675007489, "in_reply_to_user": 559118065, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 559118065 }}, "user": { "id": 1561900056, "name": "Madison", "screen_name": "mj_reinhard", "lang": "en", "location": "null", "create_at": date("2013-07-01"), "description": "lovers gon love", "followers_count": 443, "friends_count": 369, "statues_count": 4152 }, "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-24T10:00:19.000Z"), "id": 669213972890038272, "text": "How can I sit through this class rn��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 232253441, "name": "SHEA BUTTA✨", "screen_name": "ShesiAzeez", "lang": "en", "location": "Jersey", "create_at": date("2010-12-30"), "description": "FUTURE ARCHITECT | NAIJA BABY | NJIT | #FORFEIT ig:yungetnoir", "followers_count": 398, "friends_count": 582, "statues_count": 9893 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213973011542016, "text": "@naybaybaay basically lol ����", "in_reply_to_status": 669213672326062080, "in_reply_to_user": 356373534, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 356373534 }}, "user": { "id": 391786399, "name": "Emily Dittel", "screen_name": "editz69", "lang": "en", "location": "minnsnowta❄️, IGH ~ Bemidji ", "create_at": date("2011-10-15"), "description": "| Instagram ~ Emily_Dittel | | Snapchat ~ edittel |", "followers_count": 791, "friends_count": 440, "statues_count": 6562 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213973212823552, "text": "@moniboyce I could use more sleep, but actually my day is going really well Thank u for asking Moni. @JulieKamins You see. I can be nice :)", "in_reply_to_status": 669213123992264704, "in_reply_to_user": 110631887, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 110631887, 2383938876 }}, "user": { "id": 74285874, "name": "Ceasar V. Maximus™", "screen_name": "cyberceas", "lang": "en", "location": "Beverly Hills ✪ Hollywood Ca.", "create_at": date("2009-09-14"), "description": "Living Life in the Fastlane, Workaholic.. I Represent Clients in the ''Entertainment Business'' and I only Like Drinking Good Scotch.. Sorry, but I don't DM.", "followers_count": 71217, "friends_count": 69049, "statues_count": 93598 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213973275783168, "text": "I find it very nasty smoking un- freaked milds", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2964563806, "name": "$ELFMADE", "screen_name": "B__Hunnit", "lang": "en", "location": "Da Heights ", "create_at": date("2015-01-06"), "description": "Young nigga just out here tryna get it , #Employed ,# 18 years young and living", "followers_count": 418, "friends_count": 402, "statues_count": 2825 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213973460422656, "text": "My first big project at @classpass just went live: gifting memberships to users. Server-side rendered React + Redux. We're also hiring!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 320898935 }}, "user": { "id": 15377385, "name": "Thomas", "screen_name": "rx", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2008-07-10"), "description": "engineer @classpass / formerly @vine @twitter @zappos @trulia / gpg C27504E07119F00D", "followers_count": 2649, "friends_count": 349, "statues_count": 20972 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midtown South, Manhattan", "id": "262eae79bad8d600", "name": "Midtown South", "place_type": "neighborhood", "bounding_box": rectangle("-73.997987,40.741419 -73.979726,40.752873") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2015-11-24T10:00:19.000Z"), "id": 669213973783293952, "text": "We're #hiring! Click to apply: Truck Driver - Hiker/ Vehicle Transporter/CDL - https://t.co/20Ql1bM9uD #gopenskecareers #Montebello, CA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.12469,33.983793"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "gopenskecareers", "Montebello" }}, "user": { "id": 23475616, "name": "TMJ-LAX Transp. Jobs", "screen_name": "tmj_lax_transp", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Transportation job tweets in Los Angeles, CA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 391, "friends_count": 295, "statues_count": 176 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213973842104320, "text": "@Ljusha7 @62Roses @Ibnelkayeem @loyal_cupid @drouchemed58 @ranasilvia63 @Asian_Poetry @zbmkhan @GCHDutta @almohid https://t.co/hdCVaiIcv1", "in_reply_to_status": 669213884583161856, "in_reply_to_user": 48875310, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2965652843, 1409420528, 2963445490, 17308392, 2264139201, 2750799059, 1256913534, 3010331497, 1721848128, 140078994 }}, "user": { "id": 48875310, "name": "Alhaj Nasir Uddin", "screen_name": "MrNaassir", "lang": "en", "location": "New Yark City.New York. U.S.A.", "create_at": date("2009-06-19"), "description": "I am from Pakistan & U.S. Citizen. Love Photography, Flowers, Birds, Scenery & Nature.Please Type MrNaassir in Google & See my 16000 pictures Thanks. NO DM PLZ.", "followers_count": 5431, "friends_count": 2345, "statues_count": 134116 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213974005698560, "text": "No more stitches 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": 75380306, "name": "Carolina Diegues", "screen_name": "carolinadiegues", "lang": "en", "location": "null", "create_at": date("2009-09-18"), "description": "DSC Volleyball", "followers_count": 186, "friends_count": 172, "statues_count": 7538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jupiter Farms, FL", "id": "0054e9b90d4f8402", "name": "Jupiter Farms", "place_type": "city", "bounding_box": rectangle("-80.256457,26.896996 -80.172692,26.957247") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1235890, "cityName": "Jupiter Farms" } }
+{ "create_at": datetime("2015-11-24T10:00:19.000Z"), "id": 669213974131535872, "text": ".@Mayhem said I have to tweet this for my vote to count in the #MayhemTweetOff. Go FSU! #MeltFlorida https://t.co/jsph4fFvow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MayhemTweetOff", "MeltFlorida" }}, "user_mentions": {{ 1858823539 }}, "user": { "id": 1698022615, "name": "Rice", "screen_name": "rausch_bryce", "lang": "en", "location": "null", "create_at": date("2013-08-24"), "description": "I'm the kicker, JBHS, Massive⚽", "followers_count": 174, "friends_count": 127, "statues_count": 1789 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213974399856645, "text": "@PrincessDi1982 @joeylawrence @Monika_Kamila @JohnnyRemo1 yeah I'm thinking the trailer!! But the song would be awesome!!", "in_reply_to_status": 669213202589204481, "in_reply_to_user": 81763584, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 81763584, 145852539, 315555817, 357620167 }}, "user": { "id": 2948382268, "name": "Melissa&JoeyFan!!", "screen_name": "albritton_leisa", "lang": "en", "location": "null", "create_at": date("2014-12-28"), "description": "Mom to Courtney 20 yr &Joe13 yr old. Long time fan of Joeys since I was 16,for3 decadesJoeysAngel 4ever also a big fan Andy and Matt,Support #still3 Always", "followers_count": 651, "friends_count": 904, "statues_count": 21531 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213974429208577, "text": "you're the definition of a home wrecker.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 450837671, "name": "karina lizeth", "screen_name": "Liz_Medina13", "lang": "en", "location": "chickasha, ok", "create_at": date("2011-12-30"), "description": "ig: karina_lizeth || sc: karina-lizeth", "followers_count": 1361, "friends_count": 757, "statues_count": 23302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chickasha, OK", "id": "ba083f1f28634b26", "name": "Chickasha", "place_type": "city", "bounding_box": rectangle("-97.979369,34.992737 -97.917193,35.07315") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40051, "countyName": "Grady", "cityID": 4013950, "cityName": "Chickasha" } }
+{ "create_at": datetime("2015-11-24T10:00:19.000Z"), "id": 669213974492155904, "text": "So excited for my sissy date tonight with @Megaan_Whitee ������ I haven't seen you 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_mentions": {{ 412645654 }}, "user": { "id": 44003392, "name": "Taylor Nicole White", "screen_name": "taytaynicole3", "lang": "en", "location": "Nacogdoches, Texas", "create_at": date("2009-06-01"), "description": "SFASU '15. ∞", "followers_count": 546, "friends_count": 532, "statues_count": 8181 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213974546681856, "text": "شلون اعرف منو مو ضايفني بسناب؟", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 3276698054, "name": "عويد.", "screen_name": "3lawAl3waid", "lang": "en", "location": "null", "create_at": date("2015-07-11"), "description": "وفخري لآل هاشم انتمي. بأمانة موسى ابن جعفر. #HalaMadrid", "followers_count": 1501, "friends_count": 414, "statues_count": 34904 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213974919966720, "text": "Dear Marley, you can have this puppies any time now.������ #sickofwaiting", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sickofwaiting" }}, "user": { "id": 798371370, "name": "Madison Henderson♛", "screen_name": "madisonelisee5", "lang": "en", "location": "SC", "create_at": date("2012-09-02"), "description": "My reputation follows me around, just makes me wanna give'em more to talk about. ♛", "followers_count": 217, "friends_count": 153, "statues_count": 2199 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213974982885377, "text": "@jayyy_loo \"pinche puto baboso\" HAHAHAHAHAHAHAHAHA omg����", "in_reply_to_status": 669200335282397184, "in_reply_to_user": 1661880390, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user_mentions": {{ 1661880390 }}, "user": { "id": 1667130038, "name": "Leonor✨", "screen_name": "leonororozco2", "lang": "en", "location": "Corvallis, OR", "create_at": date("2013-08-13"), "description": "oh hi", "followers_count": 447, "friends_count": 347, "statues_count": 17779 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213975318536193, "text": "@D_Lo122 I'm fascinated that you call something with overwhelming evidence behind it \"ridiculous\" & prefer something with no evidence at all", "in_reply_to_status": 669160985241001985, "in_reply_to_user": 1737598206, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1737598206 }}, "user": { "id": 190596208, "name": "Martyn Cornell", "screen_name": "zythophiliac", "lang": "en", "location": "London", "create_at": date("2010-09-14"), "description": "Author, journalist, beer educator, beer consultant, four-times winner BGBW awards 2011-14, 4th-generation QPR fan", "followers_count": 2787, "friends_count": 1314, "statues_count": 5047 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213975444193280, "text": "I would appreciate a Marshawn Lynch update today please and thank you.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 153268960, "name": "Jaime Helm", "screen_name": "JaimeHelm", "lang": "en", "location": "Nebraska", "create_at": date("2010-06-07"), "description": "null", "followers_count": 31, "friends_count": 71, "statues_count": 2607 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McCook, NE", "id": "28020ccd5c65d616", "name": "McCook", "place_type": "city", "bounding_box": rectangle("-100.66611,40.185164 -100.577813,40.224795") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31145, "countyName": "Red Willow", "cityID": 3129925, "cityName": "McCook" } }
+{ "create_at": datetime("2015-11-24T10:00:19.000Z"), "id": 669213975486201856, "text": "Did Kool Keith drop the best album cover ever? https://t.co/KbOf0ynDZ3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29034144, "name": "malkovich [#1BAG]", "screen_name": "malkovichmusic", "lang": "en", "location": "#1BAG since 1/1/12", "create_at": date("2009-04-05"), "description": "google 'traveling rapper'. Download PREBOARDING w/ @djspinna, a @gillespeterson premiere free http://malkovichmusic.bandcamp.com. Got beats? I'll fly to you.", "followers_count": 31327, "friends_count": 15593, "statues_count": 15266 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-24T10:00:19.000Z"), "id": 669213975679123456, "text": "@PABLOalwaysicy https://t.co/QmQeHZMZh7", "in_reply_to_status": -1, "in_reply_to_user": 3033410197, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3033410197 }}, "user": { "id": 91645916, "name": "micsss", "screen_name": "mjstromain", "lang": "en", "location": "null", "create_at": date("2009-11-21"), "description": "Yung Pocahontas", "followers_count": 556, "friends_count": 490, "statues_count": 6979 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213975716872193, "text": "Help Desk Analyst I - Robert Half Technology: (#OklahomaCity, OK) https://t.co/HIR4hxrfpV #CustomerService #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.6385368,35.6131551"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "OklahomaCity", "CustomerService", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 71822093, "name": "TMJ-OKC CstSrv Jobs", "screen_name": "tmj_okc_cstsrv", "lang": "en", "location": "Oklahoma City, OK", "create_at": date("2009-09-05"), "description": "Follow this account for geo-targeted Customer Service job tweets in Oklahoma City, OK from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 297, "friends_count": 278, "statues_count": 148 }, "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": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2015-11-24T10:00:19.000Z"), "id": 669213975737966595, "text": "Lunch date with these pretty ladies. @ Logan's Roadhouse https://t.co/gSENKJd6IB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.52995424,37.98524277"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 281185929, "name": "Jason Hurst", "screen_name": "jasonhurst25", "lang": "en", "location": "nicholasville,ky", "create_at": date("2011-04-12"), "description": "Huge sports fan. #KentuckyWildcats #BBN @Reds #RedsTwitterFam @Bengals Hate Cardinals...Louisville and St. Louis", "followers_count": 1896, "friends_count": 1281, "statues_count": 18469 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213975800754176, "text": "@tapbots I use (and love) Tweetbot for Mac and iOS, but mute filters aren’t syncing over iCloud. Any tips? Tried rebooting devices.", "in_reply_to_status": -1, "in_reply_to_user": 16669898, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16669898 }}, "user": { "id": 18594810, "name": "Dylan Vassallo", "screen_name": "dylanvee", "lang": "en", "location": "Berkeley, CA", "create_at": date("2009-01-03"), "description": "Bay Area native. Devtools @Stripe, previously @KhanAcademy, previously @UCLA.", "followers_count": 787, "friends_count": 766, "statues_count": 7835 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213975809126400, "text": "Somebody gone appreciate me��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2374568683, "name": "Jeremi Hanks", "screen_name": "count__bandz", "lang": "en", "location": "Oakland CA", "create_at": date("2014-03-05"), "description": "Dreams to reality. Cant wait until the day i can tell my mom we made it.", "followers_count": 1036, "friends_count": 609, "statues_count": 5954 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213975914143744, "text": "Construction on #SawMillRiverParkway NB from Exit 21E - Frontage Road to Exit 23 - Old Saw Mill Road; CR 303 https://t.co/C2cSX0N3LF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.822402,41.053199"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SawMillRiverParkway" }}, "user": { "id": 50728535, "name": "511NY - Catskills", "screen_name": "511nyCatskills", "lang": "en", "location": "Catskills", "create_at": date("2009-06-25"), "description": "Traffic & transit updates for the Hudson Valley Catskill area provided by New York State 511. Visit the website for more feeds.", "followers_count": 600, "friends_count": 0, "statues_count": 265780 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elmsford, NY", "id": "7f7aef8756a20abd", "name": "Elmsford", "place_type": "city", "bounding_box": rectangle("-73.832145,41.045458 -73.801503,41.06099") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3624295, "cityName": "Elmsford" } }
+{ "create_at": datetime("2015-11-24T10:00:19.000Z"), "id": 669213975955898369, "text": "Mostly sunny this afternoon, high 65 (18 C). Low 48 (9 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575147, "name": "Baton Rouge Weather", "screen_name": "BatonRougeWX", "lang": "en", "location": "Baton Rouge, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Baton Rouge, LA", "followers_count": 2858, "friends_count": 92, "statues_count": 7747 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213975989497856, "text": "Is it weird that I know I won't get into SDSU, but when I get the letter saying it, I'll still cry?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1646199823, "name": "k-weezy", "screen_name": "kellerrrr_1", "lang": "en", "location": "Patterson, CA", "create_at": date("2013-08-04"), "description": "forever annoying Jordan Canseco #RunningOnE", "followers_count": 405, "friends_count": 307, "statues_count": 5262 }, "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("2015-11-24T10:00:19.000Z"), "id": 669213976245493761, "text": "@foxnews #syria LETS TLK HISTORY, islam hs seized brder aftr brder aftr border for 1250yrs,seizd ESTRN CHRST CAP russa hs fght thm 900yrs", "in_reply_to_status": -1, "in_reply_to_user": 1367531, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "syria" }}, "user_mentions": {{ 1367531 }}, "user": { "id": 700100198, "name": "markodum", "screen_name": "MarkOdum", "lang": "en", "location": "Dayton Ohio home of WPAFB", "create_at": date("2012-07-16"), "description": "Reaganite, pro US CONSTITUTIONALIST, blk irish,HISTORY MJR,retired, former GOP POLL captain,GAY,Catholic,relative of GEN GRG C MARSHALL,Vol. for VFW Charities", "followers_count": 1213, "friends_count": 1288, "statues_count": 17052 }, "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("2015-11-24T10:00:20.000Z"), "id": 669213976472002560, "text": "@HotChickLover1 @emmagreenmodel @FitAsFuckGirls @kandikayfan01 @18_HOT_18 good looking body and big tits", "in_reply_to_status": 669100230491684866, "in_reply_to_user": 1956870254, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1956870254, 233952512, 845154482, 2472123963, 1321275720 }}, "user": { "id": 3537267083, "name": "pimpdaddy", "screen_name": "pimpdaddy15301", "lang": "en", "location": "null", "create_at": date("2015-09-03"), "description": "null", "followers_count": 27, "friends_count": 84, "statues_count": 977 }, "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("2015-11-24T10:00:20.000Z"), "id": 669213976891297796, "text": "Hot coco sound so 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": 327015376, "name": "⚡ Bonnison! ⚡", "screen_name": "LiwangSwagg", "lang": "en", "location": "Richland, WA", "create_at": date("2011-06-30"), "description": "RHS alumni ✌", "followers_count": 166, "friends_count": 108, "statues_count": 4249 }, "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("2015-11-24T10:00:20.000Z"), "id": 669213977126289408, "text": "they all looked soo beautiful 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": 2194988149, "name": "ash", "screen_name": "MlNSYOONGI", "lang": "en", "location": "sylvia • 1/24/15", "create_at": date("2013-11-14"), "description": "whats the length of my toenail? i mean im just saying https://twitter.com/shamelessmisha/status/592808815625633793", "followers_count": 1617, "friends_count": 92, "statues_count": 100710 }, "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("2015-11-24T10:00:20.000Z"), "id": 669213977331630080, "text": "Where �� https://t.co/QobwfC2oTn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 292071105, "name": "SativaDiva✨", "screen_name": "NeiceyCutthoat", "lang": "en", "location": "where The hood meets Laos", "create_at": date("2011-05-02"), "description": "RIP Elliot and My mom Mouk Teso", "followers_count": 1229, "friends_count": 420, "statues_count": 77319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentfield, CA", "id": "15c2aaaadb3f42c2", "name": "Kentfield", "place_type": "city", "bounding_box": rectangle("-122.571067,37.926542 -122.518284,37.963882") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6041, "countyName": "Marin", "cityID": 638114, "cityName": "Kentfield" } }
+{ "create_at": datetime("2015-11-24T10:00:20.000Z"), "id": 669213977424064512, "text": "#shopsmall for Nellie! This vibe is awesome and has a fun voice/music activated function! At only $99… https://t.co/WFMCNMZbae", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.65514869,41.97828635"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "shopsmall" }}, "user": { "id": 18775318, "name": "Early to Bed", "screen_name": "Earlytobed", "lang": "en", "location": "5044 N. Clark St., Chicago, IL", "create_at": date("2009-01-08"), "description": "ETB is Chicago's Feminist Sex shop. We heart sex toys, honest sex ed and orgasms for everyone!!", "followers_count": 6535, "friends_count": 1632, "statues_count": 11088 }, "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-24T10:00:20.000Z"), "id": 669213977428123648, "text": "@Leahxole_ lmfao I feel you! Different strokes for different folks. Where you been at though? I haven't heard from you in forever ��", "in_reply_to_status": 669213576247250944, "in_reply_to_user": 1470562381, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1470562381 }}, "user": { "id": 146820523, "name": "Robert Smith", "screen_name": "RobertLee__", "lang": "en", "location": "Navy ⚓ ", "create_at": date("2010-05-22"), "description": "Livonia", "followers_count": 1153, "friends_count": 639, "statues_count": 39644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Chicago, IL", "id": "772f1cfac1076a67", "name": "North Chicago", "place_type": "city", "bounding_box": rectangle("-87.903832,42.2877 -87.825943,42.341453") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1753559, "cityName": "North Chicago" } }
+{ "create_at": datetime("2015-11-24T10:00:20.000Z"), "id": 669213977608491008, "text": "We're #hiring! Read about our latest #job opening here: Restaurant Team Member (Crew) - https://t.co/KqT3a3SpvZ #SONIC #SanAntonio, TX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.4688555,29.6113113"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "SONIC", "SanAntonio" }}, "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": 419, "friends_count": 293, "statues_count": 745 }, "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-24T10:00:20.000Z"), "id": 669213977826627584, "text": "why does everyone work today im lonely af", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 212760453, "name": "kev", "screen_name": "traankevin", "lang": "en", "location": "Dallas, TX", "create_at": date("2010-11-06"), "description": "cooling it w my dog", "followers_count": 264, "friends_count": 221, "statues_count": 7668 }, "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("2015-11-24T10:00:20.000Z"), "id": 669213977860177920, "text": "My mom and dad already getting on my nerves ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 339210839, "name": ":)", "screen_name": "raeforshort_", "lang": "en", "location": "null", "create_at": date("2011-07-20"), "description": "fmoi: __snd sc: pimprae UAM19'", "followers_count": 1177, "friends_count": 764, "statues_count": 39013 }, "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("2015-11-24T10:00:20.000Z"), "id": 669213978179067904, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.45179739,40.73494327"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 340052093, "name": "emily miccio", "screen_name": "m1cc1o", "lang": "en", "location": "null", "create_at": date("2011-07-21"), "description": "long island✖️penn state✖️zeta tau alpha", "followers_count": 946, "friends_count": 840, "statues_count": 20365 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Farmingdale, NY", "id": "80fb5c3be0583f42", "name": "Farmingdale", "place_type": "city", "bounding_box": rectangle("-73.463143,40.723554 -73.433217,40.740523") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3625384, "cityName": "Farmingdale" } }
+{ "create_at": datetime("2015-11-24T10:00:20.000Z"), "id": 669213978204205057, "text": "Our Poetry Pros for this week! #practicingfluency #onceacard https://t.co/lNOqJimAlF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "practicingfluency", "onceacard" }}, "user": { "id": 2334025104, "name": "Cheri Posner", "screen_name": "BellflowerMrsP", "lang": "en", "location": "null", "create_at": date("2014-02-08"), "description": "Second grade teacher at Bellflower Elementary", "followers_count": 125, "friends_count": 114, "statues_count": 172 }, "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("2015-11-24T10:00:20.000Z"), "id": 669213978615132161, "text": "#Marketing alert: Coordinator, Education & Communication - Sex Abuse... | Hawaii Pacific Health | #Honolulu, HI https://t.co/QSvx3qHnnU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8480364,21.3136151"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Marketing", "Honolulu" }}, "user": { "id": 23493115, "name": "TMJ-HON Advert Jobs", "screen_name": "tmj_hon_adv", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Marketing/Ad/PR job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 288, "friends_count": 193, "statues_count": 9 }, "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("2015-11-24T10:00:20.000Z"), "id": 669213978833260544, "text": "@StellyINDACUT @markysush we coming soon @4V3RYM get ya ladies ready fellas. ��", "in_reply_to_status": 669213637089689600, "in_reply_to_user": 1887459049, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1887459049, 2462465526, 3019180986 }}, "user": { "id": 60143443, "name": "Lil Curl", "screen_name": "sharkjacobs", "lang": "en", "location": "Austin, TX", "create_at": date("2009-07-25"), "description": "throwin shit - 12/12 #AllWhite⛸", "followers_count": 2104, "friends_count": 665, "statues_count": 53876 }, "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-24T10:00:20.000Z"), "id": 669213978879365121, "text": "mute? why yes please", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2559587365, "name": "Cassandra", "screen_name": "kassonxo", "lang": "en", "location": "null", "create_at": date("2014-06-10"), "description": "null", "followers_count": 33, "friends_count": 146, "statues_count": 825 }, "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("2015-11-24T10:00:20.000Z"), "id": 669213979202351104, "text": "Not in a working mood @ all", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 408004223, "name": "❌⭕suenito ⭕❌", "screen_name": "suenito23", "lang": "en", "location": "San Francisco, CA ", "create_at": date("2011-11-08"), "description": "|peter 5:7| im a dreamer believer all my faith is in god.... im getting better cause of my mistakes #teamtravel #grinddayandnight", "followers_count": 5454, "friends_count": 5102, "statues_count": 34216 }, "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("2015-11-24T10:00:20.000Z"), "id": 669213979240075264, "text": "Partly cloudy this afternoon, high 63 (17 C). Low 57 (14 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575204, "name": "New Orleans Weather", "screen_name": "NewOrleansWX", "lang": "en", "location": "New Orleans, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for New Orleans, LA", "followers_count": 2896, "friends_count": 92, "statues_count": 7765 }, "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("2015-11-24T10:00:20.000Z"), "id": 669213979269603328, "text": "Wow. https://t.co/cHyiDQNapg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 218021474, "name": "bb.", "screen_name": "XXVIIMCMXCI", "lang": "en", "location": "Lower East Side, Manhattan", "create_at": date("2010-11-20"), "description": "a fashion design major from Cincinnati doing stuff.", "followers_count": 1558, "friends_count": 873, "statues_count": 90939 }, "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("2015-11-24T10:00:20.000Z"), "id": 669213979324104705, "text": "bridgrt canceled https://t.co/OOCfFcfm3X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2550753984, "name": "bridget", "screen_name": "sivanthony", "lang": "en", "location": "null", "create_at": date("2014-06-06"), "description": "@LOHANTHONY: @sivanthony love u sexy beast!", "followers_count": 2633, "friends_count": 72, "statues_count": 40161 }, "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("2015-11-24T10:00:20.000Z"), "id": 669213979588317184, "text": "@harvardintech @anndreamoore any idea if this will be streamed?", "in_reply_to_status": 669195461509562368, "in_reply_to_user": 1975695888, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1975695888, 158612828 }}, "user": { "id": 552256998, "name": "Julius Bryant", "screen_name": "JuliusBryant", "lang": "en", "location": "Atlanta, GA by way of Michigan", "create_at": date("2012-04-12"), "description": "A digital junkie & photographer who loves music, marketing, sports, tech, & all things creative. @EmoryGoizueta '17 MBA candidate. @CMUniversity Alum @KAPsi1911", "followers_count": 465, "friends_count": 440, "statues_count": 4022 }, "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("2015-11-24T10:00:20.000Z"), "id": 669213980347510785, "text": "@Wind2Energy nah!! Don't need prayer.. But you fall on your knees and beg God for a MORON..", "in_reply_to_status": 669213312987500544, "in_reply_to_user": 14819311, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14819311 }}, "user": { "id": 345055220, "name": "@Truth", "screen_name": "kbuhurico", "lang": "en", "location": "null", "create_at": date("2011-07-29"), "description": "The truth will set you free! oh! the uneducated people worry me!", "followers_count": 30, "friends_count": 206, "statues_count": 3577 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ogden, NC", "id": "1382449ecd45a6ac", "name": "Ogden", "place_type": "city", "bounding_box": rectangle("-77.826612,34.248476 -77.770381,34.280072") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37129, "countyName": "New Hanover", "cityID": 3748790, "cityName": "Ogden" } }
+{ "create_at": datetime("2015-11-24T10:00:20.000Z"), "id": 669213980456562689, "text": "@mattkatz00 @ChrisChristie @POTUS @nprpolitics Someone may need a hearing and listening comprehension exam...just sayin'", "in_reply_to_status": 669212352374571008, "in_reply_to_user": 8011072, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 8011072, 1347285918, 1536791610, 5741722 }}, "user": { "id": 14536047, "name": "Kevin K.", "screen_name": "CNJBrownBear", "lang": "en", "location": "Metuchen, New Jersey", "create_at": date("2008-04-25"), "description": "I'm a 51 y/o Wiccan, Leather, Biker, Bear, Out, Gay Black Man who's an RN, and happily married to his husband. I am also a huge Mac/Apple Fan!", "followers_count": 1148, "friends_count": 1994, "statues_count": 14618 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Metuchen, NJ", "id": "64be9b322be0df48", "name": "Metuchen", "place_type": "city", "bounding_box": rectangle("-74.381081,40.527788 -74.344981,40.555373") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3445690, "cityName": "Metuchen" } }
+{ "create_at": datetime("2015-11-24T10:00:21.000Z"), "id": 669213980594933760, "text": "Like really ���� lol then he's like I only asked cuz you look like a doll", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 411915399, "name": "Jœy", "screen_name": "Perfect_Italian", "lang": "en", "location": "null", "create_at": date("2011-11-13"), "description": "♋️", "followers_count": 406, "friends_count": 518, "statues_count": 31143 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cranston, RI", "id": "4015afc69a8a8506", "name": "Cranston", "place_type": "city", "bounding_box": rectangle("-71.550463,41.730422 -71.377473,41.807454") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4419180, "cityName": "Cranston" } }
+{ "create_at": datetime("2015-11-24T10:00:21.000Z"), "id": 669213980603187200, "text": "@tneums2385 Friday brotha. Huskers wills reck", "in_reply_to_status": 669213816631115776, "in_reply_to_user": 326352251, "favorite_count": 0, "coordinate": point("-93.6102636,42.02080665"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 326352251 }}, "user": { "id": 495405559, "name": "Bader", "screen_name": "masterBader64", "lang": "en", "location": "Moville/Ames, Iowa", "create_at": date("2012-02-17"), "description": "i am fast. to give you a reference point somewhere between a snake and a mongoose.", "followers_count": 593, "friends_count": 386, "statues_count": 6769 }, "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("2015-11-24T10:00:21.000Z"), "id": 669213980657741824, "text": "@danyd2015 girl just bring yo ass", "in_reply_to_status": 669210318589947904, "in_reply_to_user": 2967873914, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2967873914 }}, "user": { "id": 171689042, "name": "Hollywood", "screen_name": "SlimBaddiee", "lang": "en", "location": "Texas Southern University 19'", "create_at": date("2010-07-27"), "description": "shit fucked up but ima always be good", "followers_count": 8757, "friends_count": 6904, "statues_count": 36635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:21.000Z"), "id": 669213980724867072, "text": "Thanks Kayla ����\n@kayla_Tunell", "in_reply_to_status": 669038758864392192, "in_reply_to_user": 1952841042, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1952841042 }}, "user": { "id": 4240358953, "name": "chill jill", "screen_name": "Jillian_urbane", "lang": "en", "location": "Albuquerque, NM", "create_at": date("2015-11-14"), "description": "shs", "followers_count": 22, "friends_count": 68, "statues_count": 18 }, "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("2015-11-24T10:00:21.000Z"), "id": 669213980812939266, "text": "Not smiling for pics is such a struggle #seniorpics https://t.co/x4NjKf4Y5P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "seniorpics" }}, "user": { "id": 1372931808, "name": "Matt Keenan", "screen_name": "MattCrazy_", "lang": "en", "location": "Dallas ", "create_at": date("2013-04-22"), "description": "hairy-legged all-american teenage writer✞ God bless Francis Lawrence", "followers_count": 624, "friends_count": 599, "statues_count": 12745 }, "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("2015-11-24T10:00:21.000Z"), "id": 669213981018472449, "text": "@JNoFieri why do u have an opinion on whether or not someone is using their slaves \"correctly\" & not immediate outrage @ them having slaves", "in_reply_to_status": 669213555028193281, "in_reply_to_user": 1406922781, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1406922781 }}, "user": { "id": 1406922781, "name": "shakespeare's sister", "screen_name": "JNoFieri", "lang": "en", "location": "Tempe, AZ", "create_at": date("2013-05-05"), "description": "zia record exchange // asu", "followers_count": 498, "friends_count": 298, "statues_count": 19688 }, "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("2015-11-24T10:00:21.000Z"), "id": 669213981282840576, "text": "Bounce out the flat little worm get the birdy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 380502283, "name": "GC3", "screen_name": "G_Cinello7913", "lang": "en", "location": "Pittsburgh, PA | Lower Alabama", "create_at": date("2011-09-26"), "description": "find your wings.", "followers_count": 567, "friends_count": 173, "statues_count": 25754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bethel Park, PA", "id": "7c1a8644282d6a92", "name": "Bethel Park", "place_type": "city", "bounding_box": rectangle("-80.072649,40.287528 -79.998015,40.359255") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4206064, "cityName": "Bethel Park" } }
+{ "create_at": datetime("2015-11-24T10:00:21.000Z"), "id": 669213981387558913, "text": "All time low // Therapy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3475376293, "name": "tyler", "screen_name": "ptvtyler_", "lang": "en", "location": "LA || fanboy", "create_at": date("2015-09-06"), "description": "ashton irwin :)", "followers_count": 508, "friends_count": 862, "statues_count": 308 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-24T10:00:21.000Z"), "id": 669213981618368512, "text": "Yo Fetty Wap Had Me Tripping Last Night ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 577203899, "name": "FXVKYA", "screen_name": "cortasiagoolsby", "lang": "en", "location": "Cincinnati, Oh", "create_at": date("2012-05-11"), "description": "Instagram Fxvkya | UC19", "followers_count": 302, "friends_count": 569, "statues_count": 1806 }, "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("2015-11-24T10:00:21.000Z"), "id": 669213981672914945, "text": "@MillsOdartei please don't come for me this morning", "in_reply_to_status": 669211484283658241, "in_reply_to_user": 571248228, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 571248228 }}, "user": { "id": 336136086, "name": "Toluwanimi", "screen_name": "_k3ndra", "lang": "en", "location": "Arkham World", "create_at": date("2011-07-15"), "description": "Boobs will fall but bum bum is forever", "followers_count": 1212, "friends_count": 864, "statues_count": 19152 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ferguson, MO", "id": "0a62ce0f6aa37536", "name": "Ferguson", "place_type": "city", "bounding_box": rectangle("-90.319108,38.725028 -90.234517,38.785786") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2923986, "cityName": "Ferguson" } }
+{ "create_at": datetime("2015-11-24T10:00:21.000Z"), "id": 669213981819584512, "text": "Happy birthday sugar tits miss&love you stay safe or there.Hope the Air Force isnt too crazy for you!❤️ @WhosRicardo https://t.co/q0y5WirxV8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 381290022 }}, "user": { "id": 436341751, "name": "Heriberto Lopez G.", "screen_name": "itsnotherbie", "lang": "en", "location": "SATX Puerto Vallarta San diego", "create_at": date("2011-12-13"), "description": "Just because you're not happy doesn't mean you're making the wrong decision. I love Tejas. U.S. Marine. Grunt.", "followers_count": 1081, "friends_count": 798, "statues_count": 34626 }, "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-24T10:00:21.000Z"), "id": 669213981865676800, "text": "\"Did Future have on the Gucci flip flops?\"������������������������ https://t.co/6WWt35dxoG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 368156267, "name": "doomed to die young", "screen_name": "SwaggyMatt1", "lang": "en", "location": "Htx, 3rd Koast.", "create_at": date("2011-09-04"), "description": "Your arms too short to box w/ God, i dont kill soloists, i only kill squads.", "followers_count": 614, "friends_count": 616, "statues_count": 37401 }, "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("2015-11-24T10:00:21.000Z"), "id": 669213981916073984, "text": "@TrigMasterFunk lol you know you know. Let's hop on one.", "in_reply_to_status": 669213808582266880, "in_reply_to_user": 102607411, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 102607411 }}, "user": { "id": 606909122, "name": "Derek Georgia ♌", "screen_name": "derr_wreck", "lang": "en", "location": "Pacifica, CA", "create_at": date("2012-06-12"), "description": "Engineering's the job, @erkuhderkuh's the wifey, God's the reason. Going places in life, follow up. Follow my broadcasts at http://www.twitch.tv/derr_wreck/", "followers_count": 152, "friends_count": 110, "statues_count": 10628 }, "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("2015-11-24T10:00:21.000Z"), "id": 669213982452924416, "text": "Sunny this afternoon, high 63 (17 C). Low 48 (9 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575255, "name": "Shreveport Weather", "screen_name": "ShreveportWX", "lang": "en", "location": "Shreveport, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Shreveport, LA", "followers_count": 691, "friends_count": 92, "statues_count": 7823 }, "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("2015-11-24T10:00:21.000Z"), "id": 669213983191224321, "text": "Braving the smoking pub to get a brew, I'm gonna smell like shit! - Drinking a Hopsecutioner @ TAP - https://t.co/PNtT5BGPNm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.4392,33.6381"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1323704605, "name": "Clint Caruolo", "screen_name": "ccaruolo", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2013-04-02"), "description": "Craft Beer | IPA Guy | F15 Crew Chief | Italian | Married to an amazing women | Pizza Fanatic | Giants | 49ers | CAL |", "followers_count": 84, "friends_count": 288, "statues_count": 1397 }, "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("2015-11-24T10:00:21.000Z"), "id": 669213983245750272, "text": "Thankies �� https://t.co/gLQIcsFrTa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2648990783, "name": "Cambodia", "screen_name": "Asiatheasian", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2014-06-27"), "description": "Morality, like art, means drawing a line someplace. - Wilde", "followers_count": 161, "friends_count": 173, "statues_count": 2285 }, "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-24T10:00:21.000Z"), "id": 669213983535165440, "text": "Quite a good job to have @gkbowes!!! Very exiting to see #VoiceActors going for it & landing the gig! ���������� https://t.co/WDl7GT5xpB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VoiceActors" }}, "user_mentions": {{ 627844737 }}, "user": { "id": 626206617, "name": "Bill Russell", "screen_name": "thebillrussell", "lang": "en", "location": "Franklin, Tn", "create_at": date("2012-07-03"), "description": "Audiobook Narrator for http://MeeGenius.com 130+ Children's Audiobooks, and childrens sing along book author! Bill Russell Voice Actor", "followers_count": 251, "friends_count": 384, "statues_count": 853 }, "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("2015-11-24T10:00:21.000Z"), "id": 669213983681941504, "text": "Once I order my lights for my camera then I'm not ordering anything else 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": 282390135, "name": "lowkey", "screen_name": "legacymeen", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2011-04-14"), "description": "Contact Iminjones@outlook.com", "followers_count": 629, "friends_count": 823, "statues_count": 17149 }, "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-24T10:00:21.000Z"), "id": 669213984067833857, "text": "Guess I got myself ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2746050982, "name": "itsclutchhooo", "screen_name": "blackmigoe", "lang": "en", "location": "null", "create_at": date("2014-08-17"), "description": "IG BIGGDAWGGB", "followers_count": 47, "friends_count": 87, "statues_count": 121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodlawn, MD", "id": "bf64a5dd9b5f1cf2", "name": "Woodlawn", "place_type": "city", "bounding_box": rectangle("-76.915396,38.938828 -76.711092,39.327477") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2486525, "cityName": "Woodlawn" } }
+{ "create_at": datetime("2015-11-24T10:00:21.000Z"), "id": 669213984130625536, "text": "Want to work at W.B. Mason? We're #hiring in #EastWindsor, CT! Click for details: https://t.co/ichOAB073r #FacilitiesMgmt #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.6032895,41.9060627"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "EastWindsor", "FacilitiesMgmt", "Job", "Jobs" }}, "user": { "id": 173541362, "name": "CT Facility Mgmt.", "screen_name": "tmj_CT_facmgmt", "lang": "en", "location": "Connecticut", "create_at": date("2010-08-01"), "description": "Follow this account for geo-targeted Facilities Management job tweets in Connecticut Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 136, "friends_count": 125, "statues_count": 13 }, "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": 9003, "countyName": "Hartford" } }
+{ "create_at": datetime("2015-11-24T10:00:21.000Z"), "id": 669213984151605248, "text": "this damn ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 391415966, "name": "chrispy", "screen_name": "chrispychriscat", "lang": "en", "location": "Whittier/ Santa Cruz , CA", "create_at": date("2011-10-15"), "description": "#4sUP #GoSlugs #CuzTheSkyIsPurps snapchat:cgarciadbt ig: @chrispychriscat Use this twitter 95% for sports so if you dont like sports .... sorry.... ayyy lmao", "followers_count": 453, "friends_count": 424, "statues_count": 27698 }, "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("2015-11-24T10:00:21.000Z"), "id": 669213984294371328, "text": "Easy money������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 395526981, "name": "Santino", "screen_name": "_THEREALSANTINO", "lang": "en", "location": "null", "create_at": date("2011-10-21"), "description": "null", "followers_count": 232, "friends_count": 131, "statues_count": 8740 }, "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("2015-11-24T10:00:21.000Z"), "id": 669213984352956416, "text": "@McElroyMarra @JeffGohogs @RobaRanch @nhdogmom @JoJo_Chanel @cahowell1956 @stlwrkr4889 @co_kmaldonado4 @TopherJH GM2U #MuchLove my people", "in_reply_to_status": 668926924278751232, "in_reply_to_user": 2331044448, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MuchLove" }}, "user_mentions": {{ 2331044448, 1895983010, 260534796, 21043723, 32207199, 1963664575, 2203336692, 837561853, 215996801 }}, "user": { "id": 1338149196, "name": "Happy By Nature", "screen_name": "skevkirby", "lang": "en", "location": "null", "create_at": date("2013-04-08"), "description": "CowboyNation, LonghornNation, MavNation Marijuana Activist #LegalizeIt,#BeKind,Just An All Around Chill Dude, Bleeding Heart Liberal Dallas Tx. Lifted #RPG", "followers_count": 16301, "friends_count": 16155, "statues_count": 80356 }, "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("2015-11-24T10:00:21.000Z"), "id": 669213984474558464, "text": "Customer Service Representative - UniFirst Corporation: (#Bangor, ME) https://t.co/ysYSbQo90x #CustomerService #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-68.7778138,44.8011821"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Bangor", "CustomerService", "Job", "Jobs", "Hiring" }}, "user": { "id": 71973526, "name": "ME Cust. Srv. Jobs", "screen_name": "tmj_ME_cstsrv", "lang": "en", "location": "Maine", "create_at": date("2009-09-05"), "description": "Follow this account for geo-targeted Customer Service job tweets in Maine Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 275, "friends_count": 274, "statues_count": 34 }, "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("2015-11-24T10:00:21.000Z"), "id": 669213984541700096, "text": "@flangy you forgot the \"saggy\" qualifier.", "in_reply_to_status": 669213815003725824, "in_reply_to_user": 14209746, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14209746 }}, "user": { "id": 14110325, "name": "Jay ☕️ Holler", "screen_name": "jayholler", "lang": "en", "location": "California, USA", "create_at": date("2008-03-09"), "description": "Husband, father, musician, grump, misanthrope, currently Reliability Engineer @Twitter on Core Infrastructure Services SRE team.", "followers_count": 1092, "friends_count": 1148, "statues_count": 32790 }, "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-24T10:00:21.000Z"), "id": 669213984587911168, "text": "FINALLY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 326411511, "name": "Jesi Slater", "screen_name": "jesislater_", "lang": "en", "location": "null", "create_at": date("2011-06-29"), "description": "null", "followers_count": 799, "friends_count": 625, "statues_count": 31774 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Spa, NY", "id": "6b7ea0f26db87c8b", "name": "Ballston Spa", "place_type": "city", "bounding_box": rectangle("-73.873008,42.967278 -73.835225,43.025133") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36091, "countyName": "Saratoga", "cityID": 3604253, "cityName": "Ballston Spa" } }
+{ "create_at": datetime("2015-11-24T10:00:21.000Z"), "id": 669213984642367488, "text": "@_abbywells woah also had a dream I had a basketball boyfriend last night wow", "in_reply_to_status": 669207914955931652, "in_reply_to_user": 427955297, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 427955297 }}, "user": { "id": 525030645, "name": "Ally Rosenberg", "screen_name": "ally_rosenberg", "lang": "en", "location": "California, USA", "create_at": date("2012-03-14"), "description": "chipotle enthusiast and marcus mariota's biggest fan || ΧΩ || university of nike", "followers_count": 307, "friends_count": 256, "statues_count": 4646 }, "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-24T10:00:22.000Z"), "id": 669213984730451968, "text": "@SkyyTweet https://t.co/s5tuOOkRzt", "in_reply_to_status": -1, "in_reply_to_user": 16390469, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 16390469 }}, "user": { "id": 801763837, "name": "Jamie", "screen_name": "j_lynn39", "lang": "en", "location": "null", "create_at": date("2012-09-03"), "description": "NC • UNC16", "followers_count": 250, "friends_count": 283, "statues_count": 2447 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chapel Hill, NC", "id": "23ebe811ea93fd41", "name": "Chapel Hill", "place_type": "city", "bounding_box": rectangle("-79.083153,35.869376 -78.968887,35.969507") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37135, "countyName": "Orange", "cityID": 3711800, "cityName": "Chapel Hill" } }
+{ "create_at": datetime("2015-11-24T10:00:22.000Z"), "id": 669213984743170048, "text": "�� I won't answer that. Bye https://t.co/Rz63ipdGak", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 207697289, "name": "❣", "screen_name": "TanaisiaAlston", "lang": "en", "location": "Raleigh, NC", "create_at": date("2010-10-25"), "description": "null", "followers_count": 972, "friends_count": 774, "statues_count": 45425 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213985036595200, "text": "Day 3 of lying in bed doing nothing ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "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": 103, "friends_count": 222, "statues_count": 1482 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213985128845313, "text": "Still mexican culture is hella against smokin weed than drinkin knowin DAMN well drinkin is bad af but cus its legal its deemed better...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3194797897, "name": "Lanier King", "screen_name": "kinggg_soda10", "lang": "en", "location": "Salinas, CA", "create_at": date("2015-05-13"), "description": "RIP Uncle DD RIP Ralph Wynn #NLMB", "followers_count": 286, "friends_count": 261, "statues_count": 1185 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213985166659584, "text": "Syrians need the Church's prayer and love. Give thanks. Join me in prayer. #FellowSojourner https://t.co/wBh4oswjFV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FellowSojourner" }}, "user": { "id": 33141880, "name": "Tom Murphy", "screen_name": "BCTS_TomMurphy", "lang": "en", "location": "Dallas, TX", "create_at": date("2009-04-18"), "description": "Executive Director of Biblical Counseling Through Song", "followers_count": 23, "friends_count": 111, "statues_count": 51 }, "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-24T10:00:22.000Z"), "id": 669213985426825217, "text": "Helzberg Diamonds #Job: Seasonal Associate (#Irving, TX) https://t.co/sQBVUnhMyt #Jobs #Hiring", "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", "Jobs", "Hiring" }}, "user": { "id": 1193694967, "name": "Helzberg Diamonds", "screen_name": "HelzbergJobs", "lang": "en", "location": "null", "create_at": date("2013-02-18"), "description": "null", "followers_count": 75, "friends_count": 0, "statues_count": 574 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213985573625857, "text": "Lmfao mannnn what �� https://t.co/DxBaKJLfHr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 302897020, "name": "Ray☯", "screen_name": "rvygo", "lang": "en", "location": "null", "create_at": date("2011-05-21"), "description": "love yourself", "followers_count": 1917, "friends_count": 663, "statues_count": 53315 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213985934176257, "text": "Partly cloudy this afternoon, high 67 (19 C). Low 49 (9 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575312, "name": "Lake Charles Weather", "screen_name": "LakeCharlesWX", "lang": "en", "location": "Lake Charles, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Lake Charles, LA", "followers_count": 536, "friends_count": 92, "statues_count": 7807 }, "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-24T10:00:22.000Z"), "id": 669213985988808704, "text": "Taking hoodies off in public. #ThingsILikeMoreThanAuburn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ThingsILikeMoreThanAuburn" }}, "user": { "id": 46331093, "name": "Spencer Orange", "screen_name": "SpencerOrange", "lang": "en", "location": "256", "create_at": date("2009-06-10"), "description": "Mark 10:43-45", "followers_count": 372, "friends_count": 507, "statues_count": 6334 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213986064207872, "text": "gotta make that muney bestie https://t.co/DQVLbh1MXR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 368094526, "name": "warriors 15-0", "screen_name": "kaayGUWOP", "lang": "en", "location": "Dalla$$$", "create_at": date("2011-09-04"), "description": "yung rich nigga only 16 and you better acknowledge me", "followers_count": 1927, "friends_count": 370, "statues_count": 180878 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213986299232256, "text": "Bought fingerless gloves, so everyone remembers I went to art school", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 22675235, "name": "Peggy O'Leary", "screen_name": "PeggyOLeary", "lang": "en", "location": "Queens", "create_at": date("2009-03-03"), "description": "Comedian. Aunt. Bartender. Host of Hard Lonely & Vicious:The Podcast on Cave Comedy Radio. @HLVPcast", "followers_count": 2128, "friends_count": 884, "statues_count": 4838 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213986357948416, "text": "Lunch with the man himself! (@ Governor's Club in Tallahassee, FL) https://t.co/TEWcX4axFo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.28190248,30.44049386"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48116270, "name": "Ali Raza", "screen_name": "aliraza312", "lang": "en", "location": "Tallahassee, FL", "create_at": date("2009-06-17"), "description": "Proud #TexasEx of @UTAustin B.A. '14. Current @FloridaState #SAgrad M.S. '16. Aspiring #SApro. #ENFJ #NUFP. Trying to change the world. \\m/", "followers_count": 1230, "friends_count": 1402, "statues_count": 11768 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213986391326721, "text": "@KyleBrandt #RayWouldRatherHave Beans over steak", "in_reply_to_status": -1, "in_reply_to_user": 105909196, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RayWouldRatherHave" }}, "user_mentions": {{ 105909196 }}, "user": { "id": 1653856038, "name": "Howie Feltersnatch", "screen_name": "45gotFAT", "lang": "en", "location": "null", "create_at": date("2013-08-07"), "description": "I would probably do Tony Romo", "followers_count": 452, "friends_count": 230, "statues_count": 13163 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213986542518272, "text": "Out 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": 3288852962, "name": "Solo‼️", "screen_name": "mvp_solo", "lang": "en", "location": "null", "create_at": date("2015-07-23"), "description": "BeyondBlessed . .", "followers_count": 260, "friends_count": 206, "statues_count": 2943 }, "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-24T10:00:22.000Z"), "id": 669213986630426626, "text": "Happy birthday cait ���� hope you have a good day�� @caitlinbaum https://t.co/PinPUrJ7Ku", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 477361888 }}, "user": { "id": 3007043905, "name": "Tess", "screen_name": "TessaVaca", "lang": "en", "location": "null", "create_at": date("2015-02-01"), "description": "null", "followers_count": 359, "friends_count": 239, "statues_count": 1296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakeside, CA", "id": "03c9c75ede037d93", "name": "Lakeside", "place_type": "city", "bounding_box": rectangle("-116.948624,32.826697 -116.872927,32.872074") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 639766, "cityName": "Lakeside" } }
+{ "create_at": datetime("2015-11-24T10:00:22.000Z"), "id": 669213986844516352, "text": "@petekofficial @morttagua @NuevaRecordings ������", "in_reply_to_status": 669196752763924480, "in_reply_to_user": 510081736, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 510081736, 374157905, 32793659 }}, "user": { "id": 539390540, "name": "Stan Arwell", "screen_name": "stanarwell", "lang": "en", "location": "New York, NY", "create_at": date("2012-03-28"), "description": "Progressive, Trance producer. 'Flare' is out on Armada Trice now: http://bit.ly/OMSAFlareBP Promos: promos@stanarwell.com", "followers_count": 3431, "friends_count": 198, "statues_count": 4971 }, "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-24T10:00:22.000Z"), "id": 669213986970300416, "text": "I can't wait for this weekend it's going to be amazing ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 627717055, "name": "Jessica Krisky", "screen_name": "jessicakrisky", "lang": "en", "location": "null", "create_at": date("2012-07-05"), "description": "This is my life. Try & keep up.", "followers_count": 238, "friends_count": 212, "statues_count": 3988 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spring Grove, IL", "id": "00adf35b11bba302", "name": "Spring Grove", "place_type": "city", "bounding_box": rectangle("-88.30152,42.415601 -88.198652,42.495082") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17111, "countyName": "McHenry", "cityID": 1772052, "cityName": "Spring Grove" } }
+{ "create_at": datetime("2015-11-24T10:00:22.000Z"), "id": 669213987003883520, "text": "Look how close I am to you yet we have never met", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4214573015, "name": "• :) cara", "screen_name": "welovejacob_w", "lang": "en", "location": "null", "create_at": date("2015-11-17"), "description": "I love Jacob, yes I do, I love Jacob how bout you?", "followers_count": 15, "friends_count": 21, "statues_count": 89 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213987121311746, "text": "@KpScoleri I'm glad to see I'm not the only one fired up about this. #Panera #youpicktwo", "in_reply_to_status": 668844766008741889, "in_reply_to_user": 294938219, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Panera", "youpicktwo" }}, "user_mentions": {{ 294938219 }}, "user": { "id": 821657797, "name": "Rachel Doane", "screen_name": "Rachel_Doane", "lang": "en", "location": "Grand Rapids, Michigan", "create_at": date("2012-09-13"), "description": "Just a right-wing girl, living in a left-wing world.", "followers_count": 202, "friends_count": 322, "statues_count": 2473 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213987192487937, "text": "All these holidays coming up without my dad here seems so unreal������❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 227795406, "name": "ashley", "screen_name": "gangsterashley", "lang": "en", "location": "null", "create_at": date("2010-12-17"), "description": "I miss you Dad. Forever and Always. ❤️", "followers_count": 553, "friends_count": 303, "statues_count": 47352 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213987431649281, "text": "haven't even bought the first Christmas present for anyone ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 535989197, "name": "Breanna", "screen_name": "breannadamron_", "lang": "en", "location": "KENTUCKYYY", "create_at": date("2012-03-24"), "description": "21-expecting a little princess in March.", "followers_count": 307, "friends_count": 333, "statues_count": 11704 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213988157263872, "text": "@_WillofFortune Will", "in_reply_to_status": 669213716785819648, "in_reply_to_user": 218760518, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 218760518 }}, "user": { "id": 237051181, "name": "Queen Petty", "screen_name": "50ShadesOfClair", "lang": "en", "location": "NY✈️GA", "create_at": date("2011-01-11"), "description": "Opinionated Brat. SWD Alumna. WestIndian. #VSU17. #TYNN Honorary Jackson. God does everything for a reason.", "followers_count": 5034, "friends_count": 4587, "statues_count": 233127 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213988270424065, "text": "@tcow715 oh yeah we still going", "in_reply_to_status": 669213939792613376, "in_reply_to_user": 342767218, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 342767218 }}, "user": { "id": 478106292, "name": "That One Guy", "screen_name": "brady_thomas2", "lang": "en", "location": "30.6014° N, 96.3144° W", "create_at": date("2012-01-29"), "description": "Still wearing chacos, and jammin to jams bout Jesus. #TAMU18 (Romans 14:8) #FishCamp15 #TAMUQuidditch #kb #IStandWithFrance", "followers_count": 1436, "friends_count": 1096, "statues_count": 27178 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213988371075072, "text": "@GreenRope Why do you host Spammers? \"aventigroup . com\" has been requested more than a few times to halt their communications.", "in_reply_to_status": -1, "in_reply_to_user": 223748270, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 223748270 }}, "user": { "id": 9531312, "name": "chris romero", "screen_name": "Vectrexer", "lang": "en", "location": "Pacifica, CA", "create_at": date("2007-10-18"), "description": "Enjoys his Vectrex Game System and his other fast things in life.", "followers_count": 97, "friends_count": 122, "statues_count": 954 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213988400463872, "text": "The judicial system is corrupt for the Sioux Falls SD Police Department, like all depts, disorganized. Information is not contained.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 579163841, "name": "Observation Team1", "screen_name": "walkwithmany", "lang": "en", "location": "Sioux Falls SD", "create_at": date("2012-05-13"), "description": "Don't be pushed by your problems; be led by your dreams - author unknown", "followers_count": 4686, "friends_count": 4783, "statues_count": 125504 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213988555624448, "text": "@Raawiteza awesome. . Chala friendly and here people respect each other and very polite..", "in_reply_to_status": 669212936183914496, "in_reply_to_user": 259325093, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 259325093 }}, "user": { "id": 145607289, "name": "❤**∂α∂∂у'ѕ gιяℓ**❤", "screen_name": "Tejaswi_Tweets", "lang": "en", "location": " Seattle, Washington ", "create_at": date("2010-05-19"), "description": "ƒυη / ℓσνιηg / ƒяιєη∂ly ♥ / вσяη ση oct 21st ^_^/ diehard ƒαη σƒ @PawanKalyan @urstrulyMahesh @ramsayz @Samanthaprabhu2", "followers_count": 1001, "friends_count": 101, "statues_count": 18259 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213988589301760, "text": "�� https://t.co/sb6I5ZLyB6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1577814444, "name": "Lauren Murphy", "screen_name": "love_murphyy", "lang": "en", "location": "null", "create_at": date("2013-07-08"), "description": "null", "followers_count": 248, "friends_count": 258, "statues_count": 5085 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213988668973056, "text": "all your talk of inferior turkey! there is just not enough turkey here either! #begratefulalways… https://t.co/XldzpyZC3z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.51828563,47.52229657"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "begratefulalways" }}, "user": { "id": 14864352, "name": "Johanna Baxter", "screen_name": "hanniespice", "lang": "en", "location": "Yukon Harbor, WA", "create_at": date("2008-05-21"), "description": "cute democrat•blogger since 2001•activist•social media Girl Scout •#wadem #26dems #iwilllisten http://hannie.ninja/info", "followers_count": 2594, "friends_count": 2806, "statues_count": 142554 }, "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("2015-11-24T10:00:22.000Z"), "id": 669213988803055617, "text": "@BreeNewsome @Bruiser4Eq rule, rile... which is closer?\n#FreudianTypos", "in_reply_to_status": 669212799323779072, "in_reply_to_user": 110326494, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FreudianTypos" }}, "user_mentions": {{ 110326494, 182530270 }}, "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.\n47, married, 10 yo son.\nStories and poems at http://christophermahan.com/writings/", "followers_count": 965, "friends_count": 891, "statues_count": 52496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-24T10:00:22.000Z"), "id": 669213988870164480, "text": "@MikePWInsider #NXT Superstar Solomon Crowe has had his profile removed from official WWE roster page. What happen? https://t.co/znR7Q7JznP", "in_reply_to_status": -1, "in_reply_to_user": 103761467, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NXT" }}, "user_mentions": {{ 103761467 }}, "user": { "id": 177469044, "name": "ナルトファン", "screen_name": "calimike1", "lang": "en", "location": "Woodland Hills, CA", "create_at": date("2010-08-11"), "description": "null", "followers_count": 489, "friends_count": 818, "statues_count": 25933 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-24T10:00:23.000Z"), "id": 669213988899524608, "text": "Mostly sunny this afternoon, high 67 (19 C). Low 51 (11 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575370, "name": "Lafayette Weather", "screen_name": "LafayetteWX", "lang": "en", "location": "Lafayette, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Lafayette, LA", "followers_count": 464, "friends_count": 92, "statues_count": 7846 }, "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("2015-11-24T10:00:23.000Z"), "id": 669213989046358017, "text": "Three months in Tuscaloosa made me forget that old people exist", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1481139031, "name": "Nathan Salazar", "screen_name": "n_sal14", "lang": "en", "location": "Psalm 23:4", "create_at": date("2013-06-03"), "description": "Bama '19.", "followers_count": 185, "friends_count": 165, "statues_count": 2418 }, "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-24T10:00:23.000Z"), "id": 669213989377716224, "text": "@allyportz11 https://t.co/NwCRHNc6D0", "in_reply_to_status": 669213412866543616, "in_reply_to_user": 475356052, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 475356052 }}, "user": { "id": 574251558, "name": "Beat _ichigan", "screen_name": "primetime_tony", "lang": "en", "location": "Prison City, Illinois", "create_at": date("2012-05-07"), "description": "Jack of all trades. Master of most. http://FFcouchcoach.com contributor. #GoBucks", "followers_count": 456, "friends_count": 298, "statues_count": 2889 }, "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("2015-11-24T10:00:23.000Z"), "id": 669213989390389248, "text": "Tonight on tango", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 213976096, "name": "CNOVAH13", "screen_name": "csymatha913", "lang": "en", "location": "null", "create_at": date("2010-11-09"), "description": "optimisticart@weebly.com", "followers_count": 43, "friends_count": 224, "statues_count": 462 }, "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-24T10:00:23.000Z"), "id": 669213989579177985, "text": "Can it all go back to normal with him? Please.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3463643853, "name": "Will Fouty", "screen_name": "WillFouty", "lang": "en", "location": "null", "create_at": date("2015-08-27"), "description": "null", "followers_count": 90, "friends_count": 192, "statues_count": 221 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Terre Haute, IN", "id": "8a61588aff8b0577", "name": "Terre Haute", "place_type": "city", "bounding_box": rectangle("-87.460551,39.394564 -87.303557,39.520714") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18167, "countyName": "Vigo", "cityID": 1875428, "cityName": "Terre Haute" } }
+{ "create_at": datetime("2015-11-24T10:00:23.000Z"), "id": 669213989587546115, "text": "just gained a billion pounds from newsp thanksgiving oh 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": 530028927, "name": "Hales", "screen_name": "puckfreakhaley3", "lang": "en", "location": "RHS '17", "create_at": date("2012-03-19"), "description": "The problem with people is they forget that most of the time, it's the small things that count. • WFSC ❄️ • RIP Granny ❤️• DRW • #HAP #skeletønclique |-/", "followers_count": 658, "friends_count": 610, "statues_count": 47645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wyandotte, MI", "id": "5d9b412793ef5656", "name": "Wyandotte", "place_type": "city", "bounding_box": rectangle("-83.181455,42.183685 -83.141012,42.235102") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2688900, "cityName": "Wyandotte" } }
+{ "create_at": datetime("2015-11-24T10:00:23.000Z"), "id": 669213989675495426, "text": "lol i knew i wasn't trippin. https://t.co/0i6UTxmeFH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 40681203, "name": "BIG Bank Hank", "screen_name": "MichaelStrauss_", "lang": "en", "location": "Trill, Texas", "create_at": date("2009-05-17"), "description": "For All Mines I'll Lay Yours; MOB", "followers_count": 609, "friends_count": 129, "statues_count": 88448 }, "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("2015-11-24T10:00:23.000Z"), "id": 669213989864390656, "text": "not talking to you,is absolutely 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": 2767881449, "name": "Queen B.", "screen_name": "haleighkate14", "lang": "en", "location": "731", "create_at": date("2014-09-12"), "description": "content.", "followers_count": 233, "friends_count": 308, "statues_count": 1928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Selmer, TN", "id": "5f18aa7cce01ff7d", "name": "Selmer", "place_type": "city", "bounding_box": rectangle("-88.628319,35.141433 -88.547729,35.210131") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47109, "countyName": "McNairy", "cityID": 4766940, "cityName": "Selmer" } }
+{ "create_at": datetime("2015-11-24T10:00:23.000Z"), "id": 669213989872787457, "text": "#ThanksgivingWithWhiteFamilies https://t.co/QYOmLh9mbB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "ThanksgivingWithWhiteFamilies" }}, "user": { "id": 31577691, "name": "Queen Ynobe", "screen_name": "ebonyspencer_", "lang": "en", "location": "greenville", "create_at": date("2009-04-15"), "description": "Ya-No-Be | 21 | East Carolina University | F.A.M.E. Modeling Troupe | #RIP JL & MB ❤️", "followers_count": 1655, "friends_count": 638, "statues_count": 88447 }, "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("2015-11-24T10:00:23.000Z"), "id": 669213990115938304, "text": "Vegas Utah vs BYU, \"When You Choke the Pac12 Championship Away, You Deserve BYU Duel\" sponsored by Cialis, Lunesta, and Bacardi Gold.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 600885517, "name": "Utah Pig Bus", "screen_name": "UtahPigBus", "lang": "en", "location": "Downtown, Salt Lake City, Utah", "create_at": date("2012-06-06"), "description": "University of Utah, Utes, tailgate pros, pig roast, bacon, beer, downtown Salt Lake City dweller, Real Salt Lake, Jazz, and Utah Man Am I!! Go Utes!! Sportsing!", "followers_count": 2445, "friends_count": 1530, "statues_count": 34150 }, "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("2015-11-24T10:00:23.000Z"), "id": 669213990141239300, "text": "need someone that likes watching Dexter as much as 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": 2586431395, "name": "ana gabby", "screen_name": "anaapaschee", "lang": "en", "location": "St Paul, MN", "create_at": date("2014-06-24"), "description": "classy but slightly hood, with a lot of God in her.", "followers_count": 287, "friends_count": 281, "statues_count": 2825 }, "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("2015-11-24T10:00:23.000Z"), "id": 669213990149451776, "text": "@pcxmacky toooo far", "in_reply_to_status": 669213737744666624, "in_reply_to_user": 487111621, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 487111621 }}, "user": { "id": 598923070, "name": "w/theshitz✨", "screen_name": "DopeMykiah", "lang": "en", "location": "null", "create_at": date("2012-06-03"), "description": "eat me❤ ||", "followers_count": 1735, "friends_count": 1089, "statues_count": 29505 }, "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("2015-11-24T10:00:23.000Z"), "id": 669213990216712192, "text": "Pumpkin spice ale for the Basic (alcoholic) Bitch in me https://t.co/rxuAZbQpLZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21252153, "name": "Niamh Ó Maoláin", "screen_name": "nifdiddly", "lang": "en", "location": "Guernsey", "create_at": date("2009-02-18"), "description": "Lucille Bluth in training. #Stiamh2016 representin' Selfies and feminism all damn day.", "followers_count": 252, "friends_count": 134, "statues_count": 6311 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Farmingdale, NY", "id": "80fb5c3be0583f42", "name": "Farmingdale", "place_type": "city", "bounding_box": rectangle("-73.463143,40.723554 -73.433217,40.740523") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3625384, "cityName": "Farmingdale" } }
+{ "create_at": datetime("2015-11-24T10:00:23.000Z"), "id": 669213990304788481, "text": "Dino bones & petroglyphs #travel #roadtrip #FindYourPark (@ Dinosaur National Monument in Jensen, UT) https://t.co/HOKt7sJuos", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-109.30774182,40.43817375"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "travel", "roadtrip", "FindYourPark" }}, "user": { "id": 274094922, "name": "Susan Herman", "screen_name": "TootSweetSu", "lang": "en", "location": "Sacramento, CA", "create_at": date("2011-03-29"), "description": "Sock hunter, musician, freelance editor. Editing at http://www.edit2yourcredit.com. Storytelling at my other site, Slide Show and Ice Cream.", "followers_count": 963, "friends_count": 2003, "statues_count": 3487 }, "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": 49047, "countyName": "Uintah" } }
+{ "create_at": datetime("2015-11-24T10:00:23.000Z"), "id": 669213990577278976, "text": "Interested in a #Art #job near #Milpitas, CA? This could be a great fit: https://t.co/gLZMCh860c #TheCreativeGroup #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": {{ "Art", "job", "Milpitas", "TheCreativeGroup", "Hiring" }}, "user": { "id": 793777357, "name": "TCG Jobs", "screen_name": "TCGJobs", "lang": "en", "location": "null", "create_at": date("2012-08-31"), "description": "TCG is a staffing firm that places design, interactive and marketing professionals in freelance and full-time jobs. See all our Jobs here!", "followers_count": 184, "friends_count": 7, "statues_count": 18649 }, "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("2015-11-24T10:00:23.000Z"), "id": 669213990724182016, "text": "@GovRauner I WANT MY LIFE BACK GO HUNTING WITH MY FREINDS GET MY HEALTH AND LIFE BACK AS OSHA LAW STATES 15 MIL THEY NEED PAY UP GAGNON", "in_reply_to_status": -1, "in_reply_to_user": 2862555526, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2862555526 }}, "user": { "id": 3828320302, "name": "GAGNON GARY", "screen_name": "CHKMTE0", "lang": "en", "location": "USA ", "create_at": date("2015-09-30"), "description": "HAZARD COPS NEED BILLY CLUBS", "followers_count": 182, "friends_count": 973, "statues_count": 474 }, "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("2015-11-24T10:00:23.000Z"), "id": 669213991097470976, "text": "@Forever_IMB awwwww", "in_reply_to_status": 669213017519865856, "in_reply_to_user": 348077471, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 348077471 }}, "user": { "id": 2874111543, "name": "✨NashayTiffani✨", "screen_name": "NashayTiffani", "lang": "en", "location": "null", "create_at": date("2014-11-12"), "description": "null", "followers_count": 339, "friends_count": 323, "statues_count": 5976 }, "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-24T10:00:23.000Z"), "id": 669213991227494400, "text": "I'm so excited for thanksgiving ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 736342771, "name": "Shelly Ann", "screen_name": "ginga_shelly", "lang": "en", "location": "Pewee Valley, KY", "create_at": date("2012-08-04"), "description": "•Forgive,Forget and just be happy.•", "followers_count": 508, "friends_count": 1353, "statues_count": 25900 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pewee Valley, KY", "id": "0058d4bc0d90a973", "name": "Pewee Valley", "place_type": "city", "bounding_box": rectangle("-85.526856,38.276076 -85.388264,38.35041") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2160492, "cityName": "Pewee Valley" } }
+{ "create_at": datetime("2015-11-24T10:00:23.000Z"), "id": 669213991281889281, "text": "When your cousins get into a fight over who gets the TV remote #ThanksgivingWithWhiteFamilies https://t.co/4dlHfC1GjR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ThanksgivingWithWhiteFamilies" }}, "user": { "id": 2461424049, "name": "Taking Too Many L's", "screen_name": "_JulyAlsina", "lang": "en", "location": "null", "create_at": date("2014-04-04"), "description": "SHS Graduate.FriendZone Jesus Pre Dental major Gentleman and Hopless romantic since age 5 Advocate of #PeaceLovePositivity #Tsu19 ❤️London Zhiloh", "followers_count": 766, "friends_count": 731, "statues_count": 14873 }, "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("2015-11-24T10:00:23.000Z"), "id": 669213991630200832, "text": "Looking for an #luxurious gift this #holiday? #Designer @ElinaCasell showcased her handbags available @miabossi. https://t.co/BYyPEJ0ze8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "luxurious", "holiday", "Designer" }}, "user_mentions": {{ 375411001, 1290328698 }}, "user": { "id": 296372997, "name": "You & Me", "screen_name": "WCIU_YouAndMe", "lang": "en", "location": "Chicago", "create_at": date("2011-05-10"), "description": "REAL #Chicago. REAL #Women. REAL #Fun. Catch #YMTM with hosts @MelissaForman and @JMSparrow on #WCIU - The U! Weekdays from 6-9AM!", "followers_count": 4581, "friends_count": 796, "statues_count": 8679 }, "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-24T10:00:23.000Z"), "id": 669213991651053575, "text": "Times like these make me want to live as far away from snow as possible. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1015762886, "name": "Katie Gardner", "screen_name": "katieh_gardner", "lang": "en", "location": "null", "create_at": date("2012-12-16"), "description": "Oregon.", "followers_count": 328, "friends_count": 250, "statues_count": 2729 }, "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("2015-11-24T10:00:23.000Z"), "id": 669213991734939648, "text": "That's what I'm good at apparently", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 281076914, "name": "Millie", "screen_name": "MikeMillie_", "lang": "en", "location": "LaVista, NE", "create_at": date("2011-04-12"), "description": "11-27-07 R.I.P. Juanita Maria Miller Forever in My Heart", "followers_count": 888, "friends_count": 1130, "statues_count": 26034 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Papillion, NE", "id": "43212288cc83156e", "name": "Papillion", "place_type": "city", "bounding_box": rectangle("-96.078103,41.117927 -95.962936,41.183482") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31153, "countyName": "Sarpy", "cityID": 3138295, "cityName": "Papillion" } }
+{ "create_at": datetime("2015-11-24T10:00:23.000Z"), "id": 669213991739064320, "text": "\"My kingdom is not about earning and deserving; it's about believing and receiving!\" \n#TuesdayThought", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TuesdayThought" }}, "user": { "id": 75430043, "name": "Brandon", "screen_name": "MasterBrandon97", "lang": "en", "location": "Pismo Beach, CA", "create_at": date("2009-09-18"), "description": "null", "followers_count": 277, "friends_count": 284, "statues_count": 11508 }, "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("2015-11-24T10:00:23.000Z"), "id": 669213991927984130, "text": "I got money i can worry bout hoes . I cry in traps 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": 217314506, "name": "Cleon:☮レベル 5☮", "screen_name": "DuckyMontana", "lang": "en", "location": "The Magical Land of Cleve", "create_at": date("2010-11-18"), "description": "#VCR warlord// #GR4DC ➖ducksauceceejay@gmail.com", "followers_count": 1167, "friends_count": 119, "statues_count": 101630 }, "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("2015-11-24T10:00:23.000Z"), "id": 669213991948804096, "text": "#EmptyTheTanks https://t.co/CqLAH9PJ6g", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "EmptyTheTanks" }}, "user": { "id": 872068304, "name": "Katina", "screen_name": "katinaf123", "lang": "en", "location": "null", "create_at": date("2012-10-10"), "description": "⬇ STOP Yulin! please sign petition ⬇", "followers_count": 229, "friends_count": 689, "statues_count": 1852 }, "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("2015-11-24T10:00:23.000Z"), "id": 669213992196280320, "text": "Sunny this afternoon, high 63 (17 C). Low 46 (8 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575409, "name": "Fort Smith Weather", "screen_name": "FortSmithWX", "lang": "en", "location": "Fort Smith, AR", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Fort Smith, AR", "followers_count": 440, "friends_count": 92, "statues_count": 7895 }, "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("2015-11-24T10:00:23.000Z"), "id": 669213992238223360, "text": "@atsuki66 \nてかこの春休みの旅行アメリカちゃうかっと。��", "in_reply_to_status": 669130325419692032, "in_reply_to_user": 560308327, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user_mentions": {{ 560308327 }}, "user": { "id": 556895922, "name": "澤田 陽太", "screen_name": "xyotax96", "lang": "ja", "location": "California, USA", "create_at": date("2012-04-18"), "description": "京都/左京/Rits/OIC/国際経営/2回/アメリカ/カリフォルニア/サンフランシスコ在住/Follow me!!!! http://instagram.com/yota_sawada", "followers_count": 1222, "friends_count": 808, "statues_count": 13048 }, "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-24T10:00:23.000Z"), "id": 669213992313733121, "text": "See our latest #SanClemente, CA #job and click to apply: Jr. Quality Engineer - https://t.co/s2ClTQg1my #Manufacturing #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.6119925,33.4269728"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SanClemente", "job", "Manufacturing", "Hiring" }}, "user": { "id": 843975835, "name": "CA Manufacturing", "screen_name": "tmj_ca_manuf", "lang": "en", "location": "California Non-Metro", "create_at": date("2012-09-24"), "description": "Follow this account for geo-targeted Manufacturing job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 325, "friends_count": 209, "statues_count": 331 }, "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("2015-11-24T10:00:23.000Z"), "id": 669213992984821760, "text": "So @rushlimbaugh if the Rich-Poor gap got larger during Obama's reign is it still G.W. Bush's fault?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 342887079 }}, "user": { "id": 265666449, "name": "mark wonderful", "screen_name": "markwonderful", "lang": "en", "location": "Phoenix, Az", "create_at": date("2011-03-13"), "description": "Christian Ex-1%er Asperger's Az-Born-Boomer Conservative Pro-Israel Photos Actor Writer Reader 2001 to 2014 guarding the Rich. RETIRED", "followers_count": 1820, "friends_count": 1512, "statues_count": 127182 }, "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-24T10:00:24.000Z"), "id": 669213993236430848, "text": "I love when people do that silent \"I can't breath\" laugh. It makes me lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 306933372, "name": ".", "screen_name": "Suckaaemm", "lang": "en", "location": "Riverside, CA", "create_at": date("2011-05-28"), "description": "null", "followers_count": 306, "friends_count": 27, "statues_count": 70576 }, "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-24T10:00:24.000Z"), "id": 669213993496649728, "text": "when someone faints before you give blood ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1016475356, "name": "TurkeyG", "screen_name": "gurrlpleasee", "lang": "en", "location": "null", "create_at": date("2012-12-16"), "description": "i dig it i dig it", "followers_count": 230, "friends_count": 192, "statues_count": 9488 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:24.000Z"), "id": 669213993626537985, "text": "At Work Like ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 233090529, "name": "R.I.P GRANNY", "screen_name": "LilBrian_iTsAgo", "lang": "en", "location": "jȏʟıєṭ | ċһıċѧɢȏ", "create_at": date("2011-01-01"), "description": "arтιѕт, ѕongwrιтer | LΣΩ ♌️ | 8⃣1⃣7⃣ | ѕnapcнaт | ιnѕтagraм: @LILBRIAN_ITSAGO #lвe #nғl #ιтѕago #dyнмn |corιnтнιanѕ 5:7 aѕv", "followers_count": 1134, "friends_count": 980, "statues_count": 16967 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Preston Heights, IL", "id": "8df4294fff20f553", "name": "Preston Heights", "place_type": "city", "bounding_box": rectangle("-88.088236,41.480031 -88.060955,41.502664") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1761860, "cityName": "Preston Heights" } }
+{ "create_at": datetime("2015-11-24T10:00:24.000Z"), "id": 669213993630732288, "text": "Me gusta leg day", "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": "champagnemami", "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": 531, "friends_count": 470, "statues_count": 25994 }, "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("2015-11-24T10:00:24.000Z"), "id": 669213993739923456, "text": "I wish I could explain the things my dad says... Just know they're hilarious and he is still my favorite person", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 269348678, "name": "Katherine", "screen_name": "katyhammond10", "lang": "en", "location": "LaTech", "create_at": date("2011-03-20"), "description": "just wanna be at the beach", "followers_count": 492, "friends_count": 278, "statues_count": 9147 }, "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("2015-11-24T10:00:24.000Z"), "id": 669213993958010882, "text": "Never thought I'd say this but can't wait to be able to run again", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 334288880, "name": "Gabby Dyer", "screen_name": "dyeronfireee", "lang": "en", "location": "null", "create_at": date("2011-07-12"), "description": "A trail of stardust leading to the superstar.... #Ao1", "followers_count": 473, "friends_count": 430, "statues_count": 8425 }, "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("2015-11-24T10:00:24.000Z"), "id": 669213994406666240, "text": "Why can't you take me Jason ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 876975708, "name": "♏︎addy❥❥", "screen_name": "MendozaMaddy", "lang": "en", "location": "null", "create_at": date("2012-10-12"), "description": "| jason mccall❤️⚾︎| sc➡️mendozamaddy343", "followers_count": 229, "friends_count": 237, "statues_count": 523 }, "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("2015-11-24T10:00:24.000Z"), "id": 669213994499096576, "text": "Smoke some trees, pop some p's, later she'll be on her knees https://t.co/b5ukLDE78f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 705591951, "name": "lyndsey robison", "screen_name": "lyndsmarie24", "lang": "en", "location": "null", "create_at": date("2013-10-08"), "description": "SCHS junior // ZC ❤️", "followers_count": 516, "friends_count": 575, "statues_count": 12699 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hendersonville, TN", "id": "db6c24fcc9e59ba4", "name": "Hendersonville", "place_type": "city", "bounding_box": rectangle("-86.674887,36.248263 -86.495313,36.366587") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47165, "countyName": "Sumner", "cityID": 4733280, "cityName": "Hendersonville" } }
+{ "create_at": datetime("2015-11-24T10:00:24.000Z"), "id": 669213994675142656, "text": "Damn my mom is going to leave me here forever", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 320453372, "name": "Dyana", "screen_name": "dyanaaa_98", "lang": "en", "location": "null", "create_at": date("2011-06-19"), "description": "null", "followers_count": 346, "friends_count": 233, "statues_count": 12459 }, "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("2015-11-24T10:00:24.000Z"), "id": 669213994742231040, "text": "1. #ceic\n2. Russian\n3. #6YearsOfOneTime\n4. #TravelTuesday\n5. #RuinAnOldSaying\n\n2015/11/24 11:54 CST #trndnl https://t.co/SHjd3tkINE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.309,56.9547"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ceic", "6YearsOfOneTime", "TravelTuesday", "RuinAnOldSaying", "trndnl" }}, "user": { "id": 1270239397, "name": "Trendinalia Canada", "screen_name": "trendinaliaCA", "lang": "es", "location": "Canada", "create_at": date("2013-03-15"), "description": "Information about Twitter's Trending Topics in Canada", "followers_count": 1148, "friends_count": 7, "statues_count": 238620 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Division No. 23, Unorganized, Manitoba", "id": "654fd5617c7361d0", "name": "Division No. 23, Unorganized", "place_type": "city", "bounding_box": rectangle("-102.009382,55.158501 -88.989166,60.000283") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2015-11-24T10:00:24.000Z"), "id": 669213994985631748, "text": "@CalumvonMoger @shizzam5", "in_reply_to_status": 650957426783465472, "in_reply_to_user": 549693872, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 549693872, 87292635 }}, "user": { "id": 3315895246, "name": "Wyatt Sullivan", "screen_name": "WyattSully", "lang": "en", "location": "null", "create_at": date("2015-06-09"), "description": "#YDKM", "followers_count": 567, "friends_count": 575, "statues_count": 2288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Church Point, LA", "id": "bf64505c7d0045e0", "name": "Church Point", "place_type": "city", "bounding_box": rectangle("-92.236264,30.388199 -92.195636,30.42158") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22001, "countyName": "Acadia", "cityID": 2215465, "cityName": "Church Point" } }
+{ "create_at": datetime("2015-11-24T10:00:24.000Z"), "id": 669213995170074624, "text": "Partly cloudy this afternoon, high 60 (16 C). Low 40 (4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575444, "name": "Little Rock Weather", "screen_name": "LittleRockWX", "lang": "en", "location": "Little Rock, AR", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Little Rock, AR", "followers_count": 1834, "friends_count": 92, "statues_count": 7769 }, "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("2015-11-24T10:00:24.000Z"), "id": 669213995224670208, "text": "Wait aren't they both the same thing? RT @youbeenafan_04: Boff �� https://t.co/wVd0CIGW2g https://t.co/QJ5fiMpqnK", "in_reply_to_status": 669196706085343232, "in_reply_to_user": 636383045, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 636383045 }}, "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": 1435, "friends_count": 849, "statues_count": 124301 }, "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.739262,41.923196") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2015-11-24T10:00:24.000Z"), "id": 669213995639771136, "text": "#Orlando, FL #Retail #Job: Sales Associate at Sterling Jewelers https://t.co/C3ItYBFwnd #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2059656,28.5541187"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Orlando", "Retail", "Job", "Jobs", "Hiring" }}, "user": { "id": 3248774184, "name": "Kay Jewelers", "screen_name": "JobsAtKay", "lang": "en", "location": "null", "create_at": date("2015-06-18"), "description": "Your Future Starts Here.", "followers_count": 5, "friends_count": 0, "statues_count": 1092 }, "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("2015-11-24T10:00:24.000Z"), "id": 669213995782410241, "text": "I've been harboring a deep love for 3 years now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1963590990, "name": "táhirih", "screen_name": "nurserycrymes", "lang": "en", "location": "wokeville", "create_at": date("2013-10-15"), "description": "DAB ON 'EM FOLKS", "followers_count": 328, "friends_count": 168, "statues_count": 41934 }, "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-24T10:00:24.000Z"), "id": 669213995790897152, "text": "I been waiting 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": 386355763, "name": "Marcus ☔️", "screen_name": "Im_the_KidMarco", "lang": "en", "location": "w/bff @HYFR_iTwerk", "create_at": date("2011-10-06"), "description": "#MyASU19 R.I.P Tray6⃣ R.I.P. Brandon", "followers_count": 594, "friends_count": 686, "statues_count": 31860 }, "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("2015-11-24T10:00:24.000Z"), "id": 669213996260728832, "text": "Me dancing in the head of some sugar plum fairie https://t.co/Nk0zfATd5Y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2987253782, "name": "⛎ Scoundrel Soup ⛎", "screen_name": "johncalvinn28", "lang": "en", "location": " Charlotte N.C.", "create_at": date("2015-01-17"), "description": "⛎ Being funky is a state of mind...come get you some! ⛎", "followers_count": 133, "friends_count": 386, "statues_count": 5609 }, "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("2015-11-24T10:00:24.000Z"), "id": 669213996457693184, "text": "I want a strawberry milkshake.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 51401535, "name": "Crystal", "screen_name": "JAAAZcrystal", "lang": "en", "location": "City of Angels", "create_at": date("2009-06-27"), "description": "X X V ♡ #Timers", "followers_count": 923, "friends_count": 1201, "statues_count": 41621 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-24T10:00:24.000Z"), "id": 669213996491210752, "text": "being behind on bills is probably the most stressful thing ever", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 347147548, "name": "e-ran", "screen_name": "ItsJustErin97", "lang": "en", "location": "null", "create_at": date("2011-08-02"), "description": "makeup and chicken wings", "followers_count": 337, "friends_count": 245, "statues_count": 18808 }, "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("2015-11-24T10:00:24.000Z"), "id": 669213996600299520, "text": "6. #GRAMMYs\n7. Devante Smith-Pelly\n8. Forbes\n9. Fort York\n10. Jean-Marc Fournier\n\n2015/11/24 11:54 CST #trndnl https://t.co/SHjd3tkINE", "in_reply_to_status": 669213994742231040, "in_reply_to_user": 1270239397, "favorite_count": 0, "coordinate": point("-98.309,56.9547"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GRAMMYs", "trndnl" }}, "user": { "id": 1270239397, "name": "Trendinalia Canada", "screen_name": "trendinaliaCA", "lang": "es", "location": "Canada", "create_at": date("2013-03-15"), "description": "Information about Twitter's Trending Topics in Canada", "followers_count": 1148, "friends_count": 7, "statues_count": 238621 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Division No. 23, Unorganized, Manitoba", "id": "654fd5617c7361d0", "name": "Division No. 23, Unorganized", "place_type": "city", "bounding_box": rectangle("-102.009382,55.158501 -88.989166,60.000283") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2015-11-24T10:00:24.000Z"), "id": 669213996646539265, "text": "@ATVIAssist I have all neccassar ports opened on my Xbox one but I'm repeatedly booted from multiplayer and party chat.", "in_reply_to_status": 669213377269485568, "in_reply_to_user": 85215038, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 85215038 }}, "user": { "id": 700666898, "name": "∤ THOM∀S ∤", "screen_name": "theDROOGSruler", "lang": "en", "location": "null", "create_at": date("2012-07-17"), "description": "Ruler of the Droogies. Never a dull moment. Self proclamied Music Snob. GAMER. Streamer. Joker.", "followers_count": 142, "friends_count": 221, "statues_count": 2478 }, "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("2015-11-24T10:00:24.000Z"), "id": 669213996755632128, "text": "Estas #Curioso sobre #LatinLeche en #RainbowPlayground #Queens #NYC Martes 7pm-12am? Escribe o manda un DM para info #BBBH #BBBHNYC #BB33", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "Curioso", "LatinLeche", "RainbowPlayground", "Queens", "NYC", "BBBH", "BBBHNYC", "BB33" }}, "user": { "id": 2811605702, "name": "J. C. S. #RD", "screen_name": "BBRAW727", "lang": "en", "location": "New York, NY 10018", "create_at": date("2014-09-15"), "description": "#NYC VersBottom wants to get fucked & bred can also breed holes too #BBBH #teambottom #cumwhore #cumslut #poppers #gaysex #gangbang #rawbottom #BareBbackPledge", "followers_count": 1029, "friends_count": 721, "statues_count": 6829 }, "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("2015-11-24T10:00:24.000Z"), "id": 669213996784816129, "text": "Thursday-Sunday������������#BR #NOLA https://t.co/kipJNcyy20", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BR", "NOLA" }}, "user": { "id": 320066390, "name": "KillaKayy❤️", "screen_name": "YaBoy_LoveMeh", "lang": "en", "location": "Baton Rouge, La✈️", "create_at": date("2011-06-19"), "description": "null", "followers_count": 1542, "friends_count": 1207, "statues_count": 56463 }, "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("2015-11-24T10:00:24.000Z"), "id": 669213996847788032, "text": "We're #hiring! Read about our latest #job opening here: Registered Nurse-CSOT1 Float Pool SLH ICU FT nights... - https://t.co/UcY9BP66hQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.5998517,39.1041725"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job" }}, "user": { "id": 22664577, "name": "TMJ-KAN Nursing Jobs", "screen_name": "tmj_kan_nursing", "lang": "en", "location": "Kansas City, MO", "create_at": date("2009-03-03"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Kansas City, MO. Need help? Tweet us at @CareerArc!", "followers_count": 453, "friends_count": 302, "statues_count": 267 }, "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("2015-11-24T10:00:25.000Z"), "id": 669213997317517313, "text": "See our latest #Bridgewater, NJ #job and click to apply: Server - https://t.co/TwKC4WU7KF #Hospitality #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.6193168,40.5861219"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Bridgewater", "job", "Hospitality", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 22483422, "name": "TMJ-NJN HRTA Jobs", "screen_name": "tmj_njn_hrta", "lang": "en", "location": "Northern NJ", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Northern NJ. Need help? Tweet us at @CareerArc!", "followers_count": 394, "friends_count": 295, "statues_count": 687 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bridgewater, NJ", "id": "0036e10c05fcd097", "name": "Bridgewater", "place_type": "city", "bounding_box": rectangle("-74.673756,40.5739 -74.597615,40.636758") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34035, "countyName": "Somerset", "cityID": 3444430, "cityName": "Martinsville" } }
+{ "create_at": datetime("2015-11-24T10:00:25.000Z"), "id": 669213997569146880, "text": "They gone know we high!!! ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 441157727, "name": "the8THletter", "screen_name": "_JossySays", "lang": "en", "location": "null", "create_at": date("2011-12-19"), "description": "#TAMUC Instagram: _jossyyy #LongLiveClyde", "followers_count": 1557, "friends_count": 1226, "statues_count": 83095 }, "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("2015-11-24T10:00:25.000Z"), "id": 669213997611155456, "text": "Ghirardelli #BusinessMgmt #Job: Assistant General Manager - Anaheim (#Anaheim, CA) https://t.co/ahIzul8b0W #management #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.9145036,33.8352932"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "BusinessMgmt", "Job", "Anaheim", "management", "Jobs", "Hiring" }}, "user": { "id": 2228943546, "name": "Ghirardelli Careers", "screen_name": "ghirardellijobs", "lang": "en", "location": "null", "create_at": date("2013-12-03"), "description": "Ghirardelli invites you to explore our open #jobs and discover a place where your ideas come to life, as part of our delicious legacy.", "followers_count": 54, "friends_count": 1, "statues_count": 98 }, "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-24T10:00:25.000Z"), "id": 669213997703389184, "text": "Supplemental Health Care #Healthcare : PTA - Physical Therapist... (#NorthFalmouth, MA) https://t.co/BTKP1SVQCI https://t.co/oOaQbR0sST", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.630494,41.6445703"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "NorthFalmouth" }}, "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": 732, "friends_count": 1, "statues_count": 66040 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Falmouth, MA", "id": "38adb282483098f4", "name": "North Falmouth", "place_type": "city", "bounding_box": rectangle("-70.653592,41.616787 -70.602362,41.658538") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25001, "countyName": "Barnstable", "cityID": 2547730, "cityName": "North Falmouth" } }
+{ "create_at": datetime("2015-11-24T10:00:25.000Z"), "id": 669213997779038208, "text": "������ you grew into your head �� https://t.co/3PT0Lr8ioL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 241222988, "name": "kybow.", "screen_name": "_kissmyBowen", "lang": "en", "location": "null", "create_at": date("2011-01-21"), "description": "DM[V]. 757. VUU'18.", "followers_count": 1536, "friends_count": 857, "statues_count": 55359 }, "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("2015-11-24T10:00:25.000Z"), "id": 669213998013779968, "text": "Its just more incentive to just end careers at conference this year indoor and out lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388330729, "name": "King Maddi", "screen_name": "MuvaGoldieLocs", "lang": "en", "location": "On The Track", "create_at": date("2011-10-10"), "description": "IG: goldilocks_oru\nFollow me on IG! Started a new one!\na woman of substance a beauty & a beast", "followers_count": 1418, "friends_count": 1146, "statues_count": 77988 }, "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("2015-11-24T10:00:25.000Z"), "id": 669213998026461185, "text": "@inretrospekt let's do post-work drinks soon! x", "in_reply_to_status": 669207479792836608, "in_reply_to_user": 17324721, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17324721 }}, "user": { "id": 23710368, "name": "♡ J U S T I N E ♡", "screen_name": "iamjustine", "lang": "en", "location": "tokyo. LA. airplanes.", "create_at": date("2009-03-10"), "description": "@SESAC / @wearelyka / @kynmusic | all views expressed are my own", "followers_count": 847, "friends_count": 830, "statues_count": 8526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-24T10:00:25.000Z"), "id": 669213998152183808, "text": "Partly cloudy/wind this afternoon, high 61 (16 C). Low 51 (11 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26195145, "name": "Wichita Weather", "screen_name": "WichitaWeather", "lang": "en", "location": "Wichita, KS", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Wichita, KS", "followers_count": 1851, "friends_count": 88, "statues_count": 7624 }, "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("2015-11-24T10:00:25.000Z"), "id": 669213998802309121, "text": "#thisisrena will you follow me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thisisrena" }}, "user": { "id": 3168026600, "name": "asshh", "screen_name": "rejectsqueen", "lang": "en", "location": "rowyso j.21 | bae area", "create_at": date("2015-04-22"), "description": "i am bryana holly trash", "followers_count": 271, "friends_count": 212, "statues_count": 3496 }, "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("2015-11-24T10:00:25.000Z"), "id": 669213998810705920, "text": "Beulah Chase Dodge, an American Hero https://t.co/oI3MXMCtwH #Ebay #PhotoAlbums https://t.co/bKYQ352p5Y @ROC #HABD https://t.co/4ieLG6Igl9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "sl", "is_retweet": false, "hashtags": {{ "Ebay", "PhotoAlbums", "HABD" }}, "user_mentions": {{ 6100122 }}, "user": { "id": 19068720, "name": "Charles A. Tijou", "screen_name": "RareCat", "lang": "en", "location": "Rochester, NY", "create_at": date("2009-01-16"), "description": "Located on the southern shore of Lake Ontario, I am a Photographer/Writer/Collector/Artist/Carpenter/Sociologist/Historian/Curator/Treasure Hunter/BeachComber", "followers_count": 426529, "friends_count": 442153, "statues_count": 14873 }, "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("2015-11-24T10:00:25.000Z"), "id": 669213998840201216, "text": "pour the purple over ice", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1149842960, "name": "monica gordon", "screen_name": "monicaakathryn", "lang": "en", "location": ":-)))))", "create_at": date("2013-02-04"), "description": "bodyposi", "followers_count": 673, "friends_count": 586, "statues_count": 19681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portage, MI", "id": "3af1aa005c106143", "name": "Portage", "place_type": "city", "bounding_box": rectangle("-87.222662,41.536049 -85.491151,42.245344") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26077, "countyName": "Kalamazoo", "cityID": 2665560, "cityName": "Portage" } }
+{ "create_at": datetime("2015-11-24T10:00:25.000Z"), "id": 669213998978568193, "text": "@emoblues @rubikees @HarryThetech76 you lost me", "in_reply_to_status": 669213584304533505, "in_reply_to_user": 45050039, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 45050039, 2732519856, 1898903978 }}, "user": { "id": 560264230, "name": "Arnold Slaughter", "screen_name": "ArnoldSlaughter", "lang": "en", "location": "null", "create_at": date("2012-04-22"), "description": "null", "followers_count": 82, "friends_count": 277, "statues_count": 2005 }, "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("2015-11-24T10:00:25.000Z"), "id": 669213998982799362, "text": "Ain't to many real ones out there", "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": "TheDopeDealer", "screen_name": "MadamBoatswain", "lang": "en", "location": "Road to Riches", "create_at": date("2014-05-13"), "description": "Senior at #NCAT - #UNCG ✊ #Boatswain is the last name. -Future Social Worker #Nxlevel", "followers_count": 892, "friends_count": 685, "statues_count": 65151 }, "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("2015-11-24T10:00:25.000Z"), "id": 669213999288963072, "text": "@revnace @AmericanAir @MakeAWish that is a super nice ending to what has looked like an amazing trip m; glad you guys had so much fun", "in_reply_to_status": 669213674066853888, "in_reply_to_user": 15059871, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15059871, 22536055, 61903300 }}, "user": { "id": 34016235, "name": "Justin Edwards", "screen_name": "JustinGMUhokie", "lang": "en", "location": "Reston VA", "create_at": date("2009-04-21"), "description": "I am a compassionate and passionate person who loves the Lord and cares for the world. born in Roanoke Va live in Reston Va and am a proud GMU Alum (09)", "followers_count": 2469, "friends_count": 279, "statues_count": 13641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reston, VA", "id": "ec212eb1116b92e2", "name": "Reston", "place_type": "city", "bounding_box": rectangle("-77.393246,38.90861 -77.304868,39.002993") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5166672, "cityName": "Reston" } }
+{ "create_at": datetime("2015-11-24T10:00:25.000Z"), "id": 669213999364444164, "text": "@rschaub1 I'm up for Arena 2.0. Like Caleb Porter as well.", "in_reply_to_status": 669212179242065921, "in_reply_to_user": 11440472, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 11440472 }}, "user": { "id": 16307039, "name": "mgoblog", "screen_name": "mgoblog", "lang": "en", "location": "Ann Arbor", "create_at": date("2008-09-15"), "description": "I am from space. Blake O'Neill is my emperor. header via @DJPhotoVideo", "followers_count": 45830, "friends_count": 301, "statues_count": 23548 }, "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-24T10:00:25.000Z"), "id": 669213999611912192, "text": "\"must be nice\" no........ you don't understand fam", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 349074529, "name": "Monte' G.", "screen_name": "Supreme_Tay", "lang": "en", "location": "null", "create_at": date("2011-08-05"), "description": "19. steady thumbin no droughts", "followers_count": 6872, "friends_count": 2504, "statues_count": 191193 }, "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("2015-11-24T10:00:25.000Z"), "id": 669213999909593088, "text": "Clinical Resources #Nursing : PICC Registered Nurse - Infusion Nurse - Registered Nurse - RN -... (#Fresno, CA) https://t.co/FfergaQ9UP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.7723661,36.7477272"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "Fresno" }}, "user": { "id": 67911583, "name": "Clinical Resources ", "screen_name": "ClinRes", "lang": "en", "location": "Atlanta, Georgia", "create_at": date("2009-08-22"), "description": "A healthcare staffing and professional services firm operating nationwide! / RTs ≠ endorsements / Call us at 404-343-7227! View Jobs: http://t.co/1VWhFiBPTI", "followers_count": 331, "friends_count": 438, "statues_count": 2487 }, "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("2015-11-24T10:00:25.000Z"), "id": 669214000429850624, "text": "Holy hangover.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 248105566, "name": "Ashley Rachelle", "screen_name": "AshleyXoWalden", "lang": "en", "location": "cleveland", "create_at": date("2011-02-05"), "description": "CLEV! classy with a twist of hood. Cynophilist . My heart belongs to Ro. ❤️ 10/25/2015.", "followers_count": 2544, "friends_count": 651, "statues_count": 8021 }, "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("2015-11-24T10:00:25.000Z"), "id": 669214000438210560, "text": "2015 DFHCA 1st Team All-State: Ali McKay, Polytech", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user": { "id": 733491744, "name": "Spenser Tilus", "screen_name": "STilus47ABC", "lang": "en", "location": "Salisbury, Maryland", "create_at": date("2012-08-02"), "description": "They call me shirt pants. I rock J's. KCCO. I cover sports for 47ABC.", "followers_count": 1887, "friends_count": 611, "statues_count": 21773 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salisbury, MD", "id": "00caf39d503a84e9", "name": "Salisbury", "place_type": "city", "bounding_box": rectangle("-75.71412,38.28923 -75.487032,38.431613") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24045, "countyName": "Wicomico", "cityID": 2469925, "cityName": "Salisbury" } }
+{ "create_at": datetime("2015-11-24T10:00:25.000Z"), "id": 669214000450699264, "text": "This #Sales #job might be a great fit for you: Part Time Merchandiser - https://t.co/r7e6aZehed #Hope, AR #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.5915665,33.6670615"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "job", "Hope", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 22051831, "name": "TMJ-AR-US Sales Jobs", "screen_name": "tmj_AR_sales", "lang": "en", "location": "Arkansas", "create_at": date("2009-02-26"), "description": "Follow this account for geo-targeted Sales job tweets in Arkansas Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 470, "friends_count": 298, "statues_count": 139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hope, AR", "id": "0054ee34ca496f44", "name": "Hope", "place_type": "city", "bounding_box": rectangle("-93.634394,33.62624 -93.553212,33.697883") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5057, "countyName": "Hempstead", "cityID": 533190, "cityName": "Hope" } }
+{ "create_at": datetime("2015-11-24T10:00:25.000Z"), "id": 669214000488407040, "text": "@_ChristianH @KEY4__ lol on me u weird", "in_reply_to_status": 669080002869243904, "in_reply_to_user": 344946995, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 344946995, 2855357977 }}, "user": { "id": 1526814168, "name": "KingCallier", "screen_name": "kingcallier", "lang": "en", "location": "null", "create_at": date("2013-06-17"), "description": "St. John Bosco Student Athlete. #44 DE \nClass of 2017\nNew highlights ||||", "followers_count": 269, "friends_count": 247, "statues_count": 1328 }, "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("2015-11-24T10:00:25.000Z"), "id": 669214000534523904, "text": "When ur sat percentile for math is 37% loooool", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2212808785, "name": "Lid", "screen_name": "lidiyaam", "lang": "en", "location": "null", "create_at": date("2013-11-24"), "description": "it's not a phase mom", "followers_count": 241, "friends_count": 214, "statues_count": 1094 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-24T10:00:25.000Z"), "id": 669214000698097664, "text": "There is a GOD, GOP donors are trying to crush Trump, Gota love it. \nNow Trump can go independent and destroy the fag Republican party!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 612295538, "name": "Little Bennie", "screen_name": "DFWblues", "lang": "en", "location": "The Republic Of Texas", "create_at": date("2012-06-18"), "description": "Working Musician, Observer of Political & Social Networks, Strong Opinions on American and the Communist Left.", "followers_count": 136, "friends_count": 196, "statues_count": 171 }, "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("2015-11-24T10:00:25.000Z"), "id": 669214000740089856, "text": "Join the Robert Half Management Resources team! See our latest #Sales #job opening here: https://t.co/xlC7iUJtXw #RHMRJobs #Edmonton, AB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.4951707,53.5425267"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "job", "RHMRJobs", "Edmonton" }}, "user": { "id": 791930767, "name": "RHMR Jobs", "screen_name": "RHMRJobs", "lang": "en", "location": "null", "create_at": date("2012-08-30"), "description": "Robert Half Management Resources is the world's premier provider of senior-level accounting, finance & business systems professionals. See our newest jobs here!", "followers_count": 97, "friends_count": 8, "statues_count": 1058 }, "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("2015-11-24T10:00:25.000Z"), "id": 669214001029509120, "text": "@nflnetwork @NFLTotalAccess do you hang out wit the Toko #HalotiNgata? #AskMegatron #Random", "in_reply_to_status": 669213388371791872, "in_reply_to_user": 19362299, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HalotiNgata", "AskMegatron", "Random" }}, "user_mentions": {{ 19362299, 2788207308 }}, "user": { "id": 281887345, "name": "NESS_TONGAN", "screen_name": "02kolo", "lang": "en", "location": "84104", "create_at": date("2011-04-13"), "description": "Glendale Resident", "followers_count": 334, "friends_count": 301, "statues_count": 4483 }, "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("2015-11-24T10:00:25.000Z"), "id": 669214001125965824, "text": "Partly cloudy this afternoon, high 57 (14 C). Low 44 (7 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26199170, "name": "Omaha Weather", "screen_name": "OmahaWX", "lang": "en", "location": "Omaha, Nebraska", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Omaha, Nebraska", "followers_count": 2571, "friends_count": 88, "statues_count": 7617 }, "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("2015-11-24T10:00:25.000Z"), "id": 669214001184796673, "text": "Much needed R and R!! Much needed R and R!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.28147792,29.84003139"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 85652590, "name": "Caryn Black", "screen_name": "CarynMBlack", "lang": "en", "location": "Bucks County, PA", "create_at": date("2009-10-27"), "description": "REALTOR at Addison Wolfe Real Estate, New Hope, PA; CEO/Pres Global Referral Company 215-862-5500. Locally Known, Globally Connected! Opinions r my own.", "followers_count": 2198, "friends_count": 2252, "statues_count": 10347 }, "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", "cityID": 1262525, "cityName": "St. Augustine Beach" } }
+{ "create_at": datetime("2015-11-24T10:00:25.000Z"), "id": 669214001210007552, "text": "@Ljusha7 @62Roses @Ibnelkayeem @loyal_cupid @drouchemed58 @ranasilvia63 @Asian_Poetry @zbmkhan @GCHDutta @almohid https://t.co/Z0puEwI0YP", "in_reply_to_status": 669213884583161856, "in_reply_to_user": 48875310, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2965652843, 1409420528, 2963445490, 17308392, 2264139201, 2750799059, 1256913534, 3010331497, 1721848128, 140078994 }}, "user": { "id": 48875310, "name": "Alhaj Nasir Uddin", "screen_name": "MrNaassir", "lang": "en", "location": "New Yark City.New York. U.S.A.", "create_at": date("2009-06-19"), "description": "I am from Pakistan & U.S. Citizen. Love Photography, Flowers, Birds, Scenery & Nature.Please Type MrNaassir in Google & See my 16000 pictures Thanks. NO DM PLZ.", "followers_count": 5431, "friends_count": 2345, "statues_count": 134117 }, "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("2015-11-24T10:00:25.000Z"), "id": 669214001214021632, "text": "Last piece for ICB artist winter open studios is complete. 60\" x 60\" panel untitled from new Braided… https://t.co/viT2TgiQ9t", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.49981162,37.86886106"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2481299012, "name": "Brian Huber", "screen_name": "BrianHuberArt", "lang": "en", "location": "Sausalito Ca", "create_at": date("2014-05-06"), "description": "abstract artisit in Sausalito Ca", "followers_count": 14, "friends_count": 44, "statues_count": 143 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sausalito, CA", "id": "c7b003d493c9e5ee", "name": "Sausalito", "place_type": "city", "bounding_box": rectangle("-122.507276,37.843359 -122.477251,37.873266") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6041, "countyName": "Marin", "cityID": 670364, "cityName": "Sausalito" } }
+{ "create_at": datetime("2015-11-24T10:00:25.000Z"), "id": 669214001230934016, "text": "2 cool 4 you ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.58260604,28.42021932"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 544855668, "name": "re«puta»ción.", "screen_name": "Nataavasquez", "lang": "en", "location": "Bolivia", "create_at": date("2012-04-03"), "description": "Soy como Jenni Rivera con obesidad mórbida, traicionada de los hombres, de su familia y el corazón hecho pedazos.", "followers_count": 1991, "friends_count": 1349, "statues_count": 15836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Magic Kingdom® Park", "id": "07d9e39251c81002", "name": "Magic Kingdom® Park", "place_type": "poi", "bounding_box": rectangle("-81.5812241,28.4159729 -81.581224,28.415973") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1204150, "cityName": "Bay Lake" } }
+{ "create_at": datetime("2015-11-24T10:00:25.000Z"), "id": 669214001297883137, "text": "You would've thought something sincerely tragic happened today...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2352718898, "name": "Max Moore", "screen_name": "maxmoore2love69", "lang": "en", "location": "Maryville, MO", "create_at": date("2014-02-19"), "description": "Walnut Creek, CA native, now in Kansas City, MO Northwest Missouri State Football #69", "followers_count": 315, "friends_count": 301, "statues_count": 5595 }, "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("2015-11-24T10:00:25.000Z"), "id": 669214001415323648, "text": "@Teddy765_ foh man nobody listens to it and a fucking soundtrack bro seriously?", "in_reply_to_status": 669213800751603712, "in_reply_to_user": 1099686516, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1099686516 }}, "user": { "id": 270620112, "name": "Neddy Wap", "screen_name": "Ned_Flander_", "lang": "en", "location": "crosstown ", "create_at": date("2011-03-22"), "description": "She don't want pets but Ima dog yeah yeah.. #freetae", "followers_count": 1652, "friends_count": 986, "statues_count": 98551 }, "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("2015-11-24T10:00:26.000Z"), "id": 669214001746722817, "text": "@AiyanaBatiste no problem ����", "in_reply_to_status": 669213552658419712, "in_reply_to_user": 552486726, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 552486726 }}, "user": { "id": 2371473961, "name": "Trelon Smith", "screen_name": "borntowin28_", "lang": "en", "location": "Houston, TX", "create_at": date("2014-03-03"), "description": "IF YOU WANT TO BE SUCCESSFUL AS MUCH AS YOU WANT TO BREATH THEN YOU WILL BE SUCCESSFUL!! FREE MY COUSIN JT", "followers_count": 739, "friends_count": 681, "statues_count": 1773 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:26.000Z"), "id": 669214001918689280, "text": "So anytime I'm gone, you can listen to my voice and sing along ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2159812328, "name": "indra star", "screen_name": "casualindra", "lang": "en", "location": "null", "create_at": date("2013-10-27"), "description": "I'll be here when you decide to come home", "followers_count": 1862, "friends_count": 1892, "statues_count": 14474 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Billings, MT", "id": "662aa8db4557a744", "name": "Billings", "place_type": "city", "bounding_box": rectangle("-108.692983,45.723722 -108.432965,45.871169") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30111, "countyName": "Yellowstone", "cityID": 3006550, "cityName": "Billings" } }
+{ "create_at": datetime("2015-11-24T10:00:26.000Z"), "id": 669214002015178754, "text": "@samsernaxx girl your tweets on these fuck niggas on point �� keep preachin.", "in_reply_to_status": -1, "in_reply_to_user": 2162090437, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2162090437 }}, "user": { "id": 887898247, "name": "LIL $ENIAAA", "screen_name": "Babysenia17", "lang": "en", "location": "In the wrong generation ‼️", "create_at": date("2012-10-17"), "description": "Crazy mother fucker I AM ONE #OTF All the things that we accept, Be the things that we regret", "followers_count": 1946, "friends_count": 913, "statues_count": 63837 }, "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("2015-11-24T10:00:26.000Z"), "id": 669214002119970820, "text": "Interested in a #Transportation #job near #Reno, NV? This could be a great fit: https://t.co/7jaBM9mvbe #KellyJobs #KellyServices #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.8236554,39.4953787"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Transportation", "job", "Reno", "KellyJobs", "KellyServices", "Hiring" }}, "user": { "id": 149759873, "name": "TMJ-NVR Transport.", "screen_name": "tmj_NVR_transp", "lang": "en", "location": "Reno, NV", "create_at": date("2010-05-29"), "description": "Follow this account for geo-targeted Transportation job tweets in Reno, NV from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 278, "friends_count": 274, "statues_count": 30 }, "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("2015-11-24T10:00:26.000Z"), "id": 669214002539421696, "text": "@chriscamacho_4 @FernTooFly bawmmm", "in_reply_to_status": 669107799410896896, "in_reply_to_user": 961828232, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 961828232, 2178502046 }}, "user": { "id": 1152986696, "name": "ㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤ", "screen_name": "HOMERRKINS", "lang": "en", "location": "IG: @HOMERRKINS", "create_at": date("2013-02-05"), "description": "you aint livin right", "followers_count": 1105, "friends_count": 370, "statues_count": 120150 }, "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("2015-11-24T10:00:26.000Z"), "id": 669214002824609794, "text": "Wildcat family portrait or new bookstore merchandise?! @PiperKellyAZ @UAAA @UAAlumniPres @WFL_Jill https://t.co/55oqEBvffv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 616325791, 50367928, 328258849, 157430810 }}, "user": { "id": 354738983, "name": "Marc Acuna", "screen_name": "Wildcat_Culture", "lang": "en", "location": "AZ", "create_at": date("2011-08-13"), "description": "Director: Chapters, Clubs & Student Relations\nUniversity of Arizona Alumni Association", "followers_count": 451, "friends_count": 257, "statues_count": 719 }, "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-24T10:00:26.000Z"), "id": 669214002841456640, "text": "Interested in a #Manufacturing #job near #Dundee, MI? This could be a great fit: https://t.co/qDXdqzVAqo #Veterans #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.6596596,41.9572676"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Manufacturing", "job", "Dundee", "Veterans", "Hiring" }}, "user": { "id": 1157303605, "name": "Plastipak Jobs", "screen_name": "PlastipakJobs", "lang": "en", "location": "null", "create_at": date("2013-02-07"), "description": "Plastipak Packaging is an industry leader in the design and manufacturing of plastic rigid containers of the highest quality.", "followers_count": 233, "friends_count": 30, "statues_count": 6410 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dundee, MI", "id": "37a31947cd497995", "name": "Dundee", "place_type": "city", "bounding_box": rectangle("-83.683303,41.94448 -83.634328,41.979501") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26115, "countyName": "Monroe", "cityID": 2623380, "cityName": "Dundee" } }
+{ "create_at": datetime("2015-11-24T10:00:26.000Z"), "id": 669214002862383105, "text": ".@KREMNicolle! Have you see this breaking news story?! https://t.co/hZT6HvIztC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 348178323 }}, "user": { "id": 283293114, "name": "Jen York", "screen_name": "KREMjen", "lang": "en", "location": "Spokane, WA", "create_at": date("2011-04-16"), "description": "KREM2 Anchor/Producer|WSU Alumna|PNW Native|Αφ Sister|Lover of Music, Cats & Tacos|Go Cougs!", "followers_count": 6704, "friends_count": 4471, "statues_count": 13169 }, "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("2015-11-24T10:00:26.000Z"), "id": 669214002912751616, "text": "To the beautiful boy sitting next to me in the library right now I just wanna say hi ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 768592416, "name": "that-one-girl", "screen_name": "Amazingrace919", "lang": "en", "location": "depens, who wants to know?", "create_at": date("2012-08-19"), "description": "so we vibing over our hoeism or whatever IG: godsdancingrace", "followers_count": 412, "friends_count": 431, "statues_count": 16937 }, "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("2015-11-24T10:00:26.000Z"), "id": 669214002967252992, "text": "We're #hiring! Click to apply: Senior Java Software Engineer (Maintenance) - https://t.co/eVwh9JZ6KS #IT #Bedford, MA #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.2761686,42.4906499"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "IT", "Bedford", "Job", "Jobs" }}, "user": { "id": 20831491, "name": "Boston IT Jobs", "screen_name": "tmj_bos_it", "lang": "en", "location": "Boston, MA", "create_at": date("2009-02-13"), "description": "Follow this account for geo-targeted Software Dev. - General/IT job tweets in Boston, MA. Need help? Tweet us at @CareerArc!", "followers_count": 809, "friends_count": 390, "statues_count": 522 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bedford, MA", "id": "0046deea4f1868d3", "name": "Bedford", "place_type": "city", "bounding_box": rectangle("-71.321045,42.444961 -71.231728,42.526626") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex" } }
+{ "create_at": datetime("2015-11-24T10:00:26.000Z"), "id": 669214002967277568, "text": "เธอๆ....เรายังไม่มีคู่ลอยกระทงเลย #กระทงมึงหนัก800กิโลหรอถึงได้ช่วยกันถือ CR.twitter. #ทีมโสดแล้วพาล https://t.co/4LZnfK48Gt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "th", "is_retweet": false, "hashtags": {{ "กระทงมึงหนัก800กิโลหรอถึงได้ช่วยกันถือ", "ทีมโสดแล้วพาล" }}, "user": { "id": 823108188, "name": "FlukieZZ", "screen_name": "Flukiezzz", "lang": "th", "location": "chonburi,bangkok", "create_at": date("2012-09-14"), "description": "ชื่อฟลุค :)", "followers_count": 254, "friends_count": 310, "statues_count": 24663 }, "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("2015-11-24T10:00:26.000Z"), "id": 669214003554443264, "text": "Interested in a #Manufacturing #job near #Portland, OR? This could be a great fit: https://t.co/7r6hU7jydj #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.6762071,45.5234515"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Manufacturing", "job", "Portland", "Hiring" }}, "user": { "id": 21686716, "name": "TMJ - PDX Manuf Jobs", "screen_name": "tmj_pdx_manuf", "lang": "en", "location": "Portland, OR", "create_at": date("2009-02-23"), "description": "Follow this account for geo-targeted Manufacturing job tweets in Portland, OR from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 301, "friends_count": 202, "statues_count": 121 }, "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("2015-11-24T10:00:26.000Z"), "id": 669214004036829184, "text": "@GraysonVanMeter lmaooo https://t.co/IDJSnA5ki7", "in_reply_to_status": -1, "in_reply_to_user": 350603190, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 350603190 }}, "user": { "id": 61285547, "name": "Deanna Rena.", "screen_name": "SayWeChoosin", "lang": "en", "location": "HOU ✈️ ville ", "create_at": date("2009-07-29"), "description": "Deanna Rena . SHSU19. we walk by faith not by sight 2 Cor 5:7 . Keel. Future Family Attorney", "followers_count": 978, "friends_count": 922, "statues_count": 14576 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:26.000Z"), "id": 669214004129042433, "text": "Partly cloudy this afternoon, high 61 (16 C). Low 39 (4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433220, "name": "St. Louis Weather", "screen_name": "StLouisWeather", "lang": "en", "location": "St. Louis Metro Area", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for St. Louis Metro Area", "followers_count": 3675, "friends_count": 88, "statues_count": 7590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "St. Louis, MO", "id": "60e6df5778ff9dac", "name": "St. Louis", "place_type": "city", "bounding_box": rectangle("-90.320515,38.531852 -90.166409,38.774346") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29510, "countyName": "St. Louis", "cityID": 2965000, "cityName": "St. Louis" } }
+{ "create_at": datetime("2015-11-24T10:00:26.000Z"), "id": 669214004246650881, "text": "Real Nigga Salute\nHappy Birthday T! https://t.co/V7PMbBT5CN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 582080480, "name": "pinkdolphinMakonnen", "screen_name": "kikiyyole", "lang": "en", "location": "null", "create_at": date("2012-05-16"), "description": "null", "followers_count": 836, "friends_count": 580, "statues_count": 8789 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Daytona, FL", "id": "66bc82f162d83f01", "name": "South Daytona", "place_type": "city", "bounding_box": rectangle("-81.029402,29.148366 -80.989419,29.185724") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1267325, "cityName": "South Daytona" } }
+{ "create_at": datetime("2015-11-24T10:00:26.000Z"), "id": 669214004527546368, "text": "@Twinky_Pinedo @JPinedo01 @adan_3000 @rafaelrubio75 the next champ. ����", "in_reply_to_status": 669213757055242240, "in_reply_to_user": 1144225076, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1144225076, 3542642054, 2347880203, 1368480528 }}, "user": { "id": 1144211046, "name": "Cisco", "screen_name": "CiscoAntrax", "lang": "en", "location": "Stary Oskol, Russia", "create_at": date("2013-02-02"), "description": "The Last Russian Emperor", "followers_count": 125, "friends_count": 105, "statues_count": 303 }, "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("2015-11-24T10:00:26.000Z"), "id": 669214004665946112, "text": "I need a blunt or two and someone to cuddle with", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 178962272, "name": "m.", "screen_name": "xoxo_bmichi", "lang": "en", "location": "Rancho Cucamonga, CA", "create_at": date("2010-08-15"), "description": "Por las buenas soy un ángel y por las malas mejor ni les cuento.", "followers_count": 1794, "friends_count": 786, "statues_count": 216585 }, "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("2015-11-24T10:00:26.000Z"), "id": 669214005030813696, "text": "hmmm, seem jealous��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1355272627, "name": "allie", "screen_name": "alhudge", "lang": "en", "location": "null", "create_at": date("2013-04-15"), "description": "Magnolia, Texas", "followers_count": 282, "friends_count": 103, "statues_count": 798 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stagecoach, TX", "id": "0162afbe0e5cbfab", "name": "Stagecoach", "place_type": "city", "bounding_box": rectangle("-95.804079,30.102712 -95.682507,30.157468") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48339, "countyName": "Montgomery", "cityID": 4869932, "cityName": "Stagecoach" } }
+{ "create_at": datetime("2015-11-24T10:00:26.000Z"), "id": 669214005337169920, "text": "Appreciate it bro.✊�� https://t.co/8FbdqMOWkq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3282683210, "name": "Jordan moore‼️", "screen_name": "Jo_moore2", "lang": "en", "location": "null", "create_at": date("2015-07-17"), "description": "6'1 183|yoakum,tx| sophmore| d1bound|", "followers_count": 679, "friends_count": 1005, "statues_count": 1209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yoakum, TX", "id": "a3f033ff4255c12d", "name": "Yoakum", "place_type": "city", "bounding_box": rectangle("-97.164826,29.270832 -97.12689,29.319944") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48285, "countyName": "Lavaca", "cityID": 4880560, "cityName": "Yoakum" } }
+{ "create_at": datetime("2015-11-24T10:00:26.000Z"), "id": 669214005479673858, "text": "1 Corinthians 15:57 \nBut thanks be to God, who gives us the victory through our Lord 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": 827925326, "name": "taryn ツ", "screen_name": "TarynDeshay", "lang": "en", "location": "Pineville, LA", "create_at": date("2012-09-16"), "description": "@LSUsoftball commit | @TexasGlory Shelton 18U Gold | psalm 3:5 | alexandria, la | hype out | eat a @snickers", "followers_count": 1015, "friends_count": 465, "statues_count": 23404 }, "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("2015-11-24T10:00:26.000Z"), "id": 669214005517418496, "text": "@__ytxx__ ������ I'm already one", "in_reply_to_status": 669211445897375744, "in_reply_to_user": 3390508499, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3390508499 }}, "user": { "id": 197259280, "name": "nadeah", "screen_name": "Massive_Beautyy", "lang": "en", "location": "null", "create_at": date("2010-09-30"), "description": "null", "followers_count": 755, "friends_count": 427, "statues_count": 127505 }, "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-24T10:00:26.000Z"), "id": 669214005559435265, "text": "Nothing soothes the stress of traveling like my mother calling me every 15 minutes to confirm that I'm still not on the plane yet.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15883010, "name": "~*TRANNIKA*REX*~", "screen_name": "TrannikaRex", "lang": "en", "location": "Chicago, IL", "create_at": date("2008-08-17"), "description": "I create strange drag shows in Chicago.", "followers_count": 3739, "friends_count": 642, "statues_count": 18113 }, "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-24T10:00:27.000Z"), "id": 669214005676888065, "text": "Happy birthday @ashli_sheldon love you �� https://t.co/MYX9BZ4jbg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 567746983 }}, "user": { "id": 509668333, "name": "Taylor Foster", "screen_name": "T_foster32", "lang": "en", "location": "null", "create_at": date("2012-02-29"), "description": "Burnin' it down", "followers_count": 532, "friends_count": 508, "statues_count": 6521 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milford, OH", "id": "007bd79948b9a99c", "name": "Milford", "place_type": "city", "bounding_box": rectangle("-84.302499,39.151726 -84.234182,39.21376") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39025, "countyName": "Clermont", "cityID": 3950176, "cityName": "Milford" } }
+{ "create_at": datetime("2015-11-24T10:00:27.000Z"), "id": 669214005823586304, "text": "@aprilshanae Lol for real man, I live by my ain't shit name I'm not changing fo y'all lol", "in_reply_to_status": 669213810796859393, "in_reply_to_user": 1200146114, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1200146114 }}, "user": { "id": 1894510596, "name": "JIM BOB COOTER", "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": 1359, "friends_count": 2025, "statues_count": 115129 }, "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("2015-11-24T10:00:27.000Z"), "id": 669214006062809088, "text": "@Hayley9255 ffs how late were you guys out", "in_reply_to_status": 669207776502091782, "in_reply_to_user": 478150137, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 478150137 }}, "user": { "id": 1921778940, "name": "Alex nowacki", "screen_name": "alpalnowacki", "lang": "en", "location": "waterford, wi", "create_at": date("2013-09-30"), "description": "Like to enjoy the finer things in life, lol jk im poor as hell", "followers_count": 189, "friends_count": 147, "statues_count": 4083 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mukwonago, WI", "id": "dd2ba781cc3ed3fd", "name": "Mukwonago", "place_type": "city", "bounding_box": rectangle("-88.365658,42.838769 -88.291655,42.893104") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55133, "countyName": "Waukesha", "cityID": 5555050, "cityName": "Mukwonago" } }
+{ "create_at": datetime("2015-11-24T10:00:27.000Z"), "id": 669214006142439424, "text": "@PhilaBCoulter Yep", "in_reply_to_status": 669213958809722880, "in_reply_to_user": 273658581, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 273658581 }}, "user": { "id": 19901033, "name": "Brother Fray", "screen_name": "TheFrayMovement", "lang": "en", "location": "Mobile, AL", "create_at": date("2009-02-02"), "description": "Connect the dots. Ask questions. Trust the process. Let it play out. Either you die a work or live long enough to become a shoot. FIRED UP!", "followers_count": 1278, "friends_count": 374, "statues_count": 131679 }, "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("2015-11-24T10:00:27.000Z"), "id": 669214006155038720, "text": "@CLWPublicSafety Thank you for catching the guy that MURDERED Anthony DiRienzo!!! Make sure he RECEIVES DEATH for his crime!!!", "in_reply_to_status": 667807931996372992, "in_reply_to_user": 618161170, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 618161170 }}, "user": { "id": 2978088017, "name": "Michelle Szal", "screen_name": "6f5d9e223de844c", "lang": "en", "location": "null", "create_at": date("2015-01-14"), "description": "null", "followers_count": 6, "friends_count": 62, "statues_count": 78 }, "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("2015-11-24T10:00:27.000Z"), "id": 669214006167515136, "text": "Thank you �� https://t.co/hDvTNa88tr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 399862957, "name": "Young JFK.", "screen_name": "GetLikeJon", "lang": "en", "location": "null", "create_at": date("2011-10-27"), "description": "Watching Knicks or Duke. #BlackLivesMatter", "followers_count": 1621, "friends_count": 955, "statues_count": 55532 }, "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("2015-11-24T10:00:27.000Z"), "id": 669214006314336256, "text": "Dude that picture always makes me laugh ���� well when you try to hard ���� 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": 584405900, "name": "Rich_Homie_Chris", "screen_name": "TortuqasChris", "lang": "en", "location": "South Central L.A ", "create_at": date("2012-05-18"), "description": "Please Hold, Currently Trapping...", "followers_count": 125, "friends_count": 170, "statues_count": 7814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-24T10:00:27.000Z"), "id": 669214006368841728, "text": "@robertcramb no one else been fucking w him I know so it's nice to see someone else listening to him", "in_reply_to_status": 669213474845794304, "in_reply_to_user": 77652342, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 77652342 }}, "user": { "id": 218681662, "name": "Johnny Manziel", "screen_name": "BenE_VN", "lang": "en", "location": "Kokomo", "create_at": date("2010-11-22"), "description": "From Washington & somehow ended up Idaho | Ain't no fun if the homies cant have none | Team Rite Aid | sc: benevensen | ig: idaho_of_300 | 10/05/15 @mcburney27", "followers_count": 668, "friends_count": 293, "statues_count": 32147 }, "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("2015-11-24T10:00:27.000Z"), "id": 669214006809403392, "text": "Atleast My Mom Picking 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": 467620484, "name": "Mrs. H❤️", "screen_name": "MeMyself_Chey", "lang": "en", "location": "North Philly", "create_at": date("2012-01-18"), "description": "March 25th❤️ K.M.A ❤️ FREE SHIZZ ‼️", "followers_count": 1168, "friends_count": 836, "statues_count": 62831 }, "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-24T10:00:27.000Z"), "id": 669214007136391168, "text": "Mostly sunny this afternoon, high 61 (16 C). Low 49 (9 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433608, "name": "Kansas City Weather", "screen_name": "KansasCityWX", "lang": "en", "location": "Kansas City Metro Area", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Kansas City Metro Area", "followers_count": 1921, "friends_count": 88, "statues_count": 7710 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "ddabe75a8f44fe5a", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.765917,38.827176 -94.385522,39.356662") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2015-11-24T10:00:27.000Z"), "id": 669214007220277248, "text": "Matsunaga put someone's head in a vise wrapped in barbed wire once\n\nMr Danger doesn't fuck around sirs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 23603190, "name": "Casey Nielsen", "screen_name": "luchagringo", "lang": "en", "location": "Hollywood, CA", "create_at": date("2009-03-10"), "description": "Screenwriting Ninja Rudo", "followers_count": 386, "friends_count": 266, "statues_count": 6531 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Castaic, CA", "id": "01430bf766a1d670", "name": "Castaic", "place_type": "city", "bounding_box": rectangle("-118.651842,34.440304 -118.610385,34.516522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611796, "cityName": "Castaic" } }
+{ "create_at": datetime("2015-11-24T10:00:27.000Z"), "id": 669214007408984064, "text": "Next year will bring. New changes !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2155941722, "name": "Josiah.", "screen_name": "DashTooFastt", "lang": "en", "location": "null", "create_at": date("2013-10-25"), "description": "It is better to be hated for what you are than to be loved for what you are not.", "followers_count": 269, "friends_count": 293, "statues_count": 3947 }, "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("2015-11-24T10:00:27.000Z"), "id": 669214007660666880, "text": "@CantCurveKirby happy birthday ��", "in_reply_to_status": -1, "in_reply_to_user": 2837574199, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2837574199 }}, "user": { "id": 528664485, "name": "$andy $hmurda", "screen_name": "Sandyy323", "lang": "en", "location": "South Central LA➡️Agg➡️Htown", "create_at": date("2012-03-18"), "description": "jiggy. spoiled little LA girl. #PettyHive ✨Kolby's lil burrito✨ Dm's broke.", "followers_count": 1153, "friends_count": 683, "statues_count": 44953 }, "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("2015-11-24T10:00:27.000Z"), "id": 669214007690039296, "text": "@TheFunnyVines @JaredMonhaut", "in_reply_to_status": 669196521800204289, "in_reply_to_user": 612009655, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 612009655, 1023173161 }}, "user": { "id": 591495720, "name": "Bud Fox", "screen_name": "JackNagy259", "lang": "en", "location": "Ball State", "create_at": date("2012-05-26"), "description": "Fake it till you make it. Alpha Tau Omega.", "followers_count": 334, "friends_count": 278, "statues_count": 8873 }, "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("2015-11-24T10:00:27.000Z"), "id": 669214007832608768, "text": "@GirlCode @MTV2GuyCode & Jesus turned a prostitute into a Saint & rightly true if she's healing or giving love to the disparaged", "in_reply_to_status": 669213767041875968, "in_reply_to_user": 41692977, "favorite_count": 0, "coordinate": point("-120.4897618,38.1399568"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1263465577, 523588327 }}, "user": { "id": 41692977, "name": "zoomtunes.net", "screen_name": "zoomtunes_net", "lang": "en", "location": "Sierra foothills of California", "create_at": date("2009-05-21"), "description": "I wont allow boredom which compelled me to get horny / creative", "followers_count": 660, "friends_count": 1382, "statues_count": 72832 }, "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": 6109, "countyName": "Tuolumne" } }
+{ "create_at": datetime("2015-11-24T10:00:27.000Z"), "id": 669214007941820416, "text": "@DavidCutcliffe Check out my Sophomore Year Highlights: https://t.co/JFYVn8GCjz", "in_reply_to_status": -1, "in_reply_to_user": 21335124, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21335124 }}, "user": { "id": 4255131496, "name": "Deamondre Goodwin", "screen_name": "deamondre_g7", "lang": "en", "location": "North Carolina, USA", "create_at": date("2015-11-22"), "description": "2018 ATH.... THE GRIND DOESN'T STOP #POFG", "followers_count": 7, "friends_count": 38, "statues_count": 28 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Airy, NC", "id": "46b149b6aa3775a2", "name": "Mount Airy", "place_type": "city", "bounding_box": rectangle("-80.671877,36.464892 -80.579574,36.560353") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37171, "countyName": "Surry", "cityID": 3744800, "cityName": "Mount Airy" } }
+{ "create_at": datetime("2015-11-24T10:00:27.000Z"), "id": 669214008063365121, "text": "Want to work in #Miami, FL? View our latest opening: https://t.co/9AvqeDf6Jx #Hospitality #Veterans #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.3809477,25.7869078"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Miami", "Hospitality", "Veterans", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22537143, "name": "TMJ-MIA HRTA Jobs", "screen_name": "tmj_mia_hrta", "lang": "en", "location": "Miami, FL", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Miami, FL. Need help? Tweet us at @CareerArc!", "followers_count": 419, "friends_count": 292, "statues_count": 710 }, "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("2015-11-24T10:00:27.000Z"), "id": 669214008210272257, "text": "tempted to request $1 from all my contacts in venmo cause if they all said yes I'd make like 500 bucks I could be a genius stay tuned", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 258024469, "name": "Chloe", "screen_name": "krank_daddy", "lang": "en", "location": "Skeeps", "create_at": date("2011-02-26"), "description": "*screams internally*", "followers_count": 227, "friends_count": 234, "statues_count": 8507 }, "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-24T10:00:27.000Z"), "id": 669214008268886016, "text": "This #Nursing #job might be a great fit for you: Resident Associate - https://t.co/IycnKgGXJ8 #STNA #training #Beachwood, OH #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4899224,41.5011199"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "STNA", "training", "Beachwood", "Hiring" }}, "user": { "id": 214122801, "name": "MenorahParkJobs", "screen_name": "MenorahParkJobs", "lang": "en", "location": "Beachwood, Ohio", "create_at": date("2010-11-10"), "description": "Menorah Park Center for Senior Living is one of the largest senior living communities in the state of Ohio.", "followers_count": 127, "friends_count": 82, "statues_count": 1378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beachwood, OH", "id": "fa5bd676191716ee", "name": "Beachwood", "place_type": "city", "bounding_box": rectangle("-81.532635,41.449669 -81.48037,41.501448") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3904500, "cityName": "Beachwood" } }
+{ "create_at": datetime("2015-11-24T10:00:27.000Z"), "id": 669214008315088896, "text": "You want me to talk and greet people on pain medication �� lol k", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 545647665, "name": "地下女王", "screen_name": "McKenzie81000", "lang": "en", "location": "null", "create_at": date("2012-04-04"), "description": "|Love it live your life make it lavish long |. HSE Varsity soccer #10 | R.I.P Vanessa 4/19/15| instagram: s_mckenzie10 | Chicago State University commit D1⚽️", "followers_count": 405, "friends_count": 386, "statues_count": 8809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Noblesville, IN", "id": "013e3bc05a18abdc", "name": "Noblesville", "place_type": "city", "bounding_box": rectangle("-86.090354,39.993436 -85.932944,40.10716") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18057, "countyName": "Hamilton", "cityID": 1854180, "cityName": "Noblesville" } }
+{ "create_at": datetime("2015-11-24T10:00:27.000Z"), "id": 669214008440832000, "text": "Tito.. Finished and sold. No fucking filter like my dumbass selfies... �� https://t.co/77o20A7lFo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 352317165, "name": "Real One", "screen_name": "JohnLannom", "lang": "en", "location": "Des Moines, IA", "create_at": date("2011-08-10"), "description": "I am: •an artist • a humanitarian •a world traveller •a graffiti writer •single •evolving artistically •taking on commissions", "followers_count": 3, "friends_count": 44, "statues_count": 21 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Urbandale, IA", "id": "85ec017be8926e26", "name": "Urbandale", "place_type": "city", "bounding_box": rectangle("-93.814689,41.614643 -93.697511,41.658733") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1979950, "cityName": "Urbandale" } }
+{ "create_at": datetime("2015-11-24T10:00:27.000Z"), "id": 669214008482754560, "text": "Wtffff ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 826385611, "name": "Myarandaa", "screen_name": "myarandaa", "lang": "en", "location": "livin' lawless", "create_at": date("2012-09-15"), "description": "@AugustAlsina ❤️", "followers_count": 834, "friends_count": 511, "statues_count": 55418 }, "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("2015-11-24T10:00:27.000Z"), "id": 669214008537423872, "text": "John Locke, John Nash and Stephen Hawkings.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.1118914,38.728095"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 282292991, "name": "Mr. Awesome", "screen_name": "PatrickSedjro", "lang": "en", "location": "United States", "create_at": date("2011-04-14"), "description": "live the life your 12 year old self would be proud of.", "followers_count": 257, "friends_count": 231, "statues_count": 7044 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Vernon, VA", "id": "6be71d3134ce7613", "name": "Mount Vernon", "place_type": "city", "bounding_box": rectangle("-77.140882,38.697083 -77.072778,38.747275") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5154144, "cityName": "Mount Vernon" } }
+{ "create_at": datetime("2015-11-24T10:00:27.000Z"), "id": 669214008788955136, "text": "Stars have Benn straight �� this year!! See what I did there�� #LoneStarsState https://t.co/zOWeWNoggA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LoneStarsState" }}, "user": { "id": 347286412, "name": "MFFL", "screen_name": "wesakadirkjr", "lang": "en", "location": "Dallas, Texas", "create_at": date("2011-08-02"), "description": "#MFFL #Mavsnation 17 Don't give up, don't ever give up #UARK19 #Patriots Young Life Rocks!!! Thunder Duck 4/18/97 Not that it matters, but #Single", "followers_count": 197, "friends_count": 466, "statues_count": 6688 }, "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("2015-11-24T10:00:27.000Z"), "id": 669214009304997888, "text": "If climate change is fueling terrorists then why is Obama using Jets instead of Scientists to kill terrorists?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2663115926, "name": "Count Down to 2016", "screen_name": "bad_robot_57", "lang": "en", "location": "null", "create_at": date("2014-07-20"), "description": "Opinions expressed herein are solely my own and may or may not reflect my opinion at this particular time or any other.", "followers_count": 2096, "friends_count": 1880, "statues_count": 49841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sharonville, OH", "id": "f4f98739e07e926f", "name": "Sharonville", "place_type": "city", "bounding_box": rectangle("-84.449769,39.254798 -84.352567,39.304105") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3971892, "cityName": "Sharonville" } }
+{ "create_at": datetime("2015-11-24T10:00:27.000Z"), "id": 669214009351135232, "text": "Drinking a Ranger by @newbelgium @ Flying Saucer — https://t.co/kX4cZ1cBw7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.7938,35.1637"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18057459 }}, "user": { "id": 2910256516, "name": "Frank Urankar II", "screen_name": "Blackhammer1999", "lang": "en", "location": "Arlington, TN", "create_at": date("2014-12-07"), "description": "Former Pro-Wrestler, Current Firefighter/Paramedic and all around craft beer lover.", "followers_count": 34, "friends_count": 80, "statues_count": 1687 }, "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("2015-11-24T10:00:27.000Z"), "id": 669214009422422016, "text": "@Alewidit can obama just stay another term ? Lol", "in_reply_to_status": 669213733508399104, "in_reply_to_user": 2217198132, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2217198132 }}, "user": { "id": 301265642, "name": "Mike", "screen_name": "mikeso_fresh", "lang": "en", "location": "$F Cali", "create_at": date("2011-05-18"), "description": "ig: mikeits415", "followers_count": 364, "friends_count": 192, "statues_count": 21817 }, "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-24T10:00:27.000Z"), "id": 669214009590067201, "text": "@RoboMajorVibez har inte kollat. vill inte kolla. finns svenska pepparkakor, för det äts året runt här.", "in_reply_to_status": 669213693004132353, "in_reply_to_user": 468189884, "favorite_count": 0, "retweet_count": 0, "lang": "sv", "is_retweet": false, "user_mentions": {{ 468189884 }}, "user": { "id": 339045629, "name": "RapGameLiberace", "screen_name": "Queen0fS", "lang": "en", "location": "Sweden", "create_at": date("2011-07-20"), "description": "Peppis travels= USA➡UK➡SE➡GE ➡SE Asia➡Japan➡USA. Well mannered frivolity. Random white chick be everywhere. Attitude & heart. IG: @peppisspade #vikingsNFL", "followers_count": 124, "friends_count": 238, "statues_count": 19936 }, "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("2015-11-24T10:00:28.000Z"), "id": 669214010013696000, "text": "@AllThatandMoore @KimballBarker", "in_reply_to_status": 669188167413665793, "in_reply_to_user": 157517507, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 157517507, 1249927615 }}, "user": { "id": 389563266, "name": "Coleman Eddie", "screen_name": "colemanedwards1", "lang": "en", "location": "PRovo", "create_at": date("2011-10-12"), "description": "You gotta flip it, stick it, see ya lata bye", "followers_count": 252, "friends_count": 379, "statues_count": 1343 }, "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("2015-11-24T10:00:28.000Z"), "id": 669214010059919362, "text": "S\\o to my new follower @AWayWithWords__", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 521892896 }}, "user": { "id": 327034401, "name": "11\\20", "screen_name": "PlummStupidJR", "lang": "en", "location": "985 - 601", "create_at": date("2011-06-30"), "description": "null", "followers_count": 2638, "friends_count": 2773, "statues_count": 70201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houma, LA", "id": "2a4c12f527197dab", "name": "Houma", "place_type": "city", "bounding_box": rectangle("-90.779993,29.544349 -90.649261,29.61995") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22109, "countyName": "Terrebonne", "cityID": 2236255, "cityName": "Houma" } }
+{ "create_at": datetime("2015-11-24T10:00:28.000Z"), "id": 669214010080792577, "text": "Partly cloudy this afternoon, high 62 (17 C). Low 41 (5 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433972, "name": "Memphis Weather", "screen_name": "MemphisWX", "lang": "en", "location": "Memphis, TN", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Memphis, TN", "followers_count": 1687, "friends_count": 88, "statues_count": 7622 }, "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("2015-11-24T10:00:28.000Z"), "id": 669214010563100672, "text": "What you done did https://t.co/SlBNodz95u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 258641669, "name": "chika in da cut!", "screen_name": "UmeniynoraDaII", "lang": "en", "location": "si track ka ájá igbe", "create_at": date("2011-02-27"), "description": "walking on faith.", "followers_count": 1421, "friends_count": 937, "statues_count": 151718 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:28.000Z"), "id": 669214010894520321, "text": "#Nursing in #Monterey, CA: Infection Prevention Coord-Quality Mgmt at Community Hospital of the Monter... https://t.co/JZDykDEkr0 #ORjobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.9131935,36.5787155"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "Monterey", "ORjobs" }}, "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": 52, "friends_count": 27, "statues_count": 48 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monterey, CA", "id": "49af5b43d4963f4c", "name": "Monterey", "place_type": "city", "bounding_box": rectangle("-121.927024,36.573741 -121.830974,36.618826") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 648872, "cityName": "Monterey" } }
+{ "create_at": datetime("2015-11-24T10:00:28.000Z"), "id": 669214010932269056, "text": "Miami Children's Health System: Echocardiograph Tech (Full-time, Days) (#Miami, FL) https://t.co/NbCTvyJs2U #Healthcare #pediatrics #Job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.2940652,25.7414669"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Miami", "Healthcare", "pediatrics", "Job" }}, "user": { "id": 42622658, "name": "Miami Childrens Jobs", "screen_name": "MCHCareers", "lang": "en", "location": "Miami, Florida", "create_at": date("2009-05-26"), "description": "Nicklaus Children’s Hospital, formerly Miami Children's Hospital, is S.Fla’s only licensed pediatric specialty hospital.", "followers_count": 1072, "friends_count": 97, "statues_count": 831 }, "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("2015-11-24T10:00:28.000Z"), "id": 669214010940624897, "text": "#FucKYeah https://t.co/WTBGFcnicP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "FucKYeah" }}, "user": { "id": 85380753, "name": "MyBALLS NeeD ATTN!", "screen_name": "Gor4You", "lang": "en", "location": "Ur MoM's house", "create_at": date("2009-10-26"), "description": "I want Horny Ladies to #FoLLoW My #PERVersity❌❌❌", "followers_count": 2294, "friends_count": 2080, "statues_count": 60506 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gloucester City, NJ", "id": "7c6845d4f5897da3", "name": "Gloucester City", "place_type": "city", "bounding_box": rectangle("-75.13445,39.878213 -75.096176,39.908369") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34007, "countyName": "Camden", "cityID": 3426820, "cityName": "Gloucester City" } }
+{ "create_at": datetime("2015-11-24T10:00:28.000Z"), "id": 669214010986774528, "text": "Man. Richest in the world. RT https://t.co/1mLK2yAmJ9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 854900940, "name": "X", "screen_name": "Respect_Xavier", "lang": "en", "location": "Wichita, KS", "create_at": date("2012-09-30"), "description": "6'2 WR 2016 | Doing Everything For Myself and My Mom. She The Only One That Got Me Like I Got Me. | Proverbs 22:6 |", "followers_count": 1593, "friends_count": 978, "statues_count": 31469 }, "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("2015-11-24T10:00:28.000Z"), "id": 669214011397926912, "text": "Think I'm the only one not excited for Thanksgiving ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 537657053, "name": "ER❣KAPRETTYASS", "screen_name": "Erika_LajuannX3", "lang": "en", "location": "null", "create_at": date("2012-03-26"), "description": "#RestInPeaceRachel", "followers_count": 1119, "friends_count": 1033, "statues_count": 69240 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodlawn, MD", "id": "bf64a5dd9b5f1cf2", "name": "Woodlawn", "place_type": "city", "bounding_box": rectangle("-76.915396,38.938828 -76.711092,39.327477") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2486525, "cityName": "Woodlawn" } }
+{ "create_at": datetime("2015-11-24T10:00:28.000Z"), "id": 669214011800457216, "text": "@Ttowns11 it's a different one. Just as pretty though so screw you lol", "in_reply_to_status": 669210845298069504, "in_reply_to_user": 297848858, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 297848858 }}, "user": { "id": 356014436, "name": "Bruce", "screen_name": "RyanBruce14", "lang": "en", "location": "null", "create_at": date("2011-08-15"), "description": "it is what it is", "followers_count": 804, "friends_count": 677, "statues_count": 20279 }, "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("2015-11-24T10:00:28.000Z"), "id": 669214012039524352, "text": "I am so kool", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 462481593, "name": "YESI-", "screen_name": "yeseniaesquive2", "lang": "en", "location": "Austin Tx", "create_at": date("2012-01-12"), "description": "it is what it is. @princeroyce c/o 2016", "followers_count": 4440, "friends_count": 2875, "statues_count": 18702 }, "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-24T10:00:28.000Z"), "id": 669214012396015616, "text": "@AndruEdwards looks good. I’m liking the new Volt", "in_reply_to_status": 669210311619190784, "in_reply_to_user": 782310, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 782310 }}, "user": { "id": 3602931, "name": "H.Murchison", "screen_name": "hmurchison", "lang": "en", "location": "Issaquah WA", "create_at": date("2007-04-06"), "description": "I Am Legend", "followers_count": 409, "friends_count": 2476, "statues_count": 2597 }, "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.57736") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5333805, "cityName": "Issaquah" } }
+{ "create_at": datetime("2015-11-24T10:00:28.000Z"), "id": 669214012538679296, "text": "Inova Health System #Nursing #Job: REGISTERED NURSE PRN T1 (#FAIRFAX, VA) https://t.co/IOJpWfNlBk #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.3799476,38.8848327"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "Job", "FAIRFAX", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 120915863, "name": "TMJ-VAF Nursing Jobs", "screen_name": "tmj_VAF_NURSING", "lang": "en", "location": "Fairfax, VA", "create_at": date("2010-03-07"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Fairfax, VA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 367, "friends_count": 283, "statues_count": 147 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chantilly, VA", "id": "353916ce76086be6", "name": "Chantilly", "place_type": "city", "bounding_box": rectangle("-77.485561,38.840172 -77.364749,38.917243") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5114744, "cityName": "Chantilly" } }
+{ "create_at": datetime("2015-11-24T10:00:28.000Z"), "id": 669214012815507456, "text": "@Meganmae240 I hate attention.", "in_reply_to_status": 669213560950534144, "in_reply_to_user": 2327986573, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2327986573 }}, "user": { "id": 436341751, "name": "Heriberto Lopez G.", "screen_name": "itsnotherbie", "lang": "en", "location": "SATX Puerto Vallarta San diego", "create_at": date("2011-12-13"), "description": "Just because you're not happy doesn't mean you're making the wrong decision. I love Tejas. U.S. Marine. Grunt.", "followers_count": 1081, "friends_count": 798, "statues_count": 34627 }, "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-24T10:00:28.000Z"), "id": 669214012945494016, "text": "Interested in a #Clerical #job near #WhitePlains, NY? This could be a great fit: https://t.co/NjnYM3dod5 #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.7683784,41.0322939"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Clerical", "job", "WhitePlains", "Hiring", "CareerArc" }}, "user": { "id": 22976126, "name": "NYC Clerical Jobs", "screen_name": "tmj_nyc_cler", "lang": "en", "location": "New York, NY", "create_at": date("2009-03-05"), "description": "Follow this account for geo-targeted Office/Clerical job tweets in New York, NY from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 679, "friends_count": 203, "statues_count": 337 }, "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("2015-11-24T10:00:28.000Z"), "id": 669214013033725952, "text": "Dissent https://t.co/cgSN4y7fk4 #thebattlebegins #rumblingsofwar #newcardgame #scififiction", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.6477771,40.7422154"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thebattlebegins", "rumblingsofwar", "newcardgame", "scififiction" }}, "user": { "id": 2675192989, "name": "BattleforSularia", "screen_name": "SulariaBCG", "lang": "en", "location": "Lincoln, Ne", "create_at": date("2014-07-23"), "description": "null", "followers_count": 133, "friends_count": 215, "statues_count": 347 }, "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("2015-11-24T10:00:28.000Z"), "id": 669214013037912064, "text": "How to make a B-52 https://t.co/mB959adGxU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9271644,40.7595044"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2513457279, "name": "BYBER", "screen_name": "byberapp", "lang": "en", "location": "New York", "create_at": date("2014-04-27"), "description": "BYBER helps you meet new people, connect with friends and explore what your neighborhood has to offer. \n\n\n\n\nMeet - Connect - Explore!", "followers_count": 363, "friends_count": 83, "statues_count": 555 }, "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("2015-11-24T10:00:28.000Z"), "id": 669214013046288384, "text": "Tuesday Tunes--November 24, 2015 https://t.co/Y4jok4ceJR", "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": 271167530, "name": "The Guys From", "screen_name": "guysfromblank", "lang": "en", "location": "Calgary, Alberta", "create_at": date("2011-03-23"), "description": "Home of sarcastic insight into film, tv, sports, news and gaming. Plus news on The Guys From {BLANK} comedy", "followers_count": 246, "friends_count": 18, "statues_count": 480 }, "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("2015-11-24T10:00:28.000Z"), "id": 669214013142757377, "text": "Does Michael Christmas, What A Weird Day Album Live Up to the Hype? https://t.co/wCwYeXSsLi", "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": 2337030567, "name": "❌TheRadioFailed ❌", "screen_name": "TheRadioFailed", "lang": "en", "location": "null", "create_at": date("2014-02-10"), "description": "Podcast and site about subjects the radio won't play or talk about!#goodmusic #goodmessage #newmedia. Hosted by @the_realE @acarteryo89 @yours_trulyP", "followers_count": 132, "friends_count": 257, "statues_count": 1851 }, "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("2015-11-24T10:00:28.000Z"), "id": 669214013192953856, "text": "I hope it don't take forever for the doctors to give my puppy there shots", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 606707724, "name": "JÇœl", "screen_name": "July7515", "lang": "en", "location": "null", "create_at": date("2012-06-12"), "description": "·Gods daughter ·16 ·Striving to be on TOP ·Always showing L O V E ,KINDNESS &FAITH ™Jas #318", "followers_count": 739, "friends_count": 847, "statues_count": 22215 }, "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("2015-11-24T10:00:28.000Z"), "id": 669214013566226432, "text": "thank you ! �� https://t.co/MiJJd0xFv7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1917717204, "name": "fren-slay♏️", "screen_name": "frenzelle_apaga", "lang": "en", "location": "sultana high. ", "create_at": date("2013-09-29"), "description": "keep it lowkey and let people assume", "followers_count": 322, "friends_count": 378, "statues_count": 6036 }, "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("2015-11-24T10:00:28.000Z"), "id": 669214013599956992, "text": "@TMccrorey why you not at school", "in_reply_to_status": 669213878677594113, "in_reply_to_user": 325922847, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 325922847 }}, "user": { "id": 276232243, "name": "AK✨", "screen_name": "xSHETHE_B0MB", "lang": "en", "location": "Chester, SC❤️", "create_at": date("2011-04-02"), "description": "19. snapchat. asia_1013 .", "followers_count": 1956, "friends_count": 1890, "statues_count": 29377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chester, SC", "id": "fc3d3d1697ab1ede", "name": "Chester", "place_type": "city", "bounding_box": rectangle("-81.242759,34.683155 -81.189781,34.730359") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45023, "countyName": "Chester", "cityID": 4514095, "cityName": "Chester" } }
+{ "create_at": datetime("2015-11-24T10:00:28.000Z"), "id": 669214013998415874, "text": "Prepare for Holidays https://t.co/QifmpEXqYs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.1182951,40.9818873"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3058328650, "name": "Bright and Happy", "screen_name": "brighthappyplnr", "lang": "en", "location": "San Diego", "create_at": date("2015-02-23"), "description": "Marketing strategist & Virtual Assistant | I help small businesses create marketing strategy plans, automate systems and manage social media.", "followers_count": 122, "friends_count": 54, "statues_count": 2529 }, "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("2015-11-24T10:00:29.000Z"), "id": 669214014317039617, "text": "Can you recommend anyone for this #IT #job? https://t.co/qLur3rUPNZ #Cincinnati, OH #Hiring", "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": {{ "IT", "job", "Cincinnati", "Hiring" }}, "user": { "id": 28447324, "name": "Cameron Craig Group", "screen_name": "cameroncraig", "lang": "en", "location": "USA", "create_at": date("2009-04-02"), "description": "Thousands of jobs, one click away. Find a job here today.", "followers_count": 25299, "friends_count": 25067, "statues_count": 28512 }, "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("2015-11-24T10:00:29.000Z"), "id": 669214014321356800, "text": "ima be 18 ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2227011901, "name": "sc: ckhcsfwm", "screen_name": "mmoneaaaa", "lang": "en", "location": "null", "create_at": date("2013-12-02"), "description": "FreeCee➰| i love chyna❤️ @chynastarrr |i love tori:) sc : ckhcsfwm", "followers_count": 1415, "friends_count": 1114, "statues_count": 32816 }, "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("2015-11-24T10:00:29.000Z"), "id": 669214014497517568, "text": "@Isiah_LaMar explain how they cheated this time?", "in_reply_to_status": 669018567455744000, "in_reply_to_user": 62987715, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 62987715 }}, "user": { "id": 26812007, "name": "Colin Mahoney", "screen_name": "the_colin", "lang": "en", "location": "New York City", "create_at": date("2009-03-26"), "description": "Married / father of 1 / Public relations / public affairs / government relations professional focused on clean tech, health care and defense industries", "followers_count": 320, "friends_count": 665, "statues_count": 297 }, "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("2015-11-24T10:00:29.000Z"), "id": 669214014614798336, "text": "Happy birthday Kat! We've made some of the Best memories since 6th grade and we'll be making more love you!! ❤️❤️❤️ https://t.co/8FNjVL1HNO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1510861339, "name": "taylorrr", "screen_name": "tmozisekk", "lang": "en", "location": "null", "create_at": date("2013-06-12"), "description": "My heart belongs to @THEgaryh28 ❤️", "followers_count": 370, "friends_count": 236, "statues_count": 1576 }, "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("2015-11-24T10:00:29.000Z"), "id": 669214015411912706, "text": "I'm so mad I work all day on Thanksgiving but when I get my check(s) I know it will be well worth it ... so yall have fun eating & shit ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 574822350, "name": "PRETTY DIMPLES", "screen_name": "Miss_pretty862", "lang": "en", "location": "null", "create_at": date("2012-05-08"), "description": "KEYSHIA COLE is my FAVORITE =) you may no my name BUT not my STORY ! Loyalty is a understatement which have many Definitions", "followers_count": 403, "friends_count": 428, "statues_count": 37035 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin Center, NJ", "id": "003e82625ebc05b3", "name": "Franklin Center", "place_type": "city", "bounding_box": rectangle("-74.569926,40.510487 -74.507136,40.554607") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34035, "countyName": "Somerset", "cityID": 3424965, "cityName": "Franklin Center" } }
+{ "create_at": datetime("2015-11-24T10:00:29.000Z"), "id": 669214015596437504, "text": "@acaru_ he's not that gay ��", "in_reply_to_status": 669188889634934784, "in_reply_to_user": 311791983, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 311791983 }}, "user": { "id": 1901871475, "name": "phi phi", "screen_name": "sophalofugous6", "lang": "en", "location": "null", "create_at": date("2013-09-24"), "description": "i like to cut hair ✄", "followers_count": 851, "friends_count": 505, "statues_count": 6076 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portage, MI", "id": "3af1aa005c106143", "name": "Portage", "place_type": "city", "bounding_box": rectangle("-87.222662,41.536049 -85.491151,42.245344") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26077, "countyName": "Kalamazoo", "cityID": 2665560, "cityName": "Portage" } }
+{ "create_at": datetime("2015-11-24T10:00:29.000Z"), "id": 669214015642607617, "text": "https://t.co/bCEe3KkNj1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 143148618, "name": "dax mulder", "screen_name": "daxmancher", "lang": "en", "location": "null", "create_at": date("2010-05-12"), "description": "howdy folks", "followers_count": 167, "friends_count": 316, "statues_count": 10501 }, "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("2015-11-24T10:00:29.000Z"), "id": 669214015663443968, "text": "I never wanted my kid to grow up with two dads that's why I won't allow it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 606647870, "name": "ceiraleon", "screen_name": "florescecee", "lang": "en", "location": "null", "create_at": date("2012-06-12"), "description": "null", "followers_count": 1280, "friends_count": 502, "statues_count": 54529 }, "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-24T10:00:29.000Z"), "id": 669214015764074496, "text": "good morning!! who took my blanket", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1253232554, "name": "aileen cruise", "screen_name": "aileenzurc_", "lang": "en", "location": "null", "create_at": date("2013-03-08"), "description": "ღღ", "followers_count": 231, "friends_count": 223, "statues_count": 2109 }, "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("2015-11-24T10:00:29.000Z"), "id": 669214015785054209, "text": "Want to work in #GrandRapidsWyoming, MI? View our latest opening: https://t.co/pp195KcKnH #Sales #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.6897121,42.8948162"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GrandRapidsWyoming", "Sales", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22292507, "name": "TMJ-MI-US SalesJobs", "screen_name": "tmj_mi_sales", "lang": "en", "location": "Michigan", "create_at": date("2009-02-28"), "description": "Follow this account for geo-targeted Sales job tweets in Michigan Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 396, "friends_count": 302, "statues_count": 344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wyoming, MI", "id": "77b8f2664985738a", "name": "Wyoming", "place_type": "city", "bounding_box": rectangle("-85.782295,42.85341 -85.635295,42.942348") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26081, "countyName": "Kent", "cityID": 2688940, "cityName": "Wyoming" } }
+{ "create_at": datetime("2015-11-24T10:00:29.000Z"), "id": 669214016191926272, "text": "Partly cloudy this afternoon, high 58 (14 C). Low 39 (4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119697197, "name": "Huntsville Weather", "screen_name": "HuntsvilleWX", "lang": "en", "location": "null", "create_at": date("2010-03-04"), "description": "null", "followers_count": 665, "friends_count": 65, "statues_count": 5933 }, "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("2015-11-24T10:00:29.000Z"), "id": 669214016271699968, "text": "I need a haircut", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 703889020, "name": "Steven", "screen_name": "Kinger5668", "lang": "en", "location": "Clifton Heights, PA", "create_at": date("2013-10-07"), "description": "Clifton", "followers_count": 741, "friends_count": 779, "statues_count": 31794 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clifton Heights, PA", "id": "c707e1748d92a33e", "name": "Clifton Heights", "place_type": "city", "bounding_box": rectangle("-75.308186,39.922955 -75.283514,39.935766") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42045, "countyName": "Delaware", "cityID": 4214264, "cityName": "Clifton Heights" } }
+{ "create_at": datetime("2015-11-24T10:00:29.000Z"), "id": 669214016942833664, "text": "PLS STOP �� https://t.co/nTyRtttNDg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25539141, "name": "scary spice.", "screen_name": "TrillKitten", "lang": "en", "location": "Chicaglo. SIUC. ", "create_at": date("2009-03-20"), "description": "Galatians 1:10. Your child's future psychologist. I'm just a regular, degular, schmegular girl. ❤️", "followers_count": 2829, "friends_count": 1762, "statues_count": 116352 }, "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("2015-11-24T10:00:29.000Z"), "id": 669214017165074432, "text": "You about not lying https://t.co/mqxl8MzYSf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3249902922, "name": "Li' Shawt ❣", "screen_name": "miraclelashayy", "lang": "en", "location": "@financer_ ", "create_at": date("2015-06-19"), "description": "SC: adoresmiracle || fineeassk ❤️", "followers_count": 592, "friends_count": 415, "statues_count": 20666 }, "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("2015-11-24T10:00:29.000Z"), "id": 669214017466925057, "text": "@retroglo your skin is perfect do you have pores", "in_reply_to_status": 669212862175252480, "in_reply_to_user": 25866804, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25866804 }}, "user": { "id": 54280055, "name": "Claire McKinzie", "screen_name": "clairemckinzie", "lang": "en", "location": "Los Angeles, CA ", "create_at": date("2009-07-06"), "description": "Like butta. Music & Art culture writer. Once planned a burrito tour of the East Bay. occasional : snapping @gorillavsbear ✒️ @dummymag", "followers_count": 577, "friends_count": 354, "statues_count": 10396 }, "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("2015-11-24T10:00:29.000Z"), "id": 669214017534222340, "text": "He didn't even say shit! He puts his son and the chair and comes back over my way like \"Yeah man, I really thought Vietnam was just a war\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35646496, "name": "The Wicked Bitch", "screen_name": "iFredLee", "lang": "en", "location": "D[M]V", "create_at": date("2009-04-26"), "description": "Winning since my dad skeeted in my mom....thank God I wasn't swallowed. #Team. #SteelerNation", "followers_count": 3887, "friends_count": 3116, "statues_count": 262841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wilkinsburg, PA", "id": "854c2897718ea4e3", "name": "Wilkinsburg", "place_type": "city", "bounding_box": rectangle("-79.895601,40.431321 -79.854094,40.459451") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4285188, "cityName": "Wilkinsburg" } }
+{ "create_at": datetime("2015-11-24T10:00:29.000Z"), "id": 669214017689292800, "text": "Happy birthday bros!! @ShayanNassiri @AryanNassiri hope both of y'all 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": {{ 1263611934, 790925131 }}, "user": { "id": 1912426062, "name": "Spencer Taack", "screen_name": "spencertaack34", "lang": "en", "location": "null", "create_at": date("2013-09-27"), "description": "⚾️1•1•Six", "followers_count": 224, "friends_count": 169, "statues_count": 2217 }, "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-24T10:00:29.000Z"), "id": 669214017907523584, "text": "Puppets & Pie tonight at 7pm I hope to see everyone there! @ New Life… https://t.co/Je6apqrmNR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.65139438,39.74741706"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1710215016, "name": "Stephanie Burrell", "screen_name": "stephrenee99", "lang": "en", "location": "null", "create_at": date("2013-08-29"), "description": "FCA Sophmore. Love Jesus. Netflix. Friends. Photography.", "followers_count": 108, "friends_count": 149, "statues_count": 245 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, OH", "id": "bec4732d440b4c03", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-82.670558,39.681924 -82.522696,39.792026") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39045, "countyName": "Fairfield", "cityID": 3941720, "cityName": "Lancaster" } }
+{ "create_at": datetime("2015-11-24T10:00:29.000Z"), "id": 669214017953599490, "text": "Your messages annoy me cause your sentence covers 8 lines since you hit the return key every other word ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 737757098, "name": "Tina Maleski", "screen_name": "KMaleski", "lang": "en", "location": "United States", "create_at": date("2012-08-04"), "description": "Don't just dream it, achieve it. #lvcws 99 #lvcwlax 15 301➡717", "followers_count": 422, "friends_count": 402, "statues_count": 16442 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Annville, PA", "id": "87dbd9e813c90094", "name": "Annville", "place_type": "city", "bounding_box": rectangle("-76.531036,40.322751 -76.485231,40.340757") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42075, "countyName": "Lebanon", "cityID": 4202608, "cityName": "Annville" } }
+{ "create_at": datetime("2015-11-24T10:00:29.000Z"), "id": 669214018117181440, "text": "@umlweatherman @bigten Wouldn't doubt it. Though my team is 1 of the good B1G teams this yr, we're new to D1 so it's interesting to read.", "in_reply_to_status": 669213545742114816, "in_reply_to_user": 326574754, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 326574754, 15755096 }}, "user": { "id": 359577483, "name": "Devin Boyer", "screen_name": "wxdevin", "lang": "en", "location": "Andover, MA", "create_at": date("2011-08-21"), "description": "Pittsburgh-born with a Penn State heart. Programmer in the weather enterprise. I tweet weather (especially severe storms), tech, news/media, sports & more!", "followers_count": 606, "friends_count": 798, "statues_count": 7962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Andover, MA", "id": "00a4b217142e93d0", "name": "Andover", "place_type": "city", "bounding_box": rectangle("-71.255046,42.598263 -71.075953,42.700233") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25009, "countyName": "Essex", "cityID": 2501430, "cityName": "Andover" } }
+{ "create_at": datetime("2015-11-24T10:00:29.000Z"), "id": 669214018163183616, "text": "@BroncosItaly With 1/3 of ESPN'S exposure. And 1/2 of both FS1 and NBCSN's. The low $ caused teams and ESPN to go elsewhere.", "in_reply_to_status": 669213121714737153, "in_reply_to_user": 28165505, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28165505 }}, "user": { "id": 325862777, "name": "Ivey McClelland", "screen_name": "iveyjanette_207", "lang": "en", "location": "Albuquerque & Los Angeles", "create_at": date("2011-06-28"), "description": "AKA Janette McClelland. Trombonist,street musician,comedian and foodie. Also at @ivey_mclelland. Instagram: @iveyjanette207", "followers_count": 1252, "friends_count": 1989, "statues_count": 168594 }, "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("2015-11-24T10:00:30.000Z"), "id": 669214018406477824, "text": "@BarackObama is a control freak & on a downward spiral. He tries to disqualify everything #Hollande says. @POTUS is wrong. @OutnumberedFNC", "in_reply_to_status": -1, "in_reply_to_user": 813286, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hollande" }}, "user_mentions": {{ 813286, 1536791610, 2448135692 }}, "user": { "id": 57465006, "name": "Mara Watkins", "screen_name": "MaraWatkins", "lang": "en", "location": "Dallas,TX", "create_at": date("2009-07-16"), "description": "If I lived my life by what others were thinkin', the heart inside me would've died. ~ Bob Dylan", "followers_count": 378, "friends_count": 255, "statues_count": 3985 }, "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-24T10:00:30.000Z"), "id": 669214018700201989, "text": "Why didn't you call him an asshole to 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": 29103393, "name": "Lisa", "screen_name": "Lmstone19", "lang": "en", "location": "#BostonStrong #Merica", "create_at": date("2009-04-05"), "description": "Three main loves..#Family #RedSox #CountryMusic. Laugh, it's all you can do. #tcot", "followers_count": 364, "friends_count": 733, "statues_count": 13396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Melrose, MA", "id": "70be204c215a5404", "name": "Melrose", "place_type": "city", "bounding_box": rectangle("-71.082765,42.438913 -71.025866,42.475925") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2540115, "cityName": "Melrose" } }
+{ "create_at": datetime("2015-11-24T10:00:30.000Z"), "id": 669214018838466560, "text": "idk where everyone went", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2423650946, "name": "Alana Glaspie", "screen_name": "alanalanae_", "lang": "en", "location": "Lake Charles, LA", "create_at": date("2014-04-02"), "description": "beauty is from within", "followers_count": 1333, "friends_count": 2094, "statues_count": 55300 }, "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("2015-11-24T10:00:30.000Z"), "id": 669214018884775937, "text": "�������� https://t.co/fSTM5tYSkz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 109183537, "name": "candiedyams&applepie", "screen_name": "Tay_2_SweeT", "lang": "en", "location": "null", "create_at": date("2010-01-27"), "description": "Faithful black woman that will love you forever \n#blackwomendontcheat\n#blackwomendontlie\n Just follow me and stfu\n#Aquariusbaby ♒ Feb. 18th\nI'm dope", "followers_count": 742, "friends_count": 739, "statues_count": 80457 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Woods, MI", "id": "cf4c3ec0111a59b2", "name": "Huntington Woods", "place_type": "city", "bounding_box": rectangle("-83.183354,42.474052 -83.153051,42.489443") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2640000, "cityName": "Huntington Woods" } }
+{ "create_at": datetime("2015-11-24T10:00:30.000Z"), "id": 669214018914029568, "text": "The world is coming to an end and I can't wait", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 303805862, "name": "Splash", "screen_name": "Ivan_splash", "lang": "en", "location": "Atlanta", "create_at": date("2011-05-23"), "description": "I just travel and tweet", "followers_count": 15607, "friends_count": 238, "statues_count": 227199 }, "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("2015-11-24T10:00:30.000Z"), "id": 669214019069186048, "text": "@CClark1011 Agree. The problem can be summed up in one word: Obama", "in_reply_to_status": 669211912022921216, "in_reply_to_user": 423972695, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 423972695 }}, "user": { "id": 70983585, "name": "Howard ", "screen_name": "tekhow", "lang": "en", "location": "null", "create_at": date("2009-09-02"), "description": "null", "followers_count": 38, "friends_count": 44, "statues_count": 1581 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wausau, WI", "id": "35ecddb970148417", "name": "Wausau", "place_type": "city", "bounding_box": rectangle("-89.746623,44.919084 -89.590106,45.006833") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55073, "countyName": "Marathon", "cityID": 5584475, "cityName": "Wausau" } }
+{ "create_at": datetime("2015-11-24T10:00:30.000Z"), "id": 669214019186589696, "text": "Sales Representative (S) 100677 Base Pay + Comm. - TruGreen: (#Cary, NC) https://t.co/RU5CkNONBx #Sales #TruGreen #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.7811169,35.79154"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Cary", "Sales", "TruGreen", "Job", "Jobs", "Hiring" }}, "user": { "id": 2776752523, "name": "TruGreen Jobs", "screen_name": "TruGreenJobs", "lang": "en", "location": "null", "create_at": date("2014-08-28"), "description": "null", "followers_count": 20, "friends_count": 0, "statues_count": 693 }, "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("2015-11-24T10:00:30.000Z"), "id": 669214019249553409, "text": "right like lil girl I will smack the shit outta you and not gaf �������� https://t.co/RAMK4PCq9u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1917118074, "name": "December 4th", "screen_name": "JohrdynLuvzU", "lang": "en", "location": "null", "create_at": date("2013-09-29"), "description": "Cute, funny, & made of money #Sagitarius ♐️ #Money #HisFutureMrs #062515 ❤️", "followers_count": 3623, "friends_count": 2054, "statues_count": 18963 }, "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-24T10:00:30.000Z"), "id": 669214019362955265, "text": "@jessietrammell9 ah, what an interesting name", "in_reply_to_status": 669213830145286144, "in_reply_to_user": 2684042140, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2684042140 }}, "user": { "id": 136128888, "name": "Ralph Seegobin", "screen_name": "Ralph_Seegobin", "lang": "en", "location": "Ocoee, FL", "create_at": date("2010-04-22"), "description": "Christian. Musician. Attorney. Florida Gator.", "followers_count": 678, "friends_count": 1414, "statues_count": 81230 }, "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("2015-11-24T10:00:30.000Z"), "id": 669214019367104513, "text": "@Ask_WellsFargo lol don't mind me, I'm a baby sometimes:)", "in_reply_to_status": 669213778597322753, "in_reply_to_user": 23002858, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23002858 }}, "user": { "id": 46932076, "name": "Ron Bridges™", "screen_name": "Mr_Celebrity1", "lang": "en", "location": "I'm From ATLANTA,Ga!!!", "create_at": date("2009-06-13"), "description": "Manager for @classic_atl bookings teamwethat@gmail.com 678 467 6359\nDownload\nMarijuana\nhttp://t.co/HIlkujNtDJ\n\nhttp://t.co/M9P2Wc5qmQ", "followers_count": 3137, "friends_count": 1659, "statues_count": 66470 }, "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("2015-11-24T10:00:30.000Z"), "id": 669214019408924672, "text": "Sterling Jewelers: Sales Associate (#Hillsboro, OR) https://t.co/Q34cmR9mvx #Retail #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.8755191,45.533964"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hillsboro", "Retail", "Job", "Jobs", "Hiring" }}, "user": { "id": 3248848340, "name": "Jobs at Jared", "screen_name": "JobsAtJared", "lang": "en", "location": "null", "create_at": date("2015-06-18"), "description": "Your Future Starts Here.", "followers_count": 12, "friends_count": 1, "statues_count": 503 }, "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("2015-11-24T10:00:30.000Z"), "id": 669214019450855424, "text": "Be good and righteous, and hang in there - God will save you from your brokenness https://t.co/wIiv4s8X7D", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 377449413, "name": "joshua rodrigue XΦ", "screen_name": "scoonie__", "lang": "en", "location": "Kerrville, TX", "create_at": date("2011-09-21"), "description": "she's not bleeding on the ballroom floor just for the attention, cause that's just ridiculously odd.", "followers_count": 361, "friends_count": 211, "statues_count": 8723 }, "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("2015-11-24T10:00:30.000Z"), "id": 669214019450892289, "text": "Eggnog <", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "nl", "is_retweet": false, "user": { "id": 330488565, "name": "Danielle M.", "screen_name": "Danielle_eeee", "lang": "en", "location": "San Antonio ✈️ Denton", "create_at": date("2011-07-06"), "description": "null", "followers_count": 2202, "friends_count": 1284, "statues_count": 119380 }, "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-24T10:00:30.000Z"), "id": 669214019513769986, "text": "Partly cloudy this afternoon, high 60 (16 C). Low 39 (4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119698223, "name": "Birmingham Weather", "screen_name": "BirminghamWX", "lang": "en", "location": "Birmingham, AL", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Birmingham, AL", "followers_count": 501, "friends_count": 68, "statues_count": 5945 }, "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("2015-11-24T10:00:30.000Z"), "id": 669214019610243072, "text": "@BryceMartinez hell yeah they're going to remember the team who got East back to its winning ways", "in_reply_to_status": 669213690630156288, "in_reply_to_user": 707574097, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 707574097 }}, "user": { "id": 2897101003, "name": "ShaunJon", "screen_name": "shaun523124", "lang": "en", "location": "Victoria, TX", "create_at": date("2014-11-11"), "description": "No matter how much it hurts no matter how dark it gets no matter how far you fall you are never out of the fight #31#wwbd class of 2016 SENIOR Year #VEHS", "followers_count": 1190, "friends_count": 1173, "statues_count": 23090 }, "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("2015-11-24T10:00:30.000Z"), "id": 669214019627180032, "text": "I cant even lie, bae's mom makes the best green bean casserole #fuckwhatyaheard https://t.co/ctVIogxHcd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "fuckwhatyaheard" }}, "user": { "id": 239807448, "name": "Patrón Ochoa", "screen_name": "juliaj_215", "lang": "en", "location": "null", "create_at": date("2011-01-18"), "description": "I got a degree in political science. I like my boyfriend @thecaptain1992", "followers_count": 129, "friends_count": 272, "statues_count": 7763 }, "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-24T10:00:30.000Z"), "id": 669214019798962176, "text": "���� #40sNShortys \nhttps://t.co/LcV6FH1DyN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "40sNShortys" }}, "user": { "id": 91892535, "name": "Romè The Goddess", "screen_name": "RomiiRae", "lang": "en", "location": "We Are Dallas TX", "create_at": date("2009-11-22"), "description": "who cares. curator. #WeAreDallas #PinkAvenue #40sNShortys contact.romiirae@gmail.com", "followers_count": 2273, "friends_count": 729, "statues_count": 66665 }, "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("2015-11-24T10:00:30.000Z"), "id": 669214019857707008, "text": "We're #hiring! Read about our latest #job opening here: Cashier (Team Member) - https://t.co/xo53wbe5bs https://t.co/tbRHKDs0bZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-91.5301683,41.6611277"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job" }}, "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": 300, "friends_count": 459, "statues_count": 2961 }, "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("2015-11-24T10:00:30.000Z"), "id": 669214019903815682, "text": "Fuck yeah I get paid tomorrow. And I get paid time off for Thursday and Friday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3309002431, "name": "Vicki Vallencourt", "screen_name": "aychavelaa", "lang": "en", "location": "here ", "create_at": date("2015-08-07"), "description": "for booking information email me. vescafresca@gmail.com", "followers_count": 353, "friends_count": 333, "statues_count": 3508 }, "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-24T10:00:30.000Z"), "id": 669214020038062080, "text": "Pardon me for the outburst, BUT IM SO SICK OF BEING AN ASSISTANT.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2305933254, "name": "Lois Rodrigo", "screen_name": "LoisRodrigo", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-01-22"), "description": "[IG:@loisrodrigo]", "followers_count": 244, "friends_count": 932, "statues_count": 1091 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-24T10:00:30.000Z"), "id": 669214020172279810, "text": "@em_beckk your Avi ������", "in_reply_to_status": -1, "in_reply_to_user": 4042335194, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4042335194 }}, "user": { "id": 1966103520, "name": "bay♡", "screen_name": "bay_eaton", "lang": "en", "location": "oregon ", "create_at": date("2013-10-16"), "description": "marist '17", "followers_count": 233, "friends_count": 293, "statues_count": 4568 }, "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-24T10:00:30.000Z"), "id": 669214020272963586, "text": "Emily straight up spit her gnarly chewed gum 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": 2429572801, "name": "Ava Gibson", "screen_name": "AvaGibson_", "lang": "en", "location": "null", "create_at": date("2014-04-05"), "description": ":)", "followers_count": 104, "friends_count": 133, "statues_count": 567 }, "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("2015-11-24T10:00:30.000Z"), "id": 669214021300547584, "text": "Happy birthday evil twins I fucking love you guys @pezzolasam @jillpezzolaaa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2699523750, 1959852726 }}, "user": { "id": 1054278422, "name": "Ames", "screen_name": "amdejung", "lang": "en", "location": "null", "create_at": date("2013-01-01"), "description": "null", "followers_count": 537, "friends_count": 380, "statues_count": 7253 }, "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-24T10:00:30.000Z"), "id": 669214022017880065, "text": "Ion have cousins 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": 377006941, "name": "Klaus.", "screen_name": "QuanGottii", "lang": "en", "location": "RIP POPS ", "create_at": date("2011-09-20"), "description": "everyday u wake up your destiny depends on what you do | 02.14.15", "followers_count": 1029, "friends_count": 323, "statues_count": 112692 }, "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("2015-11-24T10:00:30.000Z"), "id": 669214022252630016, "text": "A good laugh & a long sleep is sometimes the best cures for stressful days. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3042184248, "name": "Marisol Alatorre", "screen_name": "AlatorreMarisol", "lang": "en", "location": "West Hollywood, CA", "create_at": date("2015-02-25"), "description": "✨ snap chat me! m_solly follow my insta : @makeup_by_solly @marisol_alatorre✨ Hollywood, CA", "followers_count": 408, "friends_count": 268, "statues_count": 988 }, "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("2015-11-24T10:00:31.000Z"), "id": 669214022458343425, "text": "Lowkey jealous bc two of my best friends hung out with out me but I'm the mutual friend.. ������ #NeverNeeded ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NeverNeeded" }}, "user": { "id": 1007293902, "name": "griselda", "screen_name": "omg_gris", "lang": "en", "location": "dallas, tx ", "create_at": date("2012-12-12"), "description": "trying.", "followers_count": 354, "friends_count": 342, "statues_count": 9196 }, "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("2015-11-24T10:00:31.000Z"), "id": 669214022558859264, "text": "know what you talking bout,", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 98070190, "name": "T . G R A Y ", "screen_name": "TGrayWINNING", "lang": "en", "location": "H O U $ T O N , T E X A $", "create_at": date("2009-12-19"), "description": "|Vintage Royalty™ | #iProduceRoyalMusic", "followers_count": 1092, "friends_count": 1524, "statues_count": 25775 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:31.000Z"), "id": 669214022663856130, "text": "Boy getting curved by the pre teen https://t.co/PaWKZbnxDn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388540666, "name": "Justin Luke", "screen_name": "J_luke34", "lang": "en", "location": "Statesboro, GA", "create_at": date("2011-10-10"), "description": "If you can learn to adapt, its hard to fail ΚΣ in the boro : Oceans 11", "followers_count": 529, "friends_count": 459, "statues_count": 17731 }, "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("2015-11-24T10:00:31.000Z"), "id": 669214022768685060, "text": "Yall I'm bout 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": 447659568, "name": "Baby✨", "screen_name": "Lanttee_", "lang": "en", "location": "null", "create_at": date("2011-12-26"), "description": "I'm like a man smoking at a gas station", "followers_count": 748, "friends_count": 649, "statues_count": 58796 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:31.000Z"), "id": 669214022835638272, "text": "Mostly sunny this afternoon, high 63 (17 C). Low 49 (9 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119698549, "name": "Mobile Weather", "screen_name": "MobileWX", "lang": "en", "location": "Mobile, Al", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Mobile, Al", "followers_count": 387, "friends_count": 68, "statues_count": 6020 }, "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("2015-11-24T10:00:31.000Z"), "id": 669214022898548737, "text": "#Kilwins https://t.co/idxfLWeB2G great, hard https://t.co/JfcN0zBnRe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.94964835,28.94095314"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Kilwins" }}, "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": 4128, "friends_count": 876, "statues_count": 329501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lady Lake, FL", "id": "06664699750ebfde", "name": "Lady Lake", "place_type": "city", "bounding_box": rectangle("-81.954487,28.886979 -81.905814,28.960182") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12069, "countyName": "Lake", "cityID": 1237375, "cityName": "Lady Lake" } }
+{ "create_at": datetime("2015-11-24T10:00:31.000Z"), "id": 669214022911180800, "text": "@five_guys #Five Guys / https://t.co/GKzVfMfT5t good, sweet https://t.co/DFMLlSpytP", "in_reply_to_status": -1, "in_reply_to_user": 18194079, "favorite_count": 0, "coordinate": point("-81.93637311,28.93243009"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Five" }}, "user_mentions": {{ 18194079 }}, "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": 4128, "friends_count": 876, "statues_count": 329501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lady Lake, FL", "id": "06664699750ebfde", "name": "Lady Lake", "place_type": "city", "bounding_box": rectangle("-81.954487,28.886979 -81.905814,28.960182") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12069, "countyName": "Lake", "cityID": 1237375, "cityName": "Lady Lake" } }
+{ "create_at": datetime("2015-11-24T10:00:31.000Z"), "id": 669214022927945728, "text": "Want to work in #Phoenix, AZ? View our latest opening: https://t.co/Hv5bjw2Shm #Healthcare #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.0740373,33.4483771"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Phoenix", "Healthcare", "Job", "Jobs", "Hiring" }}, "user": { "id": 21683752, "name": "TMJ- PHX Health Jobs", "screen_name": "tmj_phx_health", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2009-02-23"), "description": "Follow this account for geo-targeted Healthcare job tweets in Phoenix, AZ. Need help? Tweet us at @CareerArc!", "followers_count": 1108, "friends_count": 1441, "statues_count": 911 }, "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-24T10:00:31.000Z"), "id": 669214023057956864, "text": "See our latest #Flemington, NJ #job and click to apply: Teller I - https://t.co/sIukfCkqxx #CustomerService #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.857356,40.5409651"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Flemington", "job", "CustomerService", "Hiring", "CareerArc" }}, "user": { "id": 71795134, "name": "N. NJ Cust. Serv", "screen_name": "tmj_NJN_cstsrv", "lang": "en", "location": "Northern NJ", "create_at": date("2009-09-05"), "description": "Follow this account for geo-targeted Customer Service job tweets in Northern NJ from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 327, "friends_count": 283, "statues_count": 123 }, "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": 42, "stateName": "Pennsylvania", "countyID": 42017, "countyName": "Bucks" } }
+{ "create_at": datetime("2015-11-24T10:00:31.000Z"), "id": 669214023204798464, "text": "Temp 57.1°F Rising, Pressure 30.287in Falling, Dew point 36.5°, Wind N 0mph, Rain today 0.00in https://t.co/gPtnr2dDtV #STLwx #MOwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.35555556,38.47694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "STLwx", "MOwx" }}, "user": { "id": 1191304230, "name": "Oakville Weather", "screen_name": "OakvilleWx", "lang": "en", "location": "Oakville, Missouri, USA", "create_at": date("2013-02-17"), "description": "Current conditions on the hour brought to you by @DaveStLou", "followers_count": 87, "friends_count": 80, "statues_count": 23248 }, "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("2015-11-24T10:00:31.000Z"), "id": 669214023456456704, "text": "@dhopeboii_BigT why should I?", "in_reply_to_status": 669213799254069248, "in_reply_to_user": 2874510725, "favorite_count": 0, "coordinate": point("-94.41203384,39.10345953"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2874510725 }}, "user": { "id": 1931914592, "name": "Allyson Ragan", "screen_name": "allyson_ragan", "lang": "en", "location": "Independence, Missouri ", "create_at": date("2013-10-03"), "description": "Junior at WCHS Class of 2017|Single|❤Missouri Tigers❤ #ChiefsKingdom #Royals #NASCAR #teamhendrick & #19 |RIP Mason 3/15/15|", "followers_count": 569, "friends_count": 1694, "statues_count": 13443 }, "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("2015-11-24T10:00:31.000Z"), "id": 669214023708110848, "text": "schertz here we come!¡!�� https://t.co/Qoj6wMOpqM", "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 - broncos country - university of north texas - ΔΓ - NT40 - broadcast journalism - odyssey writer - crocs", "followers_count": 2348, "friends_count": 1227, "statues_count": 58565 }, "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-24T10:00:31.000Z"), "id": 669214024102313984, "text": "@vikasdewas started when \"if you believe in this intolerance crap... god help you.\" message start coming \"you\" in that instead of incidence.", "in_reply_to_status": 669204913457598464, "in_reply_to_user": 110092731, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 110092731 }}, "user": { "id": 51757989, "name": "Neelesh Ghosalkar", "screen_name": "GNeelesh", "lang": "en", "location": "20171", "create_at": date("2009-06-28"), "description": "From Indore - Current Location Washington DC", "followers_count": 141, "friends_count": 100, "statues_count": 2117 }, "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("2015-11-24T10:00:31.000Z"), "id": 669214024136056837, "text": "Ready to go home too", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 939288685, "name": "Uzi", "screen_name": "TheBigUziii", "lang": "en", "location": "Lagrange,NC", "create_at": date("2012-11-10"), "description": "instagram : @ thebiguzi", "followers_count": 179, "friends_count": 150, "statues_count": 4579 }, "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("2015-11-24T10:00:31.000Z"), "id": 669214024173662208, "text": "Breaking News: One arrest in shooting of 5 justice protesters #Justice4Jamar", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Justice4Jamar" }}, "user": { "id": 17604419, "name": "Robert", "screen_name": "KRDNews1", "lang": "en", "location": "Seattle, United States", "create_at": date("2008-11-24"), "description": "War, social inequality and racial injustice. And Breaking News, a.k.a. media propaganda.", "followers_count": 406, "friends_count": 344, "statues_count": 10113 }, "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-24T10:00:31.000Z"), "id": 669214024349917184, "text": "“Hadoop is a distributed system for counting words” dying https://t.co/i9Et2XKEbI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30950495, "name": "Kevin Chen", "screen_name": "kevinchen", "lang": "en", "location": "New York, NY", "create_at": date("2009-04-13"), "description": "Software developer and accidental machinist. Curious about compilers, big infra, and security. Previously at @Yelp.", "followers_count": 497, "friends_count": 292, "statues_count": 30861 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morningside Heights, Manhattan", "id": "6497298c68c6fe28", "name": "Morningside Heights", "place_type": "neighborhood", "bounding_box": rectangle("-73.976289,40.801889 -73.948014,40.823257") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2015-11-24T10:00:31.000Z"), "id": 669214024500776960, "text": "When you are awkwardly in the background of someone's snapchat and they see you staring at their phone bc you're 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": 1417093002, "name": "JP©", "screen_name": "James_P_Collins", "lang": "en", "location": "null", "create_at": date("2013-05-09"), "description": "|John 10:10| The true sign of intelligence isn't knowledge but imagination| Heath, Tx. |", "followers_count": 349, "friends_count": 263, "statues_count": 10826 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rockwall, TX", "id": "14b7e6a78837521a", "name": "Rockwall", "place_type": "city", "bounding_box": rectangle("-96.495323,32.864021 -96.35829,32.977061") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48397, "countyName": "Rockwall", "cityID": 4862828, "cityName": "Rockwall" } }
+{ "create_at": datetime("2015-11-24T10:00:31.000Z"), "id": 669214024697966592, "text": "if u kno swum then you know most of the kid's influence.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 518073680, "name": "LOSTINTHESΔUCE カソ", "screen_name": "sagecasso", "lang": "en", "location": "SEΔ, JΔPΔN ", "create_at": date("2012-03-07"), "description": "[keen child third eye see]k knoweledge. yes, i finess the arts.", "followers_count": 291, "friends_count": 197, "statues_count": 2602 }, "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-24T10:00:31.000Z"), "id": 669214024861634560, "text": "I'm at Nicks Pizza in Pittsburgh, PA https://t.co/yWHgAsszSn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.03476359,40.36933344"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 976040882, "name": "Painting witha Twist", "screen_name": "PWATPittsburgh", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2012-11-28"), "description": "Painting with a Twist ® A Little Bit of Paint, A Little Bit of Wine, A Whole Lot of Fun! Pittsburgh-West & Pittsburgh-South", "followers_count": 2022, "friends_count": 2022, "statues_count": 4787 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Castle Shannon, PA", "id": "52b8e53ae833638f", "name": "Castle Shannon", "place_type": "city", "bounding_box": rectangle("-80.038088,40.354859 -80.001567,40.379642") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4211680, "cityName": "Castle Shannon" } }
+{ "create_at": datetime("2015-11-24T10:00:31.000Z"), "id": 669214024966520833, "text": "The fact that I just saw Brandon randomly driving in my town ��������WHY R U 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": 271570481, "name": "Sarahhhh☯", "screen_name": "sarahdlmao", "lang": "en", "location": "jersey", "create_at": date("2011-03-24"), "description": "insta: @sarahhhhdonegan", "followers_count": 451, "friends_count": 503, "statues_count": 15028 }, "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("2015-11-24T10:00:31.000Z"), "id": 669214025130053632, "text": "@MusxeL @aboogies send the paid", "in_reply_to_status": -1, "in_reply_to_user": 3298592403, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3298592403, 4292335034 }}, "user": { "id": 2298993378, "name": "James #OASF", "screen_name": "FearsMelt", "lang": "en", "location": "IL", "create_at": date("2014-01-18"), "description": "Stay True. BO3 DON. Silver Surfer. @Johnny1Bullet covered me in a prime", "followers_count": 544, "friends_count": 90, "statues_count": 15548 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roselle, IL", "id": "5e80ee08ea361155", "name": "Roselle", "place_type": "city", "bounding_box": rectangle("-88.124996,41.949557 -88.031515,42.000647") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1765806, "cityName": "Roselle" } }
+{ "create_at": datetime("2015-11-24T10:00:31.000Z"), "id": 669214025494982657, "text": "i miss 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": 3783872537, "name": "Kayla Barnes", "screen_name": "kabarnes1211", "lang": "en", "location": "null", "create_at": date("2015-09-26"), "description": "work so hard that one day your signature will be called an autograph.", "followers_count": 33, "friends_count": 35, "statues_count": 61 }, "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("2015-11-24T10:00:31.000Z"), "id": 669214025864073216, "text": "I see no lies. https://t.co/XLNvx3RCuf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 178058088, "name": "FunkyColdMedina", "screen_name": "MannyIsOld", "lang": "en", "location": "OUT HERE", "create_at": date("2010-08-13"), "description": "Call Your Grandma, You Schmucks.", "followers_count": 353, "friends_count": 452, "statues_count": 17568 }, "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("2015-11-24T10:00:31.000Z"), "id": 669214025939570688, "text": "https://t.co/3VYYWq9vNq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 436889609, "name": "AngryMeyers", "screen_name": "AngryMeyers", "lang": "en", "location": "null", "create_at": date("2011-12-14"), "description": "Fuck off.", "followers_count": 104, "friends_count": 0, "statues_count": 75 }, "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("2015-11-24T10:00:31.000Z"), "id": 669214026019115008, "text": "@sherilw62 @mariavsl5 @LorenaMr71 @G1ant0MAN @LindaflorDrika @rajesh6363 @monicasloves @lilian8090 @ronald9807 Exquisite-thank you Sheri", "in_reply_to_status": 668126577545342977, "in_reply_to_user": 2623373627, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2623373627, 777922579, 2977365761, 2893259300, 1726904809, 2417097516, 1466009443, 3000523623, 2777424696 }}, "user": { "id": 2771164420, "name": "Sheila Tinkham", "screen_name": "SheilaMTinkham", "lang": "en", "location": "Lincoln, Nebraska", "create_at": date("2014-09-15"), "description": "Artist in Residency, Capital City Painter, lover of the Impressionists, studied at Georgia Tech", "followers_count": 1374, "friends_count": 997, "statues_count": 14762 }, "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("2015-11-24T10:00:31.000Z"), "id": 669214026132393984, "text": "When you decide to skip class but it gets cancelled anyways �������������� #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": 35091705, "name": "Lydia Suarez", "screen_name": "LydiaCooksGood", "lang": "en", "location": "The Biggest Little City ", "create_at": date("2009-04-24"), "description": "Sometimes I'll start a sentence and I don't even know where it's going. I just hope I find it along the way.", "followers_count": 346, "friends_count": 642, "statues_count": 9111 }, "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("2015-11-24T10:00:31.000Z"), "id": 669214026224701440, "text": "Partly cloudy this afternoon, high 45 (7 C). Low 37 (3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119751854, "name": "Chicago Weather", "screen_name": "Chicago_WX", "lang": "en", "location": "Chicago, IL", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Chicago, IL", "followers_count": 1055, "friends_count": 68, "statues_count": 6302 }, "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-24T10:00:31.000Z"), "id": 669214026249863168, "text": "@Piazy_Tar Turkey dsnt need KRG or its oil.Its the other way around.KRG needs TR to sell its oil or go sell it from Persian gulf @Osaavaa", "in_reply_to_status": 669201754614071296, "in_reply_to_user": 2160182783, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2160182783, 3420107517 }}, "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": 589, "friends_count": 401, "statues_count": 10897 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-24T10:00:31.000Z"), "id": 669214026375692288, "text": "ARRIVE'S A NEW YEAR...“MUSIC MAKES A MAN” from the album 'Skillzzen' https://t.co/wUSjmPchMA https://t.co/WJDgDlruX1", "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": 606, "friends_count": 2004, "statues_count": 109128 }, "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("2015-11-24T10:00:31.000Z"), "id": 669214026396622849, "text": "Can you recommend anyone for this #job? Part Time Deli Clerk - https://t.co/TCOfaUcCmT #Ashburn, VA #Retail #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.481018,39.041408"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Ashburn", "Retail", "Hiring", "CareerArc" }}, "user": { "id": 59694665, "name": "TMJ-VA Retail Jobs", "screen_name": "tmj_va_retail", "lang": "en", "location": "Virginia", "create_at": date("2009-07-23"), "description": "Follow this account for geo-targeted Retail job tweets in Virginia Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 368, "friends_count": 304, "statues_count": 659 }, "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("2015-11-24T10:00:31.000Z"), "id": 669214026476294144, "text": "Want to work at Ryder? We're #hiring in #Milwaukee, WI! Click for details: https://t.co/0cqOyoiR7H #SupplyChain #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.9064736,43.0389025"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Milwaukee", "SupplyChain", "Job", "Jobs", "CareerArc" }}, "user": { "id": 189279467, "name": "Milwaukee S-Chain", "screen_name": "tmj_MKE_schn", "lang": "en", "location": "Milwaukee, WI", "create_at": date("2010-09-10"), "description": "Follow this account for geo-targeted Supply Chain job tweets in Milwaukee, WI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 273, "friends_count": 262, "statues_count": 14 }, "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("2015-11-24T10:00:31.000Z"), "id": 669214026514178049, "text": "It's so funny when people assume shits about them", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2576261145, "name": "Gabriel_ParkedS14", "screen_name": "majicdick_", "lang": "en", "location": "MIA", "create_at": date("2014-06-01"), "description": "Im 19, Aquarius, killian grad, proud owner of a kouki s14. Follow me I follow back. Instagram: Gabriel_ParkedS14", "followers_count": 97, "friends_count": 120, "statues_count": 2873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Three Lakes, FL", "id": "00c6858641196ed1", "name": "Three Lakes", "place_type": "city", "bounding_box": rectangle("-80.415865,25.6262 -80.379959,25.657246") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1271741, "cityName": "Three Lakes" } }
+{ "create_at": datetime("2015-11-24T10:00:31.000Z"), "id": 669214026530820096, "text": "why I always get fever blisters", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2978512956, "name": "ㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤ", "screen_name": "daltonroblin23", "lang": "en", "location": "wbhs ", "create_at": date("2015-01-12"), "description": "ig. @//daltonroblin23. Beísbol⚾️ Jr.", "followers_count": 478, "friends_count": 384, "statues_count": 875 }, "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("2015-11-24T10:00:32.000Z"), "id": 669214026648326144, "text": "UTSA almost out. https://t.co/ZhrwbZUnBu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1151288696, "name": "Javi C.", "screen_name": "BirdsUpJavi", "lang": "en", "location": "San Antonio, TX", "create_at": date("2013-02-05"), "description": "Staff Writer for @UTSARivals. @Rivals site for #UTSA football, basketball, & recruiting. #BirdsUp", "followers_count": 1282, "friends_count": 534, "statues_count": 4059 }, "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-24T10:00:32.000Z"), "id": 669214026715430914, "text": "@sportslarryknbr Let Timmy come back as Murphs clubhouse assistant and groom him to take over. More than that tell him to kick rocks.", "in_reply_to_status": -1, "in_reply_to_user": 333635486, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 333635486 }}, "user": { "id": 301846724, "name": "Jason M", "screen_name": "JayMinz", "lang": "en", "location": "San Francisco", "create_at": date("2011-05-19"), "description": "Giants, Sharks, World Champion Warriors and Niners. New found love of traveling. San Francisco native.", "followers_count": 118, "friends_count": 594, "statues_count": 2171 }, "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-24T10:00:32.000Z"), "id": 669214026899963905, "text": "@AlaskaAir my flight is delayed home to Seattle. Do something! Cancel another flight and bring that plane to my gate .......", "in_reply_to_status": -1, "in_reply_to_user": 13192972, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 13192972 }}, "user": { "id": 27188010, "name": "Kiko Cruz", "screen_name": "FJCruz", "lang": "en", "location": "San Diego, CA", "create_at": date("2009-03-27"), "description": "GOV. Bachelor. Insomniac. Periscope lurker. Foodie. Amazing person -you just don't know it yet. Northwest bound.", "followers_count": 80, "friends_count": 103, "statues_count": 1163 }, "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-24T10:00:32.000Z"), "id": 669214027038330880, "text": "#selfie with the GM of #stokesmazda Mr. Ed Seyle & #yourhondaguy Prince Sinsuat. @ Stokes Honda North https://t.co/XipFSqabcR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.04341361,32.96470972"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "selfie", "stokesmazda", "yourhondaguy" }}, "user": { "id": 25019816, "name": "Prince @Stokes Honda", "screen_name": "askforPRINCE", "lang": "en", "location": "Call or Text 843-817-8469", "create_at": date("2009-03-17"), "description": "Family man, Honda enthusiast, & Charleston SC's #1 volume Honda Sales Consultant! Serving over 4,000 clients at Stokes Honda North since 2001!", "followers_count": 489, "friends_count": 1016, "statues_count": 2437 }, "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("2015-11-24T10:00:32.000Z"), "id": 669214027084509188, "text": "Admirable, but difficult https://t.co/QcJbdCkIkn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 437695456, "name": "King Kassanova", "screen_name": "KingKassanova", "lang": "en", "location": "Space", "create_at": date("2011-12-15"), "description": "King Kassanova - Nino Brown 2-7-2015 (single) King Kassanova - The Matrix 4-20-2015 (album)", "followers_count": 399, "friends_count": 333, "statues_count": 7080 }, "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("2015-11-24T10:00:32.000Z"), "id": 669214027164327937, "text": "https://t.co/Bdb9OyikyF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 44641802, "name": "Nailheadmetalpunk", "screen_name": "SculpturebyNHMP", "lang": "en", "location": "new york", "create_at": date("2009-06-04"), "description": "Metal Sculptures made from recycled auto parts,all hand welded OAK pieces.\r\nArtist, Mechanic, From the Bronx.\r\nHot Rods Customs, all things mechanical.", "followers_count": 1952, "friends_count": 2121, "statues_count": 5959 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chester, NY", "id": "00f622ad4940616f", "name": "Chester", "place_type": "city", "bounding_box": rectangle("-74.310835,41.341301 -74.249668,41.382249") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36071, "countyName": "Orange", "cityID": 3615297, "cityName": "Chester" } }
+{ "create_at": datetime("2015-11-24T10:00:32.000Z"), "id": 669214027189391360, "text": "I'm only here so I don't get fined", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 995405761, "name": "Sean Martens", "screen_name": "ThaSituaSean", "lang": "en", "location": "null", "create_at": date("2012-12-07"), "description": "Caught in a bad bromance", "followers_count": 179, "friends_count": 201, "statues_count": 1425 }, "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("2015-11-24T10:00:32.000Z"), "id": 669214027357159424, "text": "always been the type to win in a losing situation...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3260298272, "name": "JADA.", "screen_name": "jadasky__", "lang": "en", "location": "null", "create_at": date("2015-06-29"), "description": "SC: jskyy_7 #tracknation", "followers_count": 451, "friends_count": 366, "statues_count": 1785 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Harper Woods, MI", "id": "93ff2ad4adf397c7", "name": "Harper Woods", "place_type": "city", "bounding_box": rectangle("-82.951773,42.421348 -82.910915,42.450733") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2636700, "cityName": "Harper Woods" } }
+{ "create_at": datetime("2015-11-24T10:00:32.000Z"), "id": 669214027474534400, "text": "Wish I was having this for lunch! �� https://t.co/7BlNjQYWNj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 588770893, "name": "Miriam", "screen_name": "Miriam_Elizondo", "lang": "en", "location": "null", "create_at": date("2012-05-23"), "description": "The biggest mistake you can make is removing jewels from your crown to make it easier for a man to carry.", "followers_count": 884, "friends_count": 857, "statues_count": 43232 }, "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("2015-11-24T10:00:32.000Z"), "id": 669214027709566976, "text": "cc: @CharlieBakerMA https://t.co/CDuOhIqffZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 54990588 }}, "user": { "id": 29232480, "name": "College Dems of MA", "screen_name": "CollegeDemsofMA", "lang": "en", "location": "77 Summer Street, Boston", "create_at": date("2009-04-06"), "description": "The official youth outreach arm of @MassDems and a chartered state federation of @CollegeDems. \nFacebook: http://t.co/Tf1npfcD7p \nInstagram: @macollegedems", "followers_count": 4633, "friends_count": 3553, "statues_count": 7428 }, "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("2015-11-24T10:00:32.000Z"), "id": 669214027751530496, "text": "It's true.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 340984821, "name": "N.", "screen_name": "ninanguyenx3", "lang": "en", "location": "If not Bella Noches den where?", "create_at": date("2011-07-23"), "description": "5'0 with an attitude|♒️| Made in USA & foreign ingredients| If i die, please dn't show my family my twitter|BLESSED| FYI: ya bitch borin' if she ain't foreign", "followers_count": 578, "friends_count": 350, "statues_count": 26870 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florence, SC", "id": "d49251f0d0a2652d", "name": "Florence", "place_type": "city", "bounding_box": rectangle("-79.918523,34.110516 -79.642093,34.254807") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45041, "countyName": "Florence", "cityID": 4525810, "cityName": "Florence" } }
+{ "create_at": datetime("2015-11-24T10:00:32.000Z"), "id": 669214028032376832, "text": "My uncle childish af.. \"I'm Jst watching Monday night football\" \"who's playing?\" \"DEEZ NUTS\" ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2571689365, "name": "Chacha.", "screen_name": "dashachasha", "lang": "en", "location": "K", "create_at": date("2014-06-16"), "description": "gram: dashachasha", "followers_count": 1026, "friends_count": 744, "statues_count": 38402 }, "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("2015-11-24T10:00:32.000Z"), "id": 669214028254851072, "text": "@kt_bedrich i can't with this video", "in_reply_to_status": 669209812861890564, "in_reply_to_user": 490824503, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 490824503 }}, "user": { "id": 316900824, "name": "Rachel Belisle", "screen_name": "Rachel_Belisle", "lang": "en", "location": "null", "create_at": date("2011-06-13"), "description": "I'm so...Disfunkshunal...can you dig it?", "followers_count": 292, "friends_count": 281, "statues_count": 5186 }, "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-24T10:00:32.000Z"), "id": 669214029429125120, "text": "Mostly sunny this afternoon, high 63 (17 C). Low 42 (6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119753180, "name": "Jackson Weather", "screen_name": "JacksonWX", "lang": "en", "location": "Jackson, MS", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Jackson, MS", "followers_count": 294, "friends_count": 68, "statues_count": 6251 }, "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("2015-11-24T10:00:32.000Z"), "id": 669214029747912704, "text": "@FoxNews Americans might not be terrorised under another President.", "in_reply_to_status": 669204204515278848, "in_reply_to_user": 1367531, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1367531 }}, "user": { "id": 2397740541, "name": "Jamie Lloyd", "screen_name": "jmel1959", "lang": "en", "location": "null", "create_at": date("2014-03-09"), "description": "null", "followers_count": 222, "friends_count": 652, "statues_count": 2479 }, "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("2015-11-24T10:00:32.000Z"), "id": 669214029886410752, "text": "I've never slept with anything longer than my @Jawbone #UP3 band �� https://t.co/H1vvGwQkwE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "UP3" }}, "user_mentions": {{ 26276396 }}, "user": { "id": 255296123, "name": "Tan Ha", "screen_name": "tanisacolor", "lang": "en", "location": "Chicago, IL", "create_at": date("2011-02-20"), "description": "Relentlessly learning about the things that make life wonderful", "followers_count": 1095, "friends_count": 284, "statues_count": 2672 }, "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-24T10:00:32.000Z"), "id": 669214029999681537, "text": "Congrats to Lynn Casey, our former board chair, on this well-deserved recognition from @UofStThomasMN! https://t.co/lc4KPoJFZR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 87270609 }}, "user": { "id": 239903391, "name": "The Mpls Foundation", "screen_name": "mplsfoundation", "lang": "en", "location": "Minneapolis, MN", "create_at": date("2011-01-18"), "description": "The Minneapolis Foundation: Helping you support what you love, where you live.", "followers_count": 4347, "friends_count": 716, "statues_count": 2456 }, "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("2015-11-24T10:00:32.000Z"), "id": 669214030058401792, "text": "I can't even talk about myself for 20 mins how does mrs hale expect me to talk about a disease for 20 mins", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 322278497, "name": "Al••••", "screen_name": "AaeRae_", "lang": "en", "location": "Vero Beach, FL", "create_at": date("2011-06-22"), "description": "✞", "followers_count": 403, "friends_count": 270, "statues_count": 10542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vero Beach, FL", "id": "3cf05613c1dd6dbd", "name": "Vero Beach", "place_type": "city", "bounding_box": rectangle("-80.463302,27.587008 -80.346166,27.674983") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12061, "countyName": "Indian River", "cityID": 1274150, "cityName": "Vero Beach" } }
+{ "create_at": datetime("2015-11-24T10:00:32.000Z"), "id": 669214030163132416, "text": "#CareerArc #Job alert: Residency Director | TeamHealth | #MYRTLEBEACH, SC https://t.co/r1B4b5Qp9j #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.8201676,33.7585214"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CareerArc", "Job", "MYRTLEBEACH", "Jobs", "Hiring" }}, "user": { "id": 38235898, "name": "TMJ-SCM Jobs", "screen_name": "tmj_scm_jobs", "lang": "en", "location": "Myrtle Beach, SC", "create_at": date("2009-05-06"), "description": "Follow this account for geo-targeted Other job tweets in Myrtle Beach, SC from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 590, "friends_count": 519, "statues_count": 51 }, "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("2015-11-24T10:00:32.000Z"), "id": 669214030351896577, "text": "Can you recommend anyone for this #job? Release Of Information Clerk - https://t.co/xBr5uIB7o1 #OrangePark, FL #Clerical #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.706484,30.1660736"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "OrangePark", "Clerical", "Hiring", "CareerArc" }}, "user": { "id": 22967527, "name": "TMJ-JAX Cleric. Jobs", "screen_name": "tmj_jax_cler", "lang": "en", "location": "Jacksonville, FL", "create_at": date("2009-03-05"), "description": "Follow this account for geo-targeted Office/Clerical job tweets in Jacksonville, FL from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 251, "friends_count": 202, "statues_count": 42 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange Park, FL", "id": "00c9d1dd71ddc799", "name": "Orange Park", "place_type": "city", "bounding_box": rectangle("-81.775187,30.150961 -81.691619,30.193458") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12019, "countyName": "Clay", "cityID": 1252125, "cityName": "Orange Park" } }
+{ "create_at": datetime("2015-11-24T10:00:32.000Z"), "id": 669214030607708160, "text": "Top 3 best Marvel villains \n1. Loki\n2. Purple Man\n3. The Kingpin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1452187981, "name": "NateTehGr8", "screen_name": "Naythunnn", "lang": "en", "location": "Avengers HQ", "create_at": date("2013-05-23"), "description": "Best Roy on Earth. Jeseille is my butt buddy~", "followers_count": 515, "friends_count": 393, "statues_count": 28963 }, "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-24T10:00:33.000Z"), "id": 669214030972653568, "text": "Connally. It's Irish. And girl this ring to solidify our vows https://t.co/gF4KMbq1tG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 297235907, "name": "Ah", "screen_name": "GuruJeremy", "lang": "en", "location": "Dallas, Texas", "create_at": date("2011-05-11"), "description": "Search #JeremyHour for my videos on life\nSearch #GuruHour for lessons on enlightenment\n Inquiries: JeremyTheGuru@Gmail.com", "followers_count": 15265, "friends_count": 15369, "statues_count": 377082 }, "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-24T10:00:33.000Z"), "id": 669214031123750912, "text": "Order @LastDragonPizza on your iPhone using the Homemade Food app by Grazer Labs, inc.\nStop rummaging… https://t.co/qra8QhSdlR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.85786574,40.57380576"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2329390400 }}, "user": { "id": 2329390400, "name": "LAST DRAGON PIZZA", "screen_name": "LASTDRAGONPIZZA", "lang": "en", "location": "Arverne, NY", "create_at": date("2014-02-05"), "description": "#PizzaQueen Serving friends in #ROCKAWAY #NYC on WEDNESDAYS+FRIDAYS+SATURDAYS. Info: 917-780-2570. #Pizza Follow on FB & INSTAGRAM! https://youtu.be/m0OM1rm7koE", "followers_count": 368, "friends_count": 730, "statues_count": 1966 }, "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("2015-11-24T10:00:33.000Z"), "id": 669214031157268480, "text": "Ok lost my voice again����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 396091895, "name": "Melissa❁", "screen_name": "__melssss", "lang": "en", "location": "null", "create_at": date("2011-10-22"), "description": "✨you know you love me xoxo✨", "followers_count": 669, "friends_count": 503, "statues_count": 22538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laureldale, PA", "id": "b1fcb9795805017e", "name": "Laureldale", "place_type": "city", "bounding_box": rectangle("-75.92747,40.379543 -75.899619,40.39751") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42011, "countyName": "Berks", "cityID": 4241768, "cityName": "Laureldale" } }
+{ "create_at": datetime("2015-11-24T10:00:33.000Z"), "id": 669214031304130560, "text": "Yayyyyyy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3774259342, "name": "BRONYYYYYYYYYYYYYYYY", "screen_name": "SexyGrassi", "lang": "en", "location": "Bronx, NY", "create_at": date("2015-09-25"), "description": "Multifandom - I love making friends,i follow back - but Im also insane XD - I joined @ 12:00am Sep 25 2015 - my @ is random asf", "followers_count": 1177, "friends_count": 3264, "statues_count": 8285 }, "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("2015-11-24T10:00:33.000Z"), "id": 669214031337512960, "text": "#FortWayne, IN #Healthcare #Job: Full Time PCT Fort Wayne at Fresenius Medical Care https://t.co/fz2XKvuE99 #dialysis #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.1393513,41.079273"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FortWayne", "Healthcare", "Job", "dialysis", "Jobs", "Hiring" }}, "user": { "id": 21725584, "name": "TMJ- IND Health Jobs", "screen_name": "tmj_ind_health", "lang": "en", "location": "Indianapolis, IN", "create_at": date("2009-02-23"), "description": "Follow this account for geo-targeted Healthcare job tweets in Indianapolis, IN. Need help? Tweet us at @CareerArc!", "followers_count": 383, "friends_count": 303, "statues_count": 807 }, "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("2015-11-24T10:00:33.000Z"), "id": 669214031488655360, "text": "I want Hulk Hogan & John Cena to save my life at least once together.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 148718447, "name": "Brandon Gerwig?", "screen_name": "Gerwig_", "lang": "en", "location": "Trenton, MI", "create_at": date("2010-05-27"), "description": "Lungs full of tar & mah tongue out mah mouth.", "followers_count": 653, "friends_count": 378, "statues_count": 32647 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Trenton, MI", "id": "3c4cb62907491ed8", "name": "Trenton", "place_type": "city", "bounding_box": rectangle("-83.218644,42.111921 -83.165338,42.170748") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2680420, "cityName": "Trenton" } }
+{ "create_at": datetime("2015-11-24T10:00:33.000Z"), "id": 669214031807250432, "text": "I'm going to throw this computer out 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": 338781616, "name": "raven ", "screen_name": "ravenvenegas", "lang": "en", "location": "Upper East Side ", "create_at": date("2011-07-19"), "description": "heavy is the head that wears the crown, don't let the greatness get you down.\nWalshU ⛳", "followers_count": 500, "friends_count": 451, "statues_count": 13563 }, "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("2015-11-24T10:00:33.000Z"), "id": 669214031979220992, "text": "@AntineeInBed @MalikiCox me 2", "in_reply_to_status": 669211779877109760, "in_reply_to_user": 1414529539, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1414529539, 770897364 }}, "user": { "id": 830501142, "name": "Ginger Snap", "screen_name": "erinn_nicholee", "lang": "en", "location": "null", "create_at": date("2012-09-17"), "description": "RHS??? / dance", "followers_count": 859, "friends_count": 619, "statues_count": 17401 }, "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("2015-11-24T10:00:33.000Z"), "id": 669214032017145858, "text": "#TimelineTuesday https://t.co/r1D1FufE7g", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "TimelineTuesday" }}, "user": { "id": 21469579, "name": "Ryan Blaske", "screen_name": "RyanBlaske", "lang": "en", "location": "Midwest // Chicago ", "create_at": date("2009-02-20"), "description": "Husband to @staceykozy // Filmmaker at University of Notre Dame @VideoND // Exclusive Contributor for @Dissolve // Always Learning, Always Creating", "followers_count": 379, "friends_count": 432, "statues_count": 2911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Notre Dame, IN", "id": "0001c8db0b56de00", "name": "Notre Dame", "place_type": "city", "bounding_box": rectangle("-86.250688,41.692886 -86.235438,41.708875") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18141, "countyName": "St. Joseph", "cityID": 1855386, "cityName": "Notre Dame" } }
+{ "create_at": datetime("2015-11-24T10:00:33.000Z"), "id": 669214032474349568, "text": "@sietaylor wow you're actually loved for once:-)", "in_reply_to_status": 669169576261902337, "in_reply_to_user": 370906674, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 370906674 }}, "user": { "id": 733038482, "name": "Caynnan", "screen_name": "CaynnanTaylorr", "lang": "en", "location": "Maryland, USA", "create_at": date("2012-08-02"), "description": "Professional canoer. hmu on kik if you want to join team. 3x silver Olympic medalist. Soon to be proud box turtle owner. Average at Best", "followers_count": 168, "friends_count": 274, "statues_count": 2745 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bel Air North, MD", "id": "a586c0a0034ff5a4", "name": "Bel Air North", "place_type": "city", "bounding_box": rectangle("-76.434737,39.515467 -76.289209,39.599101") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24025, "countyName": "Harford", "cityID": 2405825, "cityName": "Bel Air North" } }
+{ "create_at": datetime("2015-11-24T10:00:33.000Z"), "id": 669214032491094017, "text": "Whole lotta money that's my BFF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 471269897, "name": "Sir Garrick Rocks", "screen_name": "SirGarrickRocks", "lang": "en", "location": "Tallahassee | Palm Beach", "create_at": date("2012-01-22"), "description": "@PersonalityFits Intern | 20 & BOOMIN'", "followers_count": 307, "friends_count": 436, "statues_count": 10967 }, "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("2015-11-24T10:00:33.000Z"), "id": 669214032524627968, "text": "@Caese_Rulez I mean at Best Buy (where I work) you get gift cards $100-250 and if you are required a down payment you can use the gift card.", "in_reply_to_status": 669199798738755584, "in_reply_to_user": 427931760, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 427931760 }}, "user": { "id": 279244444, "name": "W▽", "screen_name": "ICauseCalamity", "lang": "en", "location": "Maryland", "create_at": date("2011-04-08"), "description": "Brenda|EDM|Tbh I prob hate you|jk|", "followers_count": 345, "friends_count": 259, "statues_count": 16588 }, "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("2015-11-24T10:00:33.000Z"), "id": 669214032608423936, "text": "Cloudy this afternoon, high 66 (19 C). Low 58 (14 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119754993, "name": "Waco Weather", "screen_name": "WacoWeather", "lang": "en", "location": "Waco, TX", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Waco, TX", "followers_count": 808, "friends_count": 68, "statues_count": 6231 }, "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.069786,31.639828") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4876000, "cityName": "Waco" } }
+{ "create_at": datetime("2015-11-24T10:00:33.000Z"), "id": 669214032675667969, "text": "Update on the detoxing ladies: lunch today includes their \"detox water\" and what appears to be Olive Garden leftover AND a bag of Cheetos", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 108710943, "name": "Jessica Hicks", "screen_name": "jch919", "lang": "en", "location": "Cusetown, USA", "create_at": date("2010-01-26"), "description": "its gonna be weird", "followers_count": 369, "friends_count": 838, "statues_count": 4556 }, "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("2015-11-24T10:00:33.000Z"), "id": 669214032784547840, "text": "@JusticeXena that's why I'm single", "in_reply_to_status": 669213554915061761, "in_reply_to_user": 494540396, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 494540396 }}, "user": { "id": 311125861, "name": "Keaton", "screen_name": "HellaciousHicks", "lang": "en", "location": "Manhattan, KS", "create_at": date("2011-06-04"), "description": "#KSU ΔΧ 5ever alone", "followers_count": 731, "friends_count": 484, "statues_count": 5219 }, "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("2015-11-24T10:00:33.000Z"), "id": 669214033296367616, "text": "@OSUGear @Nike ye haw #cowboystate @savvychic7 @JoAnnBelaus texas ��", "in_reply_to_status": 669166911976968194, "in_reply_to_user": 1323157578, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "cowboystate" }}, "user_mentions": {{ 1323157578, 415859364, 3038876083, 1169261954 }}, "user": { "id": 35033769, "name": "CorreyHopeTravel", "screen_name": "CorreyHope", "lang": "en", "location": "here there & everywhere", "create_at": date("2009-04-24"), "description": "⚓⌚️time2travel⏱️Looking for a trip, no hassles, with a reliable agent?Look no further. Mom and Special Education Teacher.Blogger of Travel & ED", "followers_count": 3581, "friends_count": 3367, "statues_count": 123535 }, "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-24T10:00:33.000Z"), "id": 669214033887813632, "text": "sitting in class crying about all the lil kitties I'll never get 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": 3397773243, "name": "caroline", "screen_name": "carolineedwardz", "lang": "en", "location": "null", "create_at": date("2015-07-31"), "description": "peace will win and fear will lose", "followers_count": 161, "friends_count": 149, "statues_count": 462 }, "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("2015-11-24T10:00:33.000Z"), "id": 669214034139480064, "text": "@vjm_jolie @_CollegeHumor_ buy me one for Christmas", "in_reply_to_status": 669209914179493888, "in_reply_to_user": 359142665, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 359142665, 724223456 }}, "user": { "id": 344160023, "name": "Sophia DeVito", "screen_name": "sophia_devito", "lang": "en", "location": "Mechanicville | Plattsburgh 19", "create_at": date("2011-07-28"), "description": "null", "followers_count": 912, "friends_count": 473, "statues_count": 21170 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westmere, NY", "id": "8f86ad23716cb2c6", "name": "Westmere", "place_type": "city", "bounding_box": rectangle("-73.924378,42.664366 -73.838217,42.702642") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36001, "countyName": "Albany", "cityID": 3680423, "cityName": "Westmere" } }
+{ "create_at": datetime("2015-11-24T10:00:33.000Z"), "id": 669214034567307264, "text": "Wind 8 mph NNE. Barometer 30.32 in, Falling. Temperature 74.1 °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": 50, "friends_count": 67, "statues_count": 23804 }, "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("2015-11-24T10:00:33.000Z"), "id": 669214034839797760, "text": "@IBalenduS during the finals?", "in_reply_to_status": 669211838819659776, "in_reply_to_user": 132850616, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 132850616 }}, "user": { "id": 141172449, "name": "Rishi vijay", "screen_name": "rishi_saladi", "lang": "en", "location": "Houston, TX", "create_at": date("2010-05-07"), "description": "Read, Tv, Play, Bikes, Elon, Novak, Arsenal!", "followers_count": 23, "friends_count": 214, "statues_count": 435 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:33.000Z"), "id": 669214034843996162, "text": "@AnnaKendrick47 looks like a tiny Tina Fey. #AndIfYaDontKnowNowYaKnow", "in_reply_to_status": -1, "in_reply_to_user": 59949396, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AndIfYaDontKnowNowYaKnow" }}, "user_mentions": {{ 59949396 }}, "user": { "id": 39803017, "name": "Joe Cox", "screen_name": "JoeBoy90", "lang": "en", "location": "Tucson, AZ", "create_at": date("2009-05-13"), "description": "You can't make eggs without cracking a few omelets. #Tizzie Bacon enthusiast. #Frenchie YouTuber. @BigBang_CBS fan. #Clipster @nbcsnl is my fave show #BearDown", "followers_count": 86, "friends_count": 1130, "statues_count": 1804 }, "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-24T10:00:34.000Z"), "id": 669214035389210625, "text": "Why do we have to take the kids to get Amanda's car?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1271362063, "name": "pablo", "screen_name": "Hagan_Hoskins", "lang": "en", "location": "null", "create_at": date("2013-03-15"), "description": "October 4th #clappedoutcrew", "followers_count": 191, "friends_count": 187, "statues_count": 7172 }, "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("2015-11-24T10:00:34.000Z"), "id": 669214035481657344, "text": "I got like 3 meals lined up on Thursday ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 619590814, "name": "OverLooked", "screen_name": "Bdowlen_LOE", "lang": "en", "location": "Louisville, KY/ Pembroke, KY", "create_at": date("2012-06-26"), "description": "| Simmons College of Ky | IG: @Bdowlen_LOE", "followers_count": 974, "friends_count": 770, "statues_count": 34685 }, "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-24T10:00:34.000Z"), "id": 669214035586322434, "text": "@Devonte_20 click on this and watch the bottom video.. ���������������� https://t.co/MKr2SOaZCy", "in_reply_to_status": -1, "in_reply_to_user": 421626399, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 421626399 }}, "user": { "id": 1026563522, "name": "Jesse", "screen_name": "Joswalt1", "lang": "en", "location": "ETX ", "create_at": date("2012-12-21"), "description": "#BoomerSooner #Baker4Heisman", "followers_count": 595, "friends_count": 423, "statues_count": 15209 }, "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("2015-11-24T10:00:34.000Z"), "id": 669214035586490368, "text": "@alicemazzy Consider updating make to bazel. It's what Googlers use to compile their code. https://t.co/tjzEq6FPzO", "in_reply_to_status": 669154896520851456, "in_reply_to_user": 63506279, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 63506279 }}, "user": { "id": 370418635, "name": "Justine Tunney", "screen_name": "JustineTunney", "lang": "en", "location": "Concrete Jungle, NY", "create_at": date("2011-09-08"), "description": "Counter-counter-counter-revolutionary. My views do not represent Google's.", "followers_count": 4254, "friends_count": 941, "statues_count": 14603 }, "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("2015-11-24T10:00:34.000Z"), "id": 669214035615748096, "text": "@mtomlin94 you were home on Sunday though ��", "in_reply_to_status": 669202640543203329, "in_reply_to_user": 979590360, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 979590360 }}, "user": { "id": 4035404179, "name": "Zach Wilson", "screen_name": "ZachWilson_27", "lang": "en", "location": "null", "create_at": date("2015-10-27"), "description": "Follower of Christ. Future Father of America. Ballplayer. #HutchBlueDragonAlum", "followers_count": 140, "friends_count": 208, "statues_count": 339 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lantana, TX", "id": "01ecbd283dbfeb57", "name": "Lantana", "place_type": "city", "bounding_box": rectangle("-97.151745,33.072178 -97.107345,33.113247") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4841404, "cityName": "Lantana" } }
+{ "create_at": datetime("2015-11-24T10:00:34.000Z"), "id": 669214035737362432, "text": "@yungwitchcraft he's like \"babe your toes are so cute\" I'm like don't lie to my corn", "in_reply_to_status": 669213895983149060, "in_reply_to_user": 489850584, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 489850584 }}, "user": { "id": 245007253, "name": "Nataly Quinonez", "screen_name": "natalyquinonez", "lang": "en", "location": "null", "create_at": date("2011-01-30"), "description": "coco butter bitch", "followers_count": 119, "friends_count": 99, "statues_count": 7173 }, "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("2015-11-24T10:00:34.000Z"), "id": 669214035741511680, "text": "Partly cloudy this afternoon, high 58 (14 C). Low 35 (2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119755728, "name": "Louisville Weather", "screen_name": "Louisville_WX", "lang": "en", "location": "Louisville, KY", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Louisville, KY", "followers_count": 1302, "friends_count": 68, "statues_count": 6258 }, "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-24T10:00:34.000Z"), "id": 669214036509065216, "text": "@wolfrvbe it's eventually going to snow and I'm not down for no fucking snow", "in_reply_to_status": 669213860528697344, "in_reply_to_user": 1282215332, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1282215332 }}, "user": { "id": 1902722840, "name": "ᴸᵁᴸᵁ❥", "screen_name": "kissaurora_", "lang": "en", "location": "Oregon, USA", "create_at": date("2013-09-24"), "description": "ᶠᵁᵀᵁᴿᴱ ᴴᴼᴹᴱ ᶜᴬᴸᴵᶠᴼᴿᴺᴵᴬ", "followers_count": 746, "friends_count": 517, "statues_count": 27986 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eagle Point, OR", "id": "c9c9e99176b473a9", "name": "Eagle Point", "place_type": "city", "bounding_box": rectangle("-122.83262,42.450963 -122.782306,42.487297") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41029, "countyName": "Jackson", "cityID": 4121550, "cityName": "Eagle Point" } }
+{ "create_at": datetime("2015-11-24T10:00:34.000Z"), "id": 669214036689420288, "text": "Put the ��⬇️ & enjoy your ��‍��‍��‍�� this season! Life is always changing, so cherish those here now! ❤️ https://t.co/PQdMzcFhmm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24220965, "name": "Angie Daniels", "screen_name": "AngieMDan", "lang": "en", "location": "null", "create_at": date("2009-03-13"), "description": "null", "followers_count": 105, "friends_count": 140, "statues_count": 125 }, "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("2015-11-24T10:00:34.000Z"), "id": 669214036760760320, "text": "If you want to\nLove me now,\nThen you have\nTo learn to love\nMy past. Because\nThe struggles I\nWent through\nYesterday\nMade me who\nI 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": 3585711972, "name": "Brookie Cookie", "screen_name": "brookie1939", "lang": "en", "location": "Allen, TX", "create_at": date("2015-09-16"), "description": "All the make up in the world won't make u less insecure❤️ Dalton stole my heart on 8-29-15 @daltoncgeorge__", "followers_count": 235, "friends_count": 328, "statues_count": 2288 }, "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("2015-11-24T10:00:34.000Z"), "id": 669214036794454016, "text": "With @raywongy eating lunch!! (at @ShakeShack in New York, NY) https://t.co/1OAjtolymt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.98817778,40.74149794"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 100375482, 212255045 }}, "user": { "id": 9866582, "name": "Christina Warren", "screen_name": "film_girl", "lang": "en", "location": "Brooklyn, New York", "create_at": date("2007-11-01"), "description": "Senior Tech Correspondent at Mashable, co-host @ovrtrd & @_RocketFM & @bbgtl. Obsessed with media and technology. I rule. christina@mashable.com opinions = own", "followers_count": 74377, "friends_count": 10608, "statues_count": 69427 }, "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("2015-11-24T10:00:34.000Z"), "id": 669214037364727812, "text": "Unfollowing everyone who got food being posted Thanksgiving day best believe that☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 299462605, "name": "☄©®➰➰K☄", "screen_name": "ClaytonCrook", "lang": "en", "location": "☄MO CITY TX☄", "create_at": date("2011-05-15"), "description": "✨The Last Mansa ✨ HTX✈️CoMo", "followers_count": 1591, "friends_count": 939, "statues_count": 8623 }, "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("2015-11-24T10:00:34.000Z"), "id": 669214037654175744, "text": "@Steffvannesa why did I have a weird feeling you would answer that ��", "in_reply_to_status": 669213824642211846, "in_reply_to_user": 178482322, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 178482322 }}, "user": { "id": 1007986212, "name": "Adriana Gonzalez", "screen_name": "adriana_gon27", "lang": "en", "location": "null", "create_at": date("2012-12-12"), "description": "UNT 19", "followers_count": 151, "friends_count": 179, "statues_count": 3442 }, "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("2015-11-24T10:00:34.000Z"), "id": 669214037847109632, "text": "Don't skip lunch.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21630579, "name": "Adrian Rivera", "screen_name": "_adrianrivera", "lang": "en", "location": "null", "create_at": date("2009-02-22"), "description": "null", "followers_count": 533, "friends_count": 1104, "statues_count": 4454 }, "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-24T10:00:34.000Z"), "id": 669214037968887808, "text": "Graduation can't come fast enough!��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 412589956, "name": "Nobody But ©⭕️®ey", "screen_name": "ELITE23_CT", "lang": "en", "location": "Mo-City, TX & LSU", "create_at": date("2011-11-14"), "description": "Dushaun Thompson: LSU (DBU) FS, Sports Marketing major, blah blah blah, whatever...Twitter is all fun & games.", "followers_count": 2314, "friends_count": 1037, "statues_count": 26221 }, "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("2015-11-24T10:00:34.000Z"), "id": 669214037981437953, "text": "Giordano's with \"Italian born, Chicago raised\" on their front sign ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28296570, "name": "Nick.", "screen_name": "nburzych", "lang": "en", "location": "Chicago, IL", "create_at": date("2009-04-01"), "description": "Full time banker, full time broadcasting school student, full time jackass. Man behind @HuskieReview. Coming to a radio near you...", "followers_count": 1449, "friends_count": 2005, "statues_count": 112685 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakbrook Terrace, IL", "id": "b1ca84ec4c7a122e", "name": "Oakbrook Terrace", "place_type": "city", "bounding_box": rectangle("-87.992587,41.844781 -87.948325,41.868679") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1754560, "cityName": "Oakbrook Terrace" } }
+{ "create_at": datetime("2015-11-24T10:00:34.000Z"), "id": 669214038224584704, "text": "Damnnnn https://t.co/Zn4ZCoGcJV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3049800856, "name": "jazzzyjaz", "screen_name": "jazzlitty", "lang": "en", "location": "abuelas kitchen", "create_at": date("2015-02-21"), "description": "null", "followers_count": 218, "friends_count": 205, "statues_count": 15095 }, "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("2015-11-24T10:00:34.000Z"), "id": 669214038404935680, "text": "@Mr_McStevie check your contract get a lawyer @TMZ", "in_reply_to_status": 669201011593109505, "in_reply_to_user": 198337759, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 198337759, 16331010 }}, "user": { "id": 3292500478, "name": "TQSB5689", "screen_name": "tqsb56891", "lang": "en", "location": "United States ", "create_at": date("2015-05-20"), "description": "Cognitve Experimental Statistical NeuroScience Artificial Intelligence Robotics Physics Chemistry Design", "followers_count": 74, "friends_count": 326, "statues_count": 18689 }, "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("2015-11-24T10:00:34.000Z"), "id": 669214038514102272, "text": "I love black people. https://t.co/np9qm6Z7Mn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 476404729, "name": "imani", "screen_name": "imani_greenMB", "lang": "en", "location": "Bossed up w/ Sebastian ", "create_at": date("2012-01-27"), "description": "If you don't want me, dont fuck with my feelings.", "followers_count": 1126, "friends_count": 2034, "statues_count": 25644 }, "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("2015-11-24T10:00:34.000Z"), "id": 669214038782447616, "text": "@MOFFITW @braulioadiaz @cf97history like this is all bland sports speak, just as much as THP is bland sports speak", "in_reply_to_status": 669209819027546112, "in_reply_to_user": 1694065164, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1694065164, 151660489, 3053696853 }}, "user": { "id": 28877873, "name": "It's Louis You Guys", "screen_name": "StevensBoudreau", "lang": "en", "location": "2 Blue Stripes, 4 Red Stars", "create_at": date("2009-04-04"), "description": "Once retweeted by @amthna. Followed by @MattDunphey. #EamusCatuli #wearegood", "followers_count": 525, "friends_count": 750, "statues_count": 63188 }, "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-24T10:00:34.000Z"), "id": 669214039109713920, "text": "Professors are people toooooo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165629230, "name": "Kay Kay", "screen_name": "KaylinKaye", "lang": "en", "location": "null", "create_at": date("2010-07-11"), "description": "IM ODD YOUR EVEN #UK #ESC", "followers_count": 1034, "friends_count": 768, "statues_count": 60110 }, "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-24T10:00:35.000Z"), "id": 669214039289913344, "text": "Whenever I wish someone happy birthday and they don't acknowledge me, I make a mental note to never wish them happy birthday again��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 166670233, "name": "муяєкℓє♈️™", "screen_name": "MyrekleWhhiipp", "lang": "en", "location": "Texas, USA", "create_at": date("2010-07-14"), "description": "null", "followers_count": 435, "friends_count": 459, "statues_count": 7457 }, "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-24T10:00:35.000Z"), "id": 669214039478702082, "text": "hopefully goku becomes hokage in the end like him dad was.", "in_reply_to_status": 669213819093192704, "in_reply_to_user": 913755007, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 913755007, "name": "snack cake", "screen_name": "heypunkass", "lang": "en", "location": "316", "create_at": date("2012-10-29"), "description": "kella • 20 • half gay, half dead inside. genderfluid scorpio, intersectional feminist. @t_bearz. ||032313|| it's cold in the deep end.", "followers_count": 86, "friends_count": 74, "statues_count": 21663 }, "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("2015-11-24T10:00:35.000Z"), "id": 669214039495581696, "text": "@HammerNation19 f u idiot always trying to dump on authority . True limp waisted lib", "in_reply_to_status": 669205575478198272, "in_reply_to_user": 385989819, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 385989819 }}, "user": { "id": 541221393, "name": "me", "screen_name": "vandoosky", "lang": "en", "location": "here", "create_at": date("2012-03-30"), "description": "null", "followers_count": 22, "friends_count": 148, "statues_count": 286 }, "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("2015-11-24T10:00:35.000Z"), "id": 669214039600271360, "text": "@Dance6odPls I go to galena park.", "in_reply_to_status": 669213948625874944, "in_reply_to_user": 2385241507, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2385241507 }}, "user": { "id": 1332667434, "name": "nani", "screen_name": "Valerieeexo_", "lang": "en", "location": "clutch city , tx", "create_at": date("2013-04-06"), "description": "what do you mean ?", "followers_count": 1327, "friends_count": 1895, "statues_count": 33513 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-24T10:00:35.000Z"), "id": 669214039604596736, "text": "Ion even smoke, Ion even smoke.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 631608528, "name": "DeeiNferno", "screen_name": "Dee_lamery", "lang": "en", "location": "null", "create_at": date("2012-07-09"), "description": "Ur a lil pussy.", "followers_count": 1455, "friends_count": 1357, "statues_count": 11306 }, "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("2015-11-24T10:00:35.000Z"), "id": 669214039747239936, "text": "@menloinnovation .... I just discovered you today. I like what I see!", "in_reply_to_status": -1, "in_reply_to_user": 25995457, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25995457 }}, "user": { "id": 92264739, "name": "OpsTempo", "screen_name": "OpsTempo", "lang": "en", "location": "Rome, GA", "create_at": date("2009-11-24"), "description": "CHARLES HOWELL -- PMP, ITIL, KCS, Project Management, Resource Planning, Healthcare IT Security. HR Business Partner, PMAikido Master. #PMOT", "followers_count": 1029, "friends_count": 2071, "statues_count": 3222 }, "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("2015-11-24T10:00:35.000Z"), "id": 669214040019742724, "text": "@GoGeocaching @girlscouts amazing!", "in_reply_to_status": 669210249409093632, "in_reply_to_user": 72675862, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 72675862, 103018203 }}, "user": { "id": 14430075, "name": "Andrea Ball", "screen_name": "AndreaBall", "lang": "en", "location": "Los Angeles", "create_at": date("2008-04-17"), "description": "International commercial director, writing, and dancing till dawn", "followers_count": 2244, "friends_count": 629, "statues_count": 1255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-24T10:00:35.000Z"), "id": 669214040103583744, "text": "The condition of my bedroom reflects the condition of my life �� Right now my bedroom is a fucking mess.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 468993496, "name": "Chels", "screen_name": "chelseeea_g", "lang": "en", "location": "null", "create_at": date("2012-01-19"), "description": "A weirdo but im real though ♓️ RIP J.C.S.", "followers_count": 658, "friends_count": 321, "statues_count": 21847 }, "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("2015-11-24T10:00:35.000Z"), "id": 669214040460136448, "text": "Be attentive, don't be a prick, and stay loyal. Is it that hard?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2399207552, "name": "Blondre 3000", "screen_name": "xCHaines", "lang": "en", "location": "Patterson, CA", "create_at": date("2014-03-20"), "description": "♎️ Cody Haines|PHS Junior|Martial artist|Musician|Competitive gymnast|R.I.P. E ❤️", "followers_count": 116, "friends_count": 168, "statues_count": 4613 }, "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("2015-11-24T10:00:35.000Z"), "id": 669214040477057029, "text": "sweat to god this is me https://t.co/fsqGSAKVVv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 556463842, "name": "mazzie", "screen_name": "_mpcx_", "lang": "en", "location": "Michigan, USA", "create_at": date("2012-04-17"), "description": "rhs | varsity soccer | jac is the best BFF ever |", "followers_count": 609, "friends_count": 588, "statues_count": 9356 }, "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("2015-11-24T10:00:35.000Z"), "id": 669214040606969860, "text": "@UmeniynoraDaII @iAmTreboR_TSOE �� we coming back...", "in_reply_to_status": 669193965971767296, "in_reply_to_user": 258641669, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 258641669, 230136714 }}, "user": { "id": 356422240, "name": "B Thomp™", "screen_name": "Kingthomp_", "lang": "en", "location": "null", "create_at": date("2011-08-16"), "description": "Huston Tillotson Basketball #0️⃣", "followers_count": 1106, "friends_count": 766, "statues_count": 21300 }, "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("2015-11-24T10:00:35.000Z"), "id": 669214040888123393, "text": "Yay I'm dying :D", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1615470782, "name": "Steven Sergal", "screen_name": "steven_sergal", "lang": "en", "location": "Marietta, GA", "create_at": date("2013-07-23"), "description": "A big monkey from the giant flying spaghetti monster.", "followers_count": 47, "friends_count": 46, "statues_count": 101 }, "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("2015-11-24T10:00:35.000Z"), "id": 669214041123004416, "text": "Because today is my Friday! @thewhiskeywolf_ @shopwhiskywater ������☕️�� @… https://t.co/6iyuFow1E2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.78149065,36.19188806"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 453640627, 3589734434 }}, "user": { "id": 95403106, "name": "CarriAnne Stone", "screen_name": "stonec09", "lang": "en", "location": "Nasville | TN ", "create_at": date("2009-12-08"), "description": "Born and raised Illinoisan, currently a nashvillian and loving every minute of adventure life is bringing me... adventure is out there", "followers_count": 229, "friends_count": 442, "statues_count": 1524 }, "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.99451 -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("2015-11-24T10:00:35.000Z"), "id": 669214041328455685, "text": "@HaleySavannah_ I don't know about all of that, but I get pretty accurate with my left hand in beer pong as the night goes on. Lol.", "in_reply_to_status": 669208099262083072, "in_reply_to_user": 144357709, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 144357709 }}, "user": { "id": 68333173, "name": "Tasha Sade Nickerson", "screen_name": "Tasha_Face", "lang": "en", "location": "Hollywood, California ", "create_at": date("2009-08-23"), "description": "24. ♐️ Musicians Institute. Guitar Player/Singer/Song writer. I Live and breathe for Music. A life without music, Isnt worth living. Happily Taken. 02-09-15 ❤️", "followers_count": 534, "friends_count": 269, "statues_count": 16605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-24T10:00:35.000Z"), "id": 669214041357840385, "text": "@CoachTheDON @MiggFAM @TweetfrmtheSole @The_Fat_John @troy_jefferson @JR_BeFoolin any of yall got a flag set?", "in_reply_to_status": 669199546896007168, "in_reply_to_user": 83470893, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 83470893, 33209796, 259748646, 91261704, 849874681, 276637168 }}, "user": { "id": 98221214, "name": "Dwayne Wayne III", "screen_name": "_92Infinity", "lang": "en", "location": "D [M] V", "create_at": date("2009-12-20"), "description": "| UMD Alum | 23 | ESFP |", "followers_count": 1315, "friends_count": 984, "statues_count": 164410 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Konterra, MD", "id": "005fa99b62024974", "name": "Konterra", "place_type": "city", "bounding_box": rectangle("-76.92979,39.051029 -76.868355,39.103992") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2444715, "cityName": "Konterra" } }
+{ "create_at": datetime("2015-11-24T10:00:35.000Z"), "id": 669214041450004480, "text": "@CakeLikeBeth Fkekwlfkfkfd LMG. DKDKCKDKCP", "in_reply_to_status": 669213601266343936, "in_reply_to_user": 2258061044, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 2258061044 }}, "user": { "id": 258383572, "name": "Bloody Markyy", "screen_name": "MarkyyHollywood", "lang": "en", "location": "Lady Gaga's Nose", "create_at": date("2011-02-27"), "description": "Kanye and Gaga's love child.", "followers_count": 1292, "friends_count": 226, "statues_count": 71998 }, "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("2015-11-24T10:00:35.000Z"), "id": 669214041454198788, "text": "@puebla34 you get my message?", "in_reply_to_status": 669213827708248064, "in_reply_to_user": 575518763, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 575518763 }}, "user": { "id": 474477901, "name": "Rod", "screen_name": "CallMeTaesean", "lang": "en", "location": "Desoto, TX", "create_at": date("2012-01-25"), "description": "Rest in peace pops! love you forever. MVC College. How Can I Get Killed for It If I'm Built For It? #HTTR #KnicksTape", "followers_count": 1132, "friends_count": 691, "statues_count": 69261 }, "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-24T10:00:35.000Z"), "id": 669214041592758272, "text": "@deirdrecee @riychu FIC IT DEI", "in_reply_to_status": 669213389235843073, "in_reply_to_user": 24041349, "favorite_count": 0, "retweet_count": 0, "lang": "ro", "is_retweet": false, "user_mentions": {{ 24041349, 2529255813 }}, "user": { "id": 156110658, "name": "188-HOT-LEGS", "screen_name": "iwinsoiwin", "lang": "en", "location": "null", "create_at": date("2010-06-15"), "description": "You trying to tell me you're a bigger knob than you've been letting on?", "followers_count": 525, "friends_count": 453, "statues_count": 85897 }, "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-24T10:00:35.000Z"), "id": 669214041609498624, "text": "@CORSAIRmiami nope what's a good restaurant near fort Lauderdale? Send me 25% coupon hook me up ��", "in_reply_to_status": 669213202828382208, "in_reply_to_user": 2804300748, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2804300748 }}, "user": { "id": 1265238146, "name": "DrizzyDrake", "screen_name": "dnissen24", "lang": "en", "location": "Atlanta, GA", "create_at": date("2013-03-13"), "description": "21. Tattoos. Leo. Single. Young & Reckless", "followers_count": 191, "friends_count": 99, "statues_count": 4775 }, "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("2015-11-24T10:00:35.000Z"), "id": 669214041634680832, "text": "Thanks for all the follows,I'm notabsent on purpose. Had a stroke over the weeke", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 910753122, "name": "MajrPayneNurAss", "screen_name": "cobwebznmyboxrz", "lang": "en", "location": "NurAss", "create_at": date("2012-10-28"), "description": "i have trained 20,000 combat killing machines under this hat. Give me a sniper rifle point me in his general direction & get the fuck out of my way", "followers_count": 7510, "friends_count": 7677, "statues_count": 8785 }, "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("2015-11-24T10:00:35.000Z"), "id": 669214041672413184, "text": "@Doylemakinmoves ������ u know how to get to Simmons or the Dowd if u was Ray Charles twin brother tho", "in_reply_to_status": 669213794632122368, "in_reply_to_user": 62721074, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 62721074 }}, "user": { "id": 97826314, "name": "6:30 Trizzy", "screen_name": "Trizyckerodgers", "lang": "en", "location": "tryna make a billion", "create_at": date("2009-12-18"), "description": "I Just want my Bank account to look Crazy & come home to thick Hiapanic goddess everyday.", "followers_count": 512, "friends_count": 515, "statues_count": 51165 }, "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("2015-11-24T10:00:35.000Z"), "id": 669214041680830464, "text": "We are taking pre-orders on the Air Jordan Retro 6 \"Maroon.\" The price will be $250 plus tax for men… https://t.co/T8ELxZYs9S", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.7286606,33.5281181"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18320853, "name": "Sneaker Lounge", "screen_name": "SneakerLoungeAL", "lang": "en", "location": "Birmingham, AL", "create_at": date("2008-12-22"), "description": "7001 Crestwood Blvd Suite 410 Birmingham, Alabama 35210 • 205-91-KICKS • Sunday 1-5PM • Monday: Closed • Tuesday-Thursday: 10-6PM • Friday-Sat: 11-7PM", "followers_count": 141, "friends_count": 193, "statues_count": 445 }, "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("2015-11-24T10:00:35.000Z"), "id": 669214041856806912, "text": "@dinahjane97 I have an extra bunk on my parahoy cabin. Come to cozumel with me hoe", "in_reply_to_status": -1, "in_reply_to_user": 259435698, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 259435698 }}, "user": { "id": 34407494, "name": "♕Mariana #5H2", "screen_name": "maaare", "lang": "en", "location": "null", "create_at": date("2009-04-22"), "description": "overprotective Dinally girl ⚠", "followers_count": 1716, "friends_count": 238, "statues_count": 187002 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wolf Trap, VA", "id": "9e7bac339bedc5f7", "name": "Wolf Trap", "place_type": "city", "bounding_box": rectangle("-77.317275,38.906273 -77.246819,38.977758") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5187240, "cityName": "Wolf Trap" } }
+{ "create_at": datetime("2015-11-24T10:00:35.000Z"), "id": 669214041894617088, "text": "Wishing I could physically block people from 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": 524958205, "name": "aly", "screen_name": "alyciaorndorf", "lang": "en", "location": "vhs senior", "create_at": date("2012-03-14"), "description": "just want to be wrapped up in a blanket watching a movie with my dog", "followers_count": 203, "friends_count": 185, "statues_count": 5948 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buckeye, AZ", "id": "0015cc0d71d49e19", "name": "Buckeye", "place_type": "city", "bounding_box": rectangle("-112.625877,33.355798 -112.461428,33.515442") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 407940, "cityName": "Buckeye" } }
+{ "create_at": datetime("2015-11-24T10:00:35.000Z"), "id": 669214042267910144, "text": "Every morning I grab my wallet and count", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 431271675, "name": "Feliz", "screen_name": "Felic_Amaya", "lang": "en", "location": "null", "create_at": date("2011-12-07"), "description": "#DWMTM", "followers_count": 497, "friends_count": 400, "statues_count": 17648 }, "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-24T10:00:35.000Z"), "id": 669214042297266176, "text": "����������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 303523830, "name": "Long Live Lenny", "screen_name": "RioDaFlippa", "lang": "en", "location": "null", "create_at": date("2011-05-22"), "description": "This is god's plan homie this ain't mine #1K #FloridaBoy 7✌️7 http://www.hudl.com/athlete/1450250/highlights/310773376/v2", "followers_count": 2656, "friends_count": 2348, "statues_count": 108701 }, "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("2015-11-24T10:00:35.000Z"), "id": 669214042444054528, "text": "The @Browns are a joke. Benching him because he was drinking? Quit treating @JManziel2 like he's a 9th grader.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 40358743, 342165503 }}, "user": { "id": 514923997, "name": "Anthony Salvatore", "screen_name": "Ant_Sal", "lang": "en", "location": "null", "create_at": date("2012-03-04"), "description": "living a #603 NH life but full time Floridian.", "followers_count": 199, "friends_count": 577, "statues_count": 2893 }, "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("2015-11-24T10:00:35.000Z"), "id": 669214042637148160, "text": "Pops is taking it hard, but in stride https://t.co/Q0tYdrZKBt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 571127653, "name": "Ben", "screen_name": "ben_died", "lang": "en", "location": "Tim's Bedroom ", "create_at": date("2012-05-04"), "description": "Filmmaker/pole dancer", "followers_count": 490, "friends_count": 327, "statues_count": 23933 }, "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("2015-11-24T10:00:35.000Z"), "id": 669214042641174528, "text": "kick back friday. my house hmu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 599606837, "name": "L Y D I A . W A P", "screen_name": "LydiaaKillEm", "lang": "en", "location": "null", "create_at": date("2012-06-04"), "description": "GodBlessinAllTheTrapNiggas.", "followers_count": 852, "friends_count": 490, "statues_count": 31951 }, "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("2015-11-24T10:00:35.000Z"), "id": 669214042641334273, "text": "Doing all my Christmas shopping online 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": 489943203, "name": "Karina", "screen_name": "_karinapineda", "lang": "en", "location": "null", "create_at": date("2012-02-11"), "description": "Instagram: @_karinapineda", "followers_count": 214, "friends_count": 194, "statues_count": 11399 }, "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-24T10:00:35.000Z"), "id": 669214042788126721, "text": "@TheWeirdVines @SavageJihad @leighann_shoe", "in_reply_to_status": 668189697781428224, "in_reply_to_user": 1242915108, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1242915108, 2305405063, 417256548 }}, "user": { "id": 2353562909, "name": "Kyle", "screen_name": "kmartellacci", "lang": "en", "location": "The Burg", "create_at": date("2014-02-19"), "description": "⚾️", "followers_count": 146, "friends_count": 236, "statues_count": 441 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hollidaysburg, PA", "id": "69837d8b6ba9810b", "name": "Hollidaysburg", "place_type": "city", "bounding_box": rectangle("-78.429385,40.400965 -78.339366,40.475159") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42013, "countyName": "Blair", "cityID": 4235224, "cityName": "Hollidaysburg" } }
+{ "create_at": datetime("2015-11-24T10:00:35.000Z"), "id": 669214043391963136, "text": "https://t.co/O6eSNbInUy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "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": 1171, "friends_count": 1814, "statues_count": 177697 }, "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-24T10:00:36.000Z"), "id": 669214043488452608, "text": "swear people don't know how to stay the fuck out of stuff that doesn't involve them in any shape, way, or form. FOH bitch.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1455504194, "name": "babygirl", "screen_name": "sstephzz", "lang": "en", "location": "Compton, CA", "create_at": date("2013-05-24"), "description": "Rather be feared, Than loved", "followers_count": 2096, "friends_count": 1829, "statues_count": 34341 }, "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("2015-11-24T10:00:36.000Z"), "id": 669214043635384320, "text": "More importantly there is no clear evidence presently that Turkey made a mistake in downing the Russian Fighter jet.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 261698166, "name": "Ronald B. Saunders ", "screen_name": "BlackBuzzNews", "lang": "en", "location": "Scarsdale, New York", "create_at": date("2011-03-06"), "description": "Vast expertise on many subjects and expertise on EEO, Africa& the Middle East.\r\nWriter, poet, human & civil rights activist.\r\nFounder, NBPC.", "followers_count": 1206, "friends_count": 1942, "statues_count": 83248 }, "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("2015-11-24T10:00:36.000Z"), "id": 669214043694043138, "text": "Just paid a $260 ransom to get my car back after it was held hostage for 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": 1537520970, "name": "Occasional Idiot", "screen_name": "victakeall", "lang": "en", "location": "The Mitten", "create_at": date("2013-06-21"), "description": "Draw your own conclusions", "followers_count": 191, "friends_count": 177, "statues_count": 1245 }, "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("2015-11-24T10:00:36.000Z"), "id": 669214043727613953, "text": "Hour 2 @jwlowery29 on his Iron Bowl memories & his thoughts on the match-up w/ AU at 1:05; Locker room report 1:30 & @TiderInsider at 1:45", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 144299173, 70231500 }}, "user": { "id": 1332853562, "name": "Drew DeArmond", "screen_name": "DrewD977ESPN", "lang": "en", "location": "Huntsville, Alabama", "create_at": date("2013-04-06"), "description": "Host of Talkin' Ball on Huntsville's @977ESPN Monday-Friday from 12-2; Staff writer/ Contributor for http://Tiderinsider.com & Duke Blue Devil since 1986", "followers_count": 3134, "friends_count": 3329, "statues_count": 36234 }, "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("2015-11-24T10:00:36.000Z"), "id": 669214043849236480, "text": "@TheJimMoran happy birthday bro", "in_reply_to_status": -1, "in_reply_to_user": 558937406, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 558937406 }}, "user": { "id": 396337248, "name": "Thurgood Jenkins", "screen_name": "JBIRD0003", "lang": "en", "location": " Louisville ", "create_at": date("2011-10-22"), "description": "Undderrated is an Understatement •502 come up• WKU19 Ig: king.jbird #DoItForState RIP Aunt DeeDee", "followers_count": 1431, "friends_count": 1512, "statues_count": 21687 }, "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("2015-11-24T10:00:36.000Z"), "id": 669214043878526976, "text": "@catbedamned Where is option C?!?", "in_reply_to_status": 669210921911328768, "in_reply_to_user": 2248270194, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2248270194 }}, "user": { "id": 26874014, "name": "Rocker™", "screen_name": "tofuttiqt", "lang": "en", "location": "Albuquerque, NM", "create_at": date("2009-03-26"), "description": "aimlessly exploring the world. vse. nm.", "followers_count": 367, "friends_count": 285, "statues_count": 12921 }, "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("2015-11-24T10:00:36.000Z"), "id": 669214043945627648, "text": "\"I'm alot friendlyer when I'm fucked up\" @NikoRojas97", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 381793507 }}, "user": { "id": 104323126, "name": "Queen B✨", "screen_name": "brookelucas5683", "lang": "en", "location": "tampa bæ", "create_at": date("2010-01-12"), "description": "bongs, thongs, and sexual songs❤️", "followers_count": 406, "friends_count": 399, "statues_count": 8615 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westchase, FL", "id": "44c12398792e903e", "name": "Westchase", "place_type": "city", "bounding_box": rectangle("-82.648577,28.038662 -82.578263,28.08544") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1276062, "cityName": "Westchase" } }
+{ "create_at": datetime("2015-11-24T10:00:36.000Z"), "id": 669214043987574784, "text": "not for thanksgiving << ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1613167566, "name": "ƙɑყᏦℐℳ^_^.", "screen_name": "kaayPrice_", "lang": "en", "location": "w/@johnnaeee_❣.", "create_at": date("2013-07-22"), "description": "thatsmysis❤️: @kieeeraaa & @__quel3 too✨..", "followers_count": 1547, "friends_count": 1343, "statues_count": 19223 }, "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("2015-11-24T10:00:36.000Z"), "id": 669214044042186752, "text": "Bob Golic and Coach David Blatt at townhallohc's 3rd Annual #FeedTheNeed #thFeedTheNeed @ Townhall https://t.co/hVqtYFQ2cv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.7041092,41.4854012"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FeedTheNeed", "thFeedTheNeed" }}, "user": { "id": 13610952, "name": "Emanuel Wallace", "screen_name": "MannyWallace", "lang": "en", "location": "ÜT: 41.453387,-81.603972", "create_at": date("2008-02-17"), "description": "Photographer for @Cleveland_Scene and @EvaNoslen. Contributing writer/reporter at @RapReviews, @PopMatters, @RESPECTMAG, etc. IG: MannyWallace", "followers_count": 1690, "friends_count": 631, "statues_count": 169017 }, "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("2015-11-24T10:00:36.000Z"), "id": 669214044620853248, "text": "This #ProjectMgmt #job might be a great fit for you: Project Manager - Infrastructure - https://t.co/ZVojELkXJh #Philadelphia, PA #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.163789,39.952335"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ProjectMgmt", "job", "Philadelphia", "Hiring" }}, "user": { "id": 22393668, "name": "TMJ-PHL IT PM Jobs", "screen_name": "tmj_phl_itpm", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2009-03-01"), "description": "Follow this account for geo-targeted IT-PM/BA job tweets in Philadelphia, PA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 247, "friends_count": 201, "statues_count": 46 }, "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-24T10:00:36.000Z"), "id": 669214045069709312, "text": "If I had known I was going to be going through this much shit so soon after, I probably would not have bought that Wizard World Evans VIP...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 74301830, "name": "Chelsea St. Juniors", "screen_name": "IfIWereMagneto", "lang": "en", "location": "New Orleans, LA", "create_at": date("2009-09-14"), "description": "I love: Carnival & Comics- esp. Avengers & X-Men (MARVEL BABY!). MCU = love, Comic Cons = life. I'm a random dork in various fandoms, but I'm a FUN, random dork", "followers_count": 1767, "friends_count": 724, "statues_count": 192284 }, "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("2015-11-24T10:00:36.000Z"), "id": 669214045220671488, "text": "Maurices #Retail #Job: Assistant Mgr FT (#Noblesville, IN) https://t.co/C2mYDQ44RZ #LoveFashionLetsTalk #Jobs https://t.co/tuMzGC6Mpi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.0085955,40.0455917"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "Job", "Noblesville", "LoveFashionLetsTalk", "Jobs" }}, "user": { "id": 2259229658, "name": "maurices careers", "screen_name": "mauricescareers", "lang": "en", "location": "U.S.", "create_at": date("2013-12-23"), "description": "Love Fashion? Join our team! @maurices is a world-class specialty retailer that caters to young-at-heart fashionistas. Check here for #job opportunities.", "followers_count": 343, "friends_count": 26, "statues_count": 2339 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Noblesville, IN", "id": "013e3bc05a18abdc", "name": "Noblesville", "place_type": "city", "bounding_box": rectangle("-86.090354,39.993436 -85.932944,40.10716") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18057, "countyName": "Hamilton", "cityID": 1854180, "cityName": "Noblesville" } }
+{ "create_at": datetime("2015-11-24T10:00:36.000Z"), "id": 669214045568933888, "text": "I think that you're my favorite babyyyyy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1152619838, "name": "Halie", "screen_name": "HalieManzo", "lang": "en", "location": "Columbus, OH", "create_at": date("2013-02-05"), "description": "Meredith Grey in training - OCU #24", "followers_count": 1074, "friends_count": 506, "statues_count": 5937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Circleville, OH", "id": "bc04497c33fee903", "name": "Circleville", "place_type": "city", "bounding_box": rectangle("-82.961991,39.574744 -82.903716,39.633876") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39129, "countyName": "Pickaway", "cityID": 3915070, "cityName": "Circleville" } }
+{ "create_at": datetime("2015-11-25T10:00:00.000Z"), "id": 669576280589864960, "text": "current weather in Springfield: cloudy, 55°F\n67% humidity, wind 22mph, visibility 10mi, pressure 30.1in falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.29156,37.20897"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 159575846, "name": "Springfield Weather", "screen_name": "_SpringfieldMO", "lang": "en", "location": "Springfield, MO", "create_at": date("2010-06-25"), "description": "Weather updates, forecast, warnings and information for Springfield, MO. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 251, "friends_count": 8, "statues_count": 21582 }, "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("2015-11-25T10:00:00.000Z"), "id": 669576280627539968, "text": "I hate not being able to nap because you have too much 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": 141990427, "name": "Sarah Parker", "screen_name": "sarahelaiine", "lang": "en", "location": "Wichita Falls, Texas ☀", "create_at": date("2010-05-09"), "description": "null", "followers_count": 733, "friends_count": 494, "statues_count": 11219 }, "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("2015-11-25T10:00:00.000Z"), "id": 669576280795447296, "text": "Universal is always 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": 2292811281, "name": "Nicole", "screen_name": "ATLs_gotta_give", "lang": "en", "location": "null", "create_at": date("2014-01-19"), "description": "music", "followers_count": 317, "friends_count": 194, "statues_count": 6704 }, "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("2015-11-25T10:00:00.000Z"), "id": 669576280917057536, "text": "Giving thanks to all those who have helped us give back to our community. #helpushelpothers #happyholidays https://t.co/BkantJOR31", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "helpushelpothers", "happyholidays" }}, "user": { "id": 3044974799, "name": "BenFranklinSoutheast", "screen_name": "BenFranklinSE", "lang": "en", "location": "3155 Willams Road Columbus, GA", "create_at": date("2015-02-19"), "description": "Ben Franklin Plumbing of Columbus and Auburn is a full service plumbing company offering commercial and residential services 24/7. \n(706)507-5304", "followers_count": 47, "friends_count": 205, "statues_count": 328 }, "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-25T10:00:00.000Z"), "id": 669576281302810624, "text": "@CooleySmoothie about to get in the shower", "in_reply_to_status": 669576202793824257, "in_reply_to_user": 29058565, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 29058565 }}, "user": { "id": 461569075, "name": "The 1st Negrokage", "screen_name": "CallMeMLK", "lang": "en", "location": "Wichita, KS", "create_at": date("2012-01-11"), "description": "hypeboyz", "followers_count": 1064, "friends_count": 1104, "statues_count": 15603 }, "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("2015-11-25T10:00:00.000Z"), "id": 669576281374199808, "text": "@jerrythornton1 congrats to your boy!", "in_reply_to_status": 669576012364140544, "in_reply_to_user": 133478732, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 133478732 }}, "user": { "id": 238206616, "name": "Frank", "screen_name": "Effing_Frank", "lang": "en", "location": "Boston area", "create_at": date("2011-01-14"), "description": "Husband/Dad, working stiff, runner, lifter.", "followers_count": 85, "friends_count": 303, "statues_count": 2830 }, "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("2015-11-25T10:00:00.000Z"), "id": 669576281659428865, "text": "I'm in such a good mood", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 173864201, "name": "lauryn", "screen_name": "laurynrandolph_", "lang": "en", "location": "indiana", "create_at": date("2010-08-02"), "description": "null", "followers_count": 593, "friends_count": 973, "statues_count": 11380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Terre Haute, IN", "id": "8a61588aff8b0577", "name": "Terre Haute", "place_type": "city", "bounding_box": rectangle("-87.460551,39.394564 -87.303557,39.520714") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18167, "countyName": "Vigo", "cityID": 1875428, "cityName": "Terre Haute" } }
+{ "create_at": datetime("2015-11-25T10:00:00.000Z"), "id": 669576282015842304, "text": "Writing lyrics down at my job cause I finally have some time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 172672932, "name": "Saints 4-6", "screen_name": "AyeronAlvarez", "lang": "en", "location": "null", "create_at": date("2010-07-30"), "description": "Oh well...", "followers_count": 245, "friends_count": 135, "statues_count": 13846 }, "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-25T10:00:00.000Z"), "id": 669576282162733056, "text": "it was hot as a brick", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2942445417, "name": "zuri pyles", "screen_name": "qveenzuri", "lang": "en", "location": "the throne", "create_at": date("2014-12-26"), "description": "im a good noodle.", "followers_count": 297, "friends_count": 282, "statues_count": 3586 }, "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("2015-11-25T10:00:00.000Z"), "id": 669576282196148225, "text": "Only 25 more weeks till 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": 77128587, "name": "Thomas Johnson", "screen_name": "tjohn1234", "lang": "en", "location": "Highlands Ranch, CO", "create_at": date("2009-09-24"), "description": "MVHS 2017 | Boys Varsity Volleyball MVHS 2015 | 303 Volleyball Club Boys 18-2| Junior Olympics Mens Volleyball 2010, 2011, 2012, 2013, 2014", "followers_count": 200, "friends_count": 956, "statues_count": 295 }, "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("2015-11-25T10:00:00.000Z"), "id": 669576282464780288, "text": "Why everyone order sound better than mine ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 347064391, "name": "Enoch Affum Jr.", "screen_name": "Kofi_Jr", "lang": "en", "location": "null", "create_at": date("2011-08-01"), "description": "Living life like theres no tomorrow...", "followers_count": 510, "friends_count": 502, "statues_count": 18440 }, "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("2015-11-25T10:00:00.000Z"), "id": 669576282674348032, "text": "Look at they name, think about them before deleting them", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 512968974, "name": "Treshawn Lambert", "screen_name": "Trey_Lamb", "lang": "en", "location": "James Madison University", "create_at": date("2012-03-03"), "description": "JMU ig:trey_lamb Rest In Paradise Grandma,Grandpa,and Uncle. I do this for you.", "followers_count": 1713, "friends_count": 1487, "statues_count": 64707 }, "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("2015-11-25T10:00:00.000Z"), "id": 669576282708058112, "text": "@TNCTrumpNews Trump is a Nazi and a racist", "in_reply_to_status": 669482203726917632, "in_reply_to_user": 3645576615, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3645576615 }}, "user": { "id": 4252815382, "name": "Mike Sal", "screen_name": "justandfree777", "lang": "en", "location": "null", "create_at": date("2015-11-22"), "description": "null", "followers_count": 0, "friends_count": 5, "statues_count": 28 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eatontown, NJ", "id": "76b66bffefeceaeb", "name": "Eatontown", "place_type": "city", "bounding_box": rectangle("-74.080628,40.270674 -74.027624,40.31435") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3419840, "cityName": "Eatontown" } }
+{ "create_at": datetime("2015-11-25T10:00:00.000Z"), "id": 669576282745790464, "text": "I've never related more to anything in my life https://t.co/7FIoWB4QZD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1172042040, "name": "Jacqueline Bosco", "screen_name": "jacquelinebosco", "lang": "en", "location": "Cleveland, OH", "create_at": date("2013-02-12"), "description": "be kind", "followers_count": 476, "friends_count": 564, "statues_count": 9341 }, "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("2015-11-25T10:00:00.000Z"), "id": 669576283337170944, "text": "@donnettrose happy birthday Donnett!!������", "in_reply_to_status": -1, "in_reply_to_user": 456028653, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 456028653 }}, "user": { "id": 470848979, "name": "Robby Robinson", "screen_name": "RobbyRobinson12", "lang": "en", "location": "West Seneca, NY ", "create_at": date("2012-01-21"), "description": "Bishop Timon '14 | St. Bonaventure '18 #BillsMafia #OneBuffalo", "followers_count": 850, "friends_count": 445, "statues_count": 17469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Seneca, NY", "id": "8e7819a4c1852d78", "name": "West Seneca", "place_type": "city", "bounding_box": rectangle("-78.801819,42.803843 -78.696854,42.874972") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3680907, "cityName": "West Seneca" } }
+{ "create_at": datetime("2015-11-25T10:00:00.000Z"), "id": 669576283639148544, "text": "Being patient", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2729548071, "name": "⚜Nineaux", "screen_name": "LaRwaNineaux", "lang": "en", "location": " Acadiana ✈ Screwston ", "create_at": date("2014-07-30"), "description": "Booking/Beats/Features: jawuan.skinner@yahoo.com", "followers_count": 590, "friends_count": 644, "statues_count": 47466 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:00.000Z"), "id": 669576284192686085, "text": "is it normal to be crying while watching this https://t.co/iDg6gahmxM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2940300245, "name": "Peyton Reed", "screen_name": "peytonnreed18", "lang": "en", "location": "probably smilin", "create_at": date("2014-12-24"), "description": "Ankeny Centennial", "followers_count": 578, "friends_count": 496, "statues_count": 4584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ankeny, IA", "id": "265f5cd93d66b1a1", "name": "Ankeny", "place_type": "city", "bounding_box": rectangle("-93.677966,41.688221 -93.545095,41.777677") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1902305, "cityName": "Ankeny" } }
+{ "create_at": datetime("2015-11-25T10:00:00.000Z"), "id": 669576284310212608, "text": "Bout to kiss and shit lmfao https://t.co/fD1E4MDu3s", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2969349591, "name": "Isaiah Castanon", "screen_name": "icastanon7", "lang": "en", "location": "null", "create_at": date("2015-01-09"), "description": "null", "followers_count": 156, "friends_count": 136, "statues_count": 3759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Haledon, NJ", "id": "8748bd00732d9e89", "name": "Haledon", "place_type": "city", "bounding_box": rectangle("-74.203023,40.923325 -74.174674,40.946837") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34031, "countyName": "Passaic", "cityID": 3429070, "cityName": "Haledon" } }
+{ "create_at": datetime("2015-11-25T10:00:01.000Z"), "id": 669576284587089920, "text": "This year people don't send me no generic happy thanksgiving massive text, if my full name isn't in the message I don't want 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": 42499453, "name": "Linwood Davis", "screen_name": "LnwdDvs", "lang": "en", "location": "Baltimore, MD", "create_at": date("2009-05-25"), "description": "Whether you think you can or think you can't, your're right - Henry Ford", "followers_count": 348, "friends_count": 496, "statues_count": 14058 }, "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-25T10:00:01.000Z"), "id": 669576284603678720, "text": "Me the other night @Damani_Brandon @Saavaanaahveaau https://t.co/n6hDJzHk9n", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 371072556, 196349818 }}, "user": { "id": 355265402, "name": "Nat٠", "screen_name": "NAATty_boh", "lang": "en", "location": "Baltimore, MD", "create_at": date("2011-08-14"), "description": "18 || FPU'19 || #BlackLivesMatter #YesAllWomen", "followers_count": 338, "friends_count": 291, "statues_count": 14023 }, "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("2015-11-25T10:00:01.000Z"), "id": 669576284704518144, "text": "@Briannnn_Godwin @FanJam son I'm dying right now ��", "in_reply_to_status": 669576022795411456, "in_reply_to_user": 604486624, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 604486624, 2596780296 }}, "user": { "id": 2765988623, "name": "Drew Wolsky", "screen_name": "RichHomieWolsky", "lang": "en", "location": "Maryland/NY", "create_at": date("2014-09-09"), "description": "7'0 Center 2016'", "followers_count": 700, "friends_count": 303, "statues_count": 13259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crofton, MD", "id": "63e2540eaa633704", "name": "Crofton", "place_type": "city", "bounding_box": rectangle("-76.707926,38.980007 -76.63862,39.029866") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24003, "countyName": "Anne Arundel", "cityID": 2420875, "cityName": "Crofton" } }
+{ "create_at": datetime("2015-11-25T10:00:01.000Z"), "id": 669576284775833600, "text": "#wcw to the most beautiful girl in the world! Im so lucky to have you and… https://t.co/Eh1VHlPdkq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.13825999,30.3201077"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wcw" }}, "user": { "id": 1398336055, "name": "Patrick Jones", "screen_name": "patty_j28", "lang": "en", "location": "null", "create_at": date("2013-05-02"), "description": "Parkview Football 18 Lilburn,GA Go Dawgs!!! Instagram-patty_j28 •Kayla has my heart ❤️•", "followers_count": 193, "friends_count": 212, "statues_count": 7042 }, "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": 12131, "countyName": "Walton" } }
+{ "create_at": datetime("2015-11-25T10:00:01.000Z"), "id": 669576284813529088, "text": "11/25@13:00 - Temp 52.8F, WC 52.6F. Wind 3.3mph ESE, Gust 8.0mph. Bar 30.738in, Falling slowly. Rain 0.00in. Hum 51%. UV 2.0. SolarRad 448.", "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": 45069 }, "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("2015-11-25T10:00:01.000Z"), "id": 669576284972847104, "text": "The fact that my parents start tripping and then they give me money ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1167151603, "name": "Juicy.joc", "screen_name": "JalapaJocelyn", "lang": "en", "location": "Fort Worth", "create_at": date("2013-02-10"), "description": "For a rich life you have to take risks ☕️Snapchat: Snooki_joc", "followers_count": 387, "friends_count": 333, "statues_count": 4418 }, "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("2015-11-25T10:00:01.000Z"), "id": 669576285237170177, "text": "@ScreenRotation @Jenn_Cupcakee good to know because I was about to restart my internet earlier. I never lag so I was confused", "in_reply_to_status": 669575735544299520, "in_reply_to_user": 2492051528, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2492051528, 76432326 }}, "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 and gave birth to @hsmitty3", "followers_count": 8421, "friends_count": 828, "statues_count": 296482 }, "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("2015-11-25T10:00:01.000Z"), "id": 669576285467901953, "text": "Wind 3.6 mph SSW. Barometer 30.44 in, Falling slowly. Temperature 57.2 °F. Rain today 0.00 in. Humidity 34%", "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": 80, "friends_count": 17, "statues_count": 300662 }, "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("2015-11-25T10:00:01.000Z"), "id": 669576285572608000, "text": "https://t.co/UVq4J2J0bR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 28812886, "name": "cyd", "screen_name": "cydbeacham", "lang": "en", "location": "NJ", "create_at": date("2009-04-04"), "description": "Happiness is a choice!", "followers_count": 414, "friends_count": 1080, "statues_count": 6723 }, "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("2015-11-25T10:00:01.000Z"), "id": 669576285799116800, "text": "Day 3 and I feel like ehhh. A little better. Still pumping myself with meds tho.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.0482692,33.9270103"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 429648492, "name": "HesABurnout", "screen_name": "ohthatkidmichel", "lang": "en", "location": "ImInYourHeadLikeAnEcho", "create_at": date("2011-12-05"), "description": "Bad Bad Beans.", "followers_count": 263, "friends_count": 997, "statues_count": 1388 }, "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("2015-11-25T10:00:01.000Z"), "id": 669576285870538752, "text": "Don't no real female wants to deal with a bomb ass nigga ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 296349767, "name": "youngfat_andfly✈️", "screen_name": "lilfat1010", "lang": "en", "location": "East point 1010", "create_at": date("2011-05-10"), "description": "null", "followers_count": 1003, "friends_count": 896, "statues_count": 16853 }, "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("2015-11-25T10:00:01.000Z"), "id": 669576286269022209, "text": "I 100% sole heartily believe I could rock a moustache", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2980002339, "name": "Virtual Pancake", "screen_name": "HunterPurner", "lang": "en", "location": "null", "create_at": date("2015-01-15"), "description": "Aspiring to make the greatest Geeb Station, Coby Lax '17", "followers_count": 243, "friends_count": 398, "statues_count": 3581 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glens Falls, NY", "id": "011edd780200b886", "name": "Glens Falls", "place_type": "city", "bounding_box": rectangle("-73.726867,43.287378 -73.594942,43.379646") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36113, "countyName": "Warren", "cityID": 3629333, "cityName": "Glens Falls" } }
+{ "create_at": datetime("2015-11-25T10:00:01.000Z"), "id": 669576286466019329, "text": "@dvirlar you get me", "in_reply_to_status": 669573613142568960, "in_reply_to_user": 1236979856, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1236979856 }}, "user": { "id": 279222095, "name": "holland", "screen_name": "hollyandoates", "lang": "en", "location": "Boulder, CO", "create_at": date("2011-04-08"), "description": "proud paper cut survivor", "followers_count": 351, "friends_count": 203, "statues_count": 2887 }, "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("2015-11-25T10:00:01.000Z"), "id": 669576286948433921, "text": "Need a Nathan and Haley relationship", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 521899835, "name": "Batgirl", "screen_name": "norris_morgan", "lang": "en", "location": "null", "create_at": date("2012-03-11"), "description": "Gotham Guardian", "followers_count": 264, "friends_count": 245, "statues_count": 8317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Derry, NH", "id": "01fa7dbf73290a3b", "name": "Derry", "place_type": "city", "bounding_box": rectangle("-71.346646,42.840671 -71.206356,42.9418") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33015, "countyName": "Rockingham", "cityID": 3317860, "cityName": "Derry" } }
+{ "create_at": datetime("2015-11-25T10:00:01.000Z"), "id": 669576287321657344, "text": "Good morning, Koloa, HI. #UnitedStates https://t.co/A1iAh3kt4i https://t.co/uWE3BMLAKk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-159.469162,21.9066658"), "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "UnitedStates" }}, "user": { "id": 17111948, "name": "BigZ", "screen_name": "ioDracy", "lang": "zh-cn", "location": "HangZhou", "create_at": date("2008-11-02"), "description": "@Hangzhou China,Android user,Google fan,Geeker,Blogger", "followers_count": 210, "friends_count": 223, "statues_count": 3777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Omao, HI", "id": "f1d76180ce00246a", "name": "Omao", "place_type": "city", "bounding_box": rectangle("-159.492627,21.903528 -159.466564,21.93723") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15007, "countyName": "Kauai", "cityID": 1557800, "cityName": "Omao" } }
+{ "create_at": datetime("2015-11-25T10:00:01.000Z"), "id": 669576287766360064, "text": "@AshGraves I've been here at the gym since like 9am lol", "in_reply_to_status": 669573880051314688, "in_reply_to_user": 44005529, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 44005529 }}, "user": { "id": 600500239, "name": "Brian Rey", "screen_name": "brey330", "lang": "en", "location": "MLB Bound' ⚾️", "create_at": date("2012-06-05"), "description": "UNF Signee⚾️ JM❤️ Baseball is my life, God is who I live through", "followers_count": 1195, "friends_count": 694, "statues_count": 19651 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange City, FL", "id": "8303573242b7cf82", "name": "Orange City", "place_type": "city", "bounding_box": rectangle("-81.315553,28.897085 -81.26003,28.963285") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1251825, "cityName": "Orange City" } }
+{ "create_at": datetime("2015-11-25T10:00:01.000Z"), "id": 669576287871221761, "text": "Welcome to the club! We happy to have you guys ⚽️ @louismason11 @bmendelson96 @hbswampy #MikesFC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MikesFC" }}, "user_mentions": {{ 586558878, 2389524356, 391003721 }}, "user": { "id": 1214449226, "name": "Paulo de Souza", "screen_name": "PauloDeSouza97", "lang": "en", "location": "null", "create_at": date("2013-02-23"), "description": "SHU Soccer '19 ⚽️ Astoria, New York ➡️ Greensburg, PA", "followers_count": 331, "friends_count": 262, "statues_count": 14149 }, "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("2015-11-25T10:00:01.000Z"), "id": 669576287946711041, "text": "0h 45m 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": 365, "friends_count": 891, "statues_count": 2041 }, "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("2015-11-25T10:00:01.000Z"), "id": 669576288047362049, "text": "@cheymula ����������", "in_reply_to_status": 669576239330549760, "in_reply_to_user": 227554659, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 227554659 }}, "user": { "id": 332662443, "name": "Sly", "screen_name": "ToddBandana", "lang": "en", "location": "Ave Maria", "create_at": date("2011-07-09"), "description": "Will Bandana ever find bae? Find out on the next episode of DragonBall Z! Creator and founder of BL Productions #AMFG", "followers_count": 1689, "friends_count": 1104, "statues_count": 91817 }, "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("2015-11-25T10:00:01.000Z"), "id": 669576288382918656, "text": "Seahawks over Steelers. This is another important game, but more critical for Seattle. I like the Legion of Boom 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": 49312130, "name": "Jared Stevens", "screen_name": "Trademark629", "lang": "en", "location": "New Haven, CT, USA", "create_at": date("2009-06-21"), "description": "Sports fanatic, independent wrestling ring announcer/interviewer, wannabe know-it-all pursuing the American Dream.", "followers_count": 1017, "friends_count": 424, "statues_count": 77691 }, "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("2015-11-25T10:00:01.000Z"), "id": 669576288592650240, "text": "I hate that the interstate is behind my house because I can always hear the ambulances drive by..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2770285144, "name": "Vanessa ❀", "screen_name": "_vanessafulton", "lang": "en", "location": "757", "create_at": date("2014-09-14"), "description": "David ❤️", "followers_count": 632, "friends_count": 447, "statues_count": 24711 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576288668131328, "text": "Congratulations @KDSZN! #ArcherNation https://t.co/qkA3XMVMsu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ArcherNation" }}, "user_mentions": {{ 270193067 }}, "user": { "id": 2186844131, "name": "ArcherTigerWrestling", "screen_name": "Archer_Tiger", "lang": "en", "location": "Lawrenceville, GA", "create_at": date("2013-11-18"), "description": "2013, 2014, & 2015 Georgia AAAAAA Traditional and Dual Champions. Ranked in the top 10 of all major High School Team rankings", "followers_count": 444, "friends_count": 255, "statues_count": 946 }, "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.17314,33.884965") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1377652, "cityName": "Tucker" } }
+{ "create_at": datetime("2015-11-25T10:00:02.000Z"), "id": 669576288680738820, "text": "I've been here for 10 minutes and I already want to leave", "in_reply_to_status": 669575386313920512, "in_reply_to_user": 604949409, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 604949409, "name": "Mark Buol", "screen_name": "MarkBuol", "lang": "en", "location": "Chicago", "create_at": date("2012-06-10"), "description": "Village 777 | Markbmgmt@gmail.com", "followers_count": 1756, "friends_count": 299, "statues_count": 4956 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576288789618688, "text": "Wind 0.2 mph SSE. Barometer 29.890 in, Rising slowly. 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("-94.98388889,47.66361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 67603939, "name": "Bemidji Weather", "screen_name": "BemidjiWx", "lang": "en", "location": "Bemidji, MN, USA", "create_at": date("2009-08-21"), "description": "Local Bemidji Weather", "followers_count": 274, "friends_count": 134, "statues_count": 94651 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minnesota, USA", "id": "9807c5c5f7a2c6ce", "name": "Minnesota", "place_type": "admin", "bounding_box": rectangle("-97.239256,43.499362 -89.483385,49.384359") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27007, "countyName": "Beltrami" } }
+{ "create_at": datetime("2015-11-25T10:00:02.000Z"), "id": 669576288852688897, "text": "@sky_2x @highlightertee_ Awh why not ?!", "in_reply_to_status": 669575101801758720, "in_reply_to_user": 326495753, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 326495753, 467876734 }}, "user": { "id": 2723441830, "name": "its MY day Bitch", "screen_name": "HerMaineJuice", "lang": "en", "location": "charleston✨orangeburg ", "create_at": date("2014-07-24"), "description": "sc : lojasz | claflin18 #TTC-Plug", "followers_count": 495, "friends_count": 436, "statues_count": 3962 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576288949026816, "text": "1 more day and I can sleep in tomorrow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 256063611, "name": "Gabriella Garcia", "screen_name": "Gabriella_Gee", "lang": "en", "location": "John 4:8", "create_at": date("2011-02-22"), "description": "18, Artist & Hooper #23..", "followers_count": 2833, "friends_count": 997, "statues_count": 42199 }, "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-25T10:00:02.000Z"), "id": 669576288953200640, "text": "11/25 #YR was good, i gotta say it gets interesting everyday! ❤ s/o to @YRInsider @CBS ��!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "YR" }}, "user_mentions": {{ 112605001, 97739866 }}, "user": { "id": 1959579655, "name": "♡☁ 19.7k ♡☁", "screen_name": "itscrewsworldTV", "lang": "en", "location": "Houston, TX", "create_at": date("2013-10-13"), "description": "17 | recording artist | song-writer | actress | promoter | model | booking-business inquires contact: crewloveeemusic2@gmail.com | #TheCrewTeam | #RIPRikoV ♡♕.", "followers_count": 19754, "friends_count": 8808, "statues_count": 110302 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576288953352192, "text": "Wind 0 mph --. Barometer 30.34 in, Falling slowly. Temperature 60.4 °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": 14, "friends_count": 3, "statues_count": 19144 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576289016246272, "text": "@realDonaldTrump just want to express my support and appreciate your #honesty that is something that the #WhiteHouse #lacks #Trump2016", "in_reply_to_status": -1, "in_reply_to_user": 25073877, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "honesty", "WhiteHouse", "lacks", "Trump2016" }}, "user_mentions": {{ 25073877 }}, "user": { "id": 1860866526, "name": "christina fluffers", "screen_name": "floppyfluff", "lang": "en", "location": "null", "create_at": date("2013-09-13"), "description": "Just me and my little buddy. My family and good friends, who I can trust to have in this lonely little world. Life is what you make it, so make it count.", "followers_count": 301, "friends_count": 1046, "statues_count": 1686 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gloucester, MA", "id": "2907085062d6147c", "name": "Gloucester", "place_type": "city", "bounding_box": rectangle("-70.740864,42.57228 -70.617898,42.689631") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25009, "countyName": "Essex", "cityID": 2526150, "cityName": "Gloucester" } }
+{ "create_at": datetime("2015-11-25T10:00:02.000Z"), "id": 669576289037254657, "text": "Ya https://t.co/EvSqsifsqQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 318081778, "name": "Limaaaaa ✨", "screen_name": "AllHailAlima_", "lang": "en", "location": "null", "create_at": date("2011-06-15"), "description": "& one thing about life is ... it always goes on ~ RIP Wags ❤️", "followers_count": 1574, "friends_count": 1591, "statues_count": 37755 }, "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-25T10:00:02.000Z"), "id": 669576289158881280, "text": "@leah_pepita BMS��", "in_reply_to_status": 669568141224255488, "in_reply_to_user": 282797665, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 282797665 }}, "user": { "id": 313525129, "name": "Hannah King", "screen_name": "hannahqueen_", "lang": "en", "location": "Charleston, WV", "create_at": date("2011-06-08"), "description": "If you love yourself, it'll all work out - Wesleyan volleyball c/o 2♡18", "followers_count": 1129, "friends_count": 293, "statues_count": 32108 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576289276133376, "text": "Oh nvm�� https://t.co/0fIGruEn90", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "nl", "is_retweet": false, "user": { "id": 2737031489, "name": "Saenzki♈️", "screen_name": "john_saenzki12", "lang": "en", "location": "Corpitos⛵️", "create_at": date("2014-08-08"), "description": "the girl in my profile picture doesn't like it when you hoes Dm me @miranda__13", "followers_count": 600, "friends_count": 1021, "statues_count": 8177 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576289385336832, "text": "I wonder who designed Santa's clothes/who his stylist was", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 280743018, "name": "dzzaddy", "screen_name": "chloedeee", "lang": "en", "location": "null", "create_at": date("2011-04-11"), "description": "my eyelashes are longer than your dick.\n \ni can't taste my lips, could you do it for me?", "followers_count": 826, "friends_count": 1057, "statues_count": 28851 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576289402011649, "text": "mlrt ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "lt", "is_retweet": false, "user": { "id": 2324225094, "name": "Ricki Roy", "screen_name": "annroy_", "lang": "en", "location": "null", "create_at": date("2014-02-02"), "description": "null", "followers_count": 371, "friends_count": 334, "statues_count": 4379 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576289452363776, "text": "@sternshow @MichaelRapaport @attell @jonhein don't worry about movies jan, your face is made for radio!", "in_reply_to_status": 669197211721457664, "in_reply_to_user": 28562317, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28562317, 241204643, 6541402, 94710817 }}, "user": { "id": 179662496, "name": "Garrett", "screen_name": "GDHauck", "lang": "en", "location": "Boston MA", "create_at": date("2010-08-17"), "description": "null", "followers_count": 41, "friends_count": 14, "statues_count": 85 }, "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-25T10:00:02.000Z"), "id": 669576289473437696, "text": "Above all else, Dream Big! https://t.co/PzBbCcLE1f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 78627891, "name": "Tony Hernandez", "screen_name": "TonyHTonyH", "lang": "en", "location": "null", "create_at": date("2009-09-30"), "description": "Publisher, http://TheManGuide.com; Co-founder, Immigrant Archive Project", "followers_count": 4771, "friends_count": 3751, "statues_count": 18549 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576289599234048, "text": "Wind 8.0 mph SE. Barometer 30.179 in, Falling. Temperature 57.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": 116, "friends_count": 0, "statues_count": 109124 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576289880289280, "text": "Wind 5.0 mph SW. Barometer 30.429 in, Falling slowly. Temperature 56.9 °F. Rain today 0.00 in. Humidity 44%", "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": 38, "friends_count": 25, "statues_count": 15664 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576290085785604, "text": "Not knowing where I want to go to school next year or do with the rest of my life is really stressing 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": 360998427, "name": "LouisaGerminerio", "screen_name": "Loouissa22", "lang": "en", "location": "null", "create_at": date("2011-08-23"), "description": "not ur average cup of tea it's just a mindset", "followers_count": 604, "friends_count": 350, "statues_count": 16415 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576290186502145, "text": "Meet the #local #smallbusiness owner at Perfect Pair Shoes. #durham #bullcity #perfectpairshoestore @… https://t.co/jSp0Xot3Ej", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.90807722,36.02049206"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "local", "smallbusiness", "durham", "bullcity", "perfectpairshoestore" }}, "user": { "id": 99641709, "name": "Tara Hill", "screen_name": "thillruns", "lang": "en", "location": "Raleigh, NC", "create_at": date("2009-12-26"), "description": "Learned to run and it changed me, changed life...we can do more than we think we can. Co-founder River Run Club http://facebook.com/riverrunclub", "followers_count": 277, "friends_count": 245, "statues_count": 2047 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576291138469888, "text": "When someone tries to reach near my plate. https://t.co/AsDh6IFh83", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 412549683, "name": "ترافيس مكلوسكي", "screen_name": "mccloskeytravis", "lang": "en", "location": "Lubbock, TX", "create_at": date("2011-11-14"), "description": "Wreck 'em. Aspiring Navy Corpsman. From my cold, dead hands.", "followers_count": 301, "friends_count": 354, "statues_count": 24846 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576291192975360, "text": "Temp: 67.4°F Wind:1.9mph Pressure: 30.321hpa 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": 22, "statues_count": 52065 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576291381833728, "text": "I would @ Hussein but he's a lil bitch cuz his teams suck and he can't face the facts https://t.co/YSjrmhgMTd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 419826106, "name": "Løpê", "screen_name": "LopeIsDope15", "lang": "en", "location": "Leonia, NJ", "create_at": date("2011-11-23"), "description": "Yeah my first name is like the island on the Hudson RIP Grandma 12/2/14 NJIT Class of 2019", "followers_count": 187, "friends_count": 218, "statues_count": 12756 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edgewater, NJ", "id": "53c4c0f31bd9e72b", "name": "Edgewater", "place_type": "city", "bounding_box": rectangle("-73.994474,40.800683 -73.96229,40.848296") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3420020, "cityName": "Edgewater" } }
+{ "create_at": datetime("2015-11-25T10:00:02.000Z"), "id": 669576291448782848, "text": "10 points off", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 176218486, "name": "STEVE-O", "screen_name": "_j_a_q_u_a_n_", "lang": "en", "location": "null", "create_at": date("2010-08-08"), "description": "when in doubt, take a hit", "followers_count": 317, "friends_count": 75, "statues_count": 17938 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576291897622529, "text": "@kenzienicole407 thanks Kenzie❤️❤️", "in_reply_to_status": 669572181618130946, "in_reply_to_user": 3544210992, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3544210992 }}, "user": { "id": 2463385940, "name": "November 25", "screen_name": "makaylaaaa17", "lang": "en", "location": "null", "create_at": date("2014-04-25"), "description": "take me to the beach :)", "followers_count": 327, "friends_count": 351, "statues_count": 1696 }, "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-25T10:00:02.000Z"), "id": 669576291901947905, "text": "Can't Be Scared To Take Chances", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 332462155, "name": "MillySantana⭐️", "screen_name": "milikebennett", "lang": "en", "location": "Mobile, AL", "create_at": date("2011-07-09"), "description": "Somewhere In The Trenches .......... it ✌️✋.", "followers_count": 2513, "friends_count": 2144, "statues_count": 28260 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576291952160768, "text": "Come! https://t.co/KMI4yq0Xpa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 172068494, "name": "♌️", "screen_name": "DoItLikeKIRA", "lang": "en", "location": "life in the fab lane✨", "create_at": date("2010-07-28"), "description": "@Mocitybu❣", "followers_count": 563, "friends_count": 398, "statues_count": 54204 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:02.000Z"), "id": 669576291964743680, "text": "happy bday @jaiden_chase13 ;-) https://t.co/Y485dlwh0R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1281482568 }}, "user": { "id": 2891815352, "name": "taylor kuper", "screen_name": "taykupe", "lang": "en", "location": "Lenexa, KS", "create_at": date("2014-11-05"), "description": "null", "followers_count": 404, "friends_count": 447, "statues_count": 569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shawnee, KS", "id": "f02a00a2aa11bfe3", "name": "Shawnee", "place_type": "city", "bounding_box": rectangle("-94.876678,38.941315 -94.703717,39.060541") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20091, "countyName": "Johnson", "cityID": 2064500, "cityName": "Shawnee" } }
+{ "create_at": datetime("2015-11-25T10:00:02.000Z"), "id": 669576292480753666, "text": "My ass is small compared to that lol https://t.co/FuLdG7GSfG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1184300779, "name": "✌️times", "screen_name": "jillYe2times", "lang": "en", "location": "null", "create_at": date("2013-02-15"), "description": "be somebody who makes everybody feel like a somebody . #freegucci #patsnation", "followers_count": 1635, "friends_count": 844, "statues_count": 44879 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576292535111680, "text": "Can you recommend anyone for this #CustomerService #job? https://t.co/jLebO5TEWz #Sandy, UT #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.8597222,40.5725"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CustomerService", "job", "Sandy", "Hiring", "CareerArc" }}, "user": { "id": 71940639, "name": "TMJ-SLC CstSrv Jobs", "screen_name": "tmj_slc_cstsrv", "lang": "en", "location": "Salt Lake City, UT", "create_at": date("2009-09-05"), "description": "Follow this account for geo-targeted Customer Service job tweets in Salt Lake City, UT. Need help? Tweet us at @CareerArc!", "followers_count": 307, "friends_count": 288, "statues_count": 220 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576292782575616, "text": "Wind 1.0 mph SE. Barometer 30.328 in, Falling. Temperature 54.3 °F. Rain today 0.00 in. Humidity 29%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-106.69694444,34.80694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14656738, "name": "Andrew Madden", "screen_name": "AbqDrew", "lang": "en", "location": "Los Lunas, New Mexico", "create_at": date("2008-05-04"), "description": "31 year old atheist conservative living in Albuquerque, New Mexico. KCCO", "followers_count": 174, "friends_count": 186, "statues_count": 4861 }, "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("2015-11-25T10:00:02.000Z"), "id": 669576292812001281, "text": "Anyone down to go see it with me �������������� https://t.co/uZWIefOrbi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2984388364, "name": "Tyeanna Dumas", "screen_name": "DumasTyeanna", "lang": "en", "location": "null", "create_at": date("2015-01-18"), "description": "Oregon ➡️ Arizona Millennium Highschool", "followers_count": 339, "friends_count": 520, "statues_count": 1482 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576293017649153, "text": "Sigh of happiness! Date night with @JeffProbst https://t.co/tQ4rDselDx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 36155411 }}, "user": { "id": 16950516, "name": "Lisa Ann", "screen_name": "LisaAnnRVA", "lang": "en", "location": "Richmond, VA ya'll", "create_at": date("2008-10-24"), "description": "Mother of 2, Autism Warrior, student, social media lover & all things Wonder Woman. #RVA", "followers_count": 2434, "friends_count": 1142, "statues_count": 23249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakeside, VA", "id": "5c1e04e89b26b3fb", "name": "Lakeside", "place_type": "city", "bounding_box": rectangle("-77.500713,37.58881 -77.455218,37.633827") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51087, "countyName": "Henrico", "cityID": 5143464, "cityName": "Lakeside" } }
+{ "create_at": datetime("2015-11-25T10:00:03.000Z"), "id": 669576293437054976, "text": "Kinda lit laying in the barber chair and being parallel to the ground", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2842683736, "name": "Nic Elia", "screen_name": "wizzzElifa", "lang": "en", "location": "null", "create_at": date("2014-10-24"), "description": "#MoneyTeam #KingofEverything", "followers_count": 352, "friends_count": 311, "statues_count": 5912 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shillington, PA", "id": "94fcdac029fbe169", "name": "Shillington", "place_type": "city", "bounding_box": rectangle("-75.978589,40.28683 -75.955079,40.313646") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42011, "countyName": "Berks", "cityID": 4270248, "cityName": "Shillington" } }
+{ "create_at": datetime("2015-11-25T10:00:03.000Z"), "id": 669576293462208512, "text": "Jumpman Jumpan can I get a text or something wooh!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2607887668, "name": "Giovanni", "screen_name": "The_GOD_7", "lang": "en", "location": "New London, CT", "create_at": date("2014-06-13"), "description": "null", "followers_count": 977, "friends_count": 931, "statues_count": 16432 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576293491548160, "text": "I'm at Starbucks in Raleigh, NC https://t.co/l5ds66sMRL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.560782,35.798746"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20922527, "name": "Dom", "screen_name": "ImmaBeastWitIt", "lang": "en", "location": "North Carolina", "create_at": date("2009-02-15"), "description": "Really nice guy, like to chill and meet new people. blind guy wanting to become a producer. #teamiphone#teamscorpeo#teamfollowback#teamdarkskin #teamturnup", "followers_count": 1250, "friends_count": 1970, "statues_count": 38925 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576293541740544, "text": "Folks in Westwood cannot fucking drive ‼️‼️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 290542502, "name": "big dough〽️", "screen_name": "Josh__Cole", "lang": "en", "location": "null", "create_at": date("2011-04-30"), "description": "null", "followers_count": 276, "friends_count": 261, "statues_count": 4143 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576293558521857, "text": "@borgore in Atlanta . \"Niykee Heaton\" brewski773 @borgore REMIX #atl #atlanta #borgore #buygore… https://t.co/EVUILbsbMg", "in_reply_to_status": -1, "in_reply_to_user": 66995579, "favorite_count": 0, "coordinate": point("-84.3881,33.7489"), "retweet_count": 0, "lang": "et", "is_retweet": false, "hashtags": {{ "atl", "atlanta", "borgore", "buygore" }}, "user_mentions": {{ 66995579, 66995579 }}, "user": { "id": 569544350, "name": "Hued Productions", "screen_name": "HuedProductions", "lang": "en", "location": "Atlanta, GA", "create_at": date("2012-05-02"), "description": "Top quality music videos and creative design.", "followers_count": 279, "friends_count": 64, "statues_count": 1202 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576293755719680, "text": "��☺️ https://t.co/EwxheyUskt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 480519643, "name": "Deja' Weja", "screen_name": "Dejaaaaaaa_", "lang": "en", "location": "Detroit, MI ✈ Dallas, TX", "create_at": date("2012-02-01"), "description": "18 | FREE WAYNE ‼️ | LongLiveDarrin |", "followers_count": 2023, "friends_count": 2059, "statues_count": 77064 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576293764108288, "text": "@meganhogya @thcelis lol Megan don't be siding w C over me ����", "in_reply_to_status": 669576123324440577, "in_reply_to_user": 4102195156, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4102195156, 414363232 }}, "user": { "id": 2215106798, "name": "Geo Davis", "screen_name": "GeoDaviss", "lang": "en", "location": "C L E V E L A N D", "create_at": date("2013-11-25"), "description": "measure twice, cut once", "followers_count": 332, "friends_count": 260, "statues_count": 7438 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576293885878274, "text": "La inteligencia debe vencer a la ignorancia este 6 de Diciembre. https://t.co/mqcCyyzVT0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2891437259, "name": "JENSEN", "screen_name": "JENSENSITO37", "lang": "en", "location": "null", "create_at": date("2014-11-24"), "description": "Alacran de nacimiento y Puma de Corazón! Durango México..", "followers_count": 442, "friends_count": 1011, "statues_count": 14347 }, "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-25T10:00:03.000Z"), "id": 669576293952954368, "text": "Photographers Be Taking Pics Of Trees, Buildings, & Elevators. Take Some Beautiful Pics Of Your Hood And The Gentrification.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 173588816, "name": "Joc Tarantino", "screen_name": "BigJocATL", "lang": "en", "location": "Behind The Scenes", "create_at": date("2010-08-01"), "description": "Visionary • Curator For @CADArtShow CreativesAfterDark@gmail.com", "followers_count": 3411, "friends_count": 2345, "statues_count": 102904 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576294149980160, "text": "Another beautiful day stop #atlaspeak at #TheCavesAtSodaCanyon pouring some perfect Thanksgiving Day… https://t.co/CNXmjcbM0H", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.30186342,38.40221804"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "atlaspeak", "TheCavesAtSodaCanyon" }}, "user": { "id": 17607409, "name": "PartyPartyParty", "screen_name": "PartyPartyParty", "lang": "en", "location": "Napa, ca ", "create_at": date("2008-11-24"), "description": "It's everyone's Favorite Fabulous Wine Enthusiast. Welcome to my Dinner Party!!", "followers_count": 1401, "friends_count": 1091, "statues_count": 13900 }, "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": 6095, "countyName": "Solano" } }
+{ "create_at": datetime("2015-11-25T10:00:03.000Z"), "id": 669576294267551744, "text": "Travelling to Lynchburg or just twittering about Lynchburg? https://t.co/3PJSqtJb94 #Lynchburg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.1423,37.4138"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Lynchburg" }}, "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": 935, "friends_count": 312, "statues_count": 2436676 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576294435164160, "text": "75.3F (Feels: 75.3F) - Humidity: 66% - Wind: 9.2mph E - Gust: 13.0mph - Pressure: 1018.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": 215420 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576294493847552, "text": "I don't need a man 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": 355854427, "name": "karen lizeth.", "screen_name": "Lizethhxx", "lang": "en", "location": "Oregon, USA", "create_at": date("2011-08-15"), "description": "19 | OR | Vegetarian | I can do everything through Christ, who gives me strength.", "followers_count": 405, "friends_count": 193, "statues_count": 18246 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576294510673921, "text": "@hotboobsearch", "in_reply_to_status": 669252952805240832, "in_reply_to_user": 3704545092, "favorite_count": 0, "coordinate": point("-97.511735,25.8665926"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3704545092 }}, "user": { "id": 2885217506, "name": "Carlos Arce", "screen_name": "arcetorres16", "lang": "es", "location": "null", "create_at": date("2014-10-31"), "description": "null", "followers_count": 136, "friends_count": 1198, "statues_count": 1194 }, "place": { "country": "México", "country_code": "México", "full_name": "Matamoros, Tamaulipas", "id": "3309acacf870f6f5", "name": "Matamoros", "place_type": "city", "bounding_box": rectangle("-97.946585,25.047842 -97.148199,26.070144") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4810768, "cityName": "Brownsville" } }
+{ "create_at": datetime("2015-11-25T10:00:03.000Z"), "id": 669576294884098048, "text": "@JBissman5 https://t.co/UhmdznDALV", "in_reply_to_status": 669575084634451968, "in_reply_to_user": 2897051695, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2897051695 }}, "user": { "id": 1611546896, "name": "kristen mccurley", "screen_name": "kristenmccurley", "lang": "en", "location": "springboro, ohio", "create_at": date("2013-07-21"), "description": "its lit", "followers_count": 778, "friends_count": 994, "statues_count": 11772 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Springboro, OH", "id": "6ae9b8002ff77818", "name": "Springboro", "place_type": "city", "bounding_box": rectangle("-84.277031,39.520846 -84.187373,39.586883") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39165, "countyName": "Warren", "cityID": 3974076, "cityName": "Springboro" } }
+{ "create_at": datetime("2015-11-25T10:00:03.000Z"), "id": 669576294926065665, "text": "Date night to Mockingjay @ Xtreme Xscape https://t.co/qFlrVM2KWw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.53167707,38.22234087"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1239768264, "name": "Shelby Weigel", "screen_name": "shelby_weigel", "lang": "en", "location": "null", "create_at": date("2013-03-03"), "description": "@hunter_ob - #avoidthecar - 502", "followers_count": 195, "friends_count": 300, "statues_count": 961 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576294976253954, "text": "in times of eminent stress. $6.00 per gram, $20.00 per 1/8, $39.00 per ¼, $75.00 per ½, and $138.00 per OZ.", "in_reply_to_status": 669575872098123776, "in_reply_to_user": 2750476154, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2750476154, "name": "Nature Quest LLC", "screen_name": "Nature_Questllc", "lang": "en", "location": "Salem, Oregon", "create_at": date("2014-08-20"), "description": "Nature Quest is a medical marijuana clinic and dispensary located in Salem, Oregon.", "followers_count": 203, "friends_count": 60, "statues_count": 2189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Four Corners, OR", "id": "8271f56611245451", "name": "Four Corners", "place_type": "city", "bounding_box": rectangle("-122.990493,44.909874 -122.955303,44.962335") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41047, "countyName": "Marion", "cityID": 4126750, "cityName": "Four Corners" } }
+{ "create_at": datetime("2015-11-25T10:00:03.000Z"), "id": 669576295085379584, "text": "so the other day chris and i are in steve madden and im looking around when i hear him talking to someone on the other side of the store", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 53760086, "name": "mel", "screen_name": "mlncrs", "lang": "en", "location": "Kendall, FL", "create_at": date("2009-07-04"), "description": "null", "followers_count": 371, "friends_count": 115, "statues_count": 67101 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576295269855233, "text": "Mostly sunny this afternoon, high 73 (23 C). Low 50 (10 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15953346, "name": "Amarillo Weather", "screen_name": "AmarilloWeather", "lang": "en", "location": "Amarillo, Texas", "create_at": date("2008-08-22"), "description": "Forecasts and advisories for Amarillo, Texas", "followers_count": 3991, "friends_count": 395, "statues_count": 8588 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576295601209344, "text": "@jasminemonaee_ you childish", "in_reply_to_status": 669576227443765249, "in_reply_to_user": 2386390765, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2386390765 }}, "user": { "id": 2880079118, "name": "bri.❣", "screen_name": "bridawwgg", "lang": "en", "location": "null", "create_at": date("2014-10-27"), "description": "add me on sc:brigonedoit", "followers_count": 875, "friends_count": 440, "statues_count": 7899 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576295869624320, "text": "Coconut peanut butter is bomb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2346200719, "name": "Cameryn", "screen_name": "cameryncollie", "lang": "en", "location": "null", "create_at": date("2014-02-15"), "description": "YIKES ||| http://camcollie.vsco.co", "followers_count": 572, "friends_count": 466, "statues_count": 897 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576295940907008, "text": "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": 2268225526, "name": "Alexander™", "screen_name": "youaint_drew", "lang": "en", "location": "null", "create_at": date("2014-01-07"), "description": "Darcy is the greatest ❤️ ; @admireedarcy ✨", "followers_count": 25, "friends_count": 39, "statues_count": 1800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-25T10:00:03.000Z"), "id": 669576295941054464, "text": "Where is Lynchburg on the map? Play the game at https://t.co/3PJSqtJb94 #Lynchburg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.1423,37.4138"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Lynchburg" }}, "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": 935, "friends_count": 312, "statues_count": 2436677 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576296364658688, "text": "Wind 5.7 mph S. Barometer 30.712 in, Falling. Temperature 44.5 °F. Rain today 0.00 in. Humidity 56%", "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": 4634 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576296377266177, "text": "I will always be an Alex Avila girl. Forever and ever and ever and ever.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29870382, "name": "shelbey", "screen_name": "detroitplayaljp", "lang": "en", "location": "Kentwood, Michigan", "create_at": date("2009-04-08"), "description": "shelbey / 18 / you gotta live hardcore to be hardcore / #1Daf #AvilaGirl4L", "followers_count": 1217, "friends_count": 884, "statues_count": 102500 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576296490336256, "text": "YES DRINK SPECIALS CRAZY $2 �� DROPS ALL NIGHT EVERYONE FREE HAPPY HOUR TODAY 1116 HOLLINS ST 4:00 TO… https://t.co/2npO6xh5C4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.63577372,39.28775944"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2174686873, "name": "KSDREAMCHASER", "screen_name": "ksdreamchaser", "lang": "en", "location": "null", "create_at": date("2013-11-04"), "description": "#RIPAUDIE GRIND HARD TO SHINE DIFFERENT #RIPBALL ‼️", "followers_count": 2847, "friends_count": 2512, "statues_count": 27073 }, "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-25T10:00:03.000Z"), "id": 669576296507318276, "text": "@itsalesa Aww its ok", "in_reply_to_status": 669575940276674560, "in_reply_to_user": 2856127790, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2856127790 }}, "user": { "id": 2893196713, "name": "Mega Lucario", "screen_name": "MidnightsSkies", "lang": "en", "location": "null", "create_at": date("2014-11-07"), "description": "It's ok if you disagree with me. I can't force you to be right. Gamer,Artist,Bruh Bruh,Sky Recruit, Jinvader", "followers_count": 46, "friends_count": 16, "statues_count": 1899 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576296591151105, "text": "\"Trust God's timing, because what you want is not what you're suppose to have.\" @FBruceWilliams #SermonNotes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SermonNotes" }}, "user_mentions": {{ 1666488661 }}, "user": { "id": 53229077, "name": "Bates Memorial BptCh", "screen_name": "BatesMemorial", "lang": "en", "location": "Louisville, KY", "create_at": date("2009-07-02"), "description": "We are Changed People Changing the World!; Led by Senior Pastor, Dr. F. Bruce Williams (@FBruceWilliams). Come Worship with us!", "followers_count": 3135, "friends_count": 3007, "statues_count": 11994 }, "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-25T10:00:03.000Z"), "id": 669576296607936512, "text": "I've gotten myself better", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 703389721, "name": "❣", "screen_name": "trighasnojuice", "lang": "en", "location": "null", "create_at": date("2012-07-18"), "description": "Let me set the bar since y'all couldn't get it right", "followers_count": 2409, "friends_count": 1064, "statues_count": 45621 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aiken, SC", "id": "2fe596253ff13bb5", "name": "Aiken", "place_type": "city", "bounding_box": rectangle("-81.779032,33.475332 -81.672472,33.590153") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45003, "countyName": "Aiken", "cityID": 4500550, "cityName": "Aiken" } }
+{ "create_at": datetime("2015-11-25T10:00:03.000Z"), "id": 669576296779771904, "text": "Me G Eskimo https://t.co/NMrork7Q56", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "lt", "is_retweet": false, "user": { "id": 724508209, "name": "Laura Nestor", "screen_name": "LauraNestor4", "lang": "en", "location": "L.A.Ca / Las Vegas, Nv USA", "create_at": date("2012-07-29"), "description": "60's child, Flower power Peace Love & enjoy many music genre,retired USPS Pls don't send porn to my DM", "followers_count": 8042, "friends_count": 7910, "statues_count": 64038 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576296863678465, "text": "Redditor finds rude guy hiding behind a picture frame https://t.co/X9Cq0MYMme", "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": 17715, "friends_count": 17607, "statues_count": 65026 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576296939130880, "text": "@b_sanchez9 LOL I didn't post that ��������������", "in_reply_to_status": 669576099966283776, "in_reply_to_user": 2278141603, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2278141603 }}, "user": { "id": 755475480, "name": "daleisha jazelle", "screen_name": "daleishaaa", "lang": "en", "location": "Bay Area", "create_at": date("2012-08-13"), "description": "snapchat: daleishaaa", "followers_count": 11144, "friends_count": 1463, "statues_count": 14686 }, "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("2015-11-25T10:00:03.000Z"), "id": 669576296976941056, "text": "it's national ignore isabel 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": 2755219896, "name": "izzy", "screen_name": "ninjabel", "lang": "en", "location": "btx", "create_at": date("2014-08-22"), "description": "snapchat / isabelmarieh bhs '19", "followers_count": 359, "friends_count": 355, "statues_count": 2195 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bastrop, TX", "id": "7d4a036b2ce1798b", "name": "Bastrop", "place_type": "city", "bounding_box": rectangle("-97.369763,30.090081 -97.277272,30.136312") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48021, "countyName": "Bastrop", "cityID": 4805864, "cityName": "Bastrop" } }
+{ "create_at": datetime("2015-11-25T10:00:04.000Z"), "id": 669576297161486336, "text": "It's called \"Drown\" with Travis Scott https://t.co/nN13VoMlZX", "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": "Shaine", "screen_name": "SHAINEAVELI", "lang": "en", "location": "New Orleans, LA", "create_at": date("2011-06-10"), "description": "die", "followers_count": 1065, "friends_count": 387, "statues_count": 80875 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576297308413952, "text": "Graffiti Tracking: 1223 S PCH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.3854798,33.823836"), "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": 898, "friends_count": 2, "statues_count": 309073 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576297321013249, "text": "Fina go get my shit chopped", "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": "BOSSKJ™", "screen_name": "365Kj_", "lang": "en", "location": "CHICAGO✈️MIAMI", "create_at": date("2011-08-11"), "description": "Big E & LB World ️ ️", "followers_count": 1541, "friends_count": 901, "statues_count": 50485 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576297480220672, "text": "@ChrisWebby dude! I would empty that box on 2 for $20... can I get in on that deal? Those are my favorite fuckin shirts!", "in_reply_to_status": 669556685770395649, "in_reply_to_user": 23675921, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23675921 }}, "user": { "id": 521188029, "name": "Eli Evans", "screen_name": "EvansilE09", "lang": "en", "location": "Utah", "create_at": date("2012-03-11"), "description": "@Chriswebby #WebbysWorld #23 #801", "followers_count": 97, "friends_count": 150, "statues_count": 4389 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576298000285697, "text": "It's too early https://t.co/HZ5hxMcjYH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1152807402, "name": "temp.thunderfan", "screen_name": "tilted0ne", "lang": "en", "location": "null", "create_at": date("2013-02-05"), "description": "!", "followers_count": 122, "friends_count": 114, "statues_count": 14423 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576298000424960, "text": "Wind 0.0 mph SE. Barometer 30.747 in, Steady. Temperature 50.7F. Rain today 0.00 in. Humidity 44% | 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": 564, "friends_count": 760, "statues_count": 34112 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576298252120064, "text": "YPSILANTI TONIGHT w/ @basswhlf @oshea_313 @felix_tha_great hans_play… https://t.co/T7U8ZYItac", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.6129074,42.2413254"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3302099877, 71965796, 467995459 }}, "user": { "id": 186288747, "name": "-ChristianAlexander-", "screen_name": "ChristianHoffer", "lang": "en", "location": "Earth", "create_at": date("2010-09-02"), "description": "Making and delivering good vibrations through the power of music", "followers_count": 671, "friends_count": 621, "statues_count": 10962 }, "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.32554") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26161, "countyName": "Washtenaw", "cityID": 2689140, "cityName": "Ypsilanti" } }
+{ "create_at": datetime("2015-11-25T10:00:04.000Z"), "id": 669576298281500676, "text": "Noguchi à New York\n#Manhattan #Art #Travel #Voyages #StreetPhotography #NYC @ Zuccotti Park https://t.co/DTnc6TYfTN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.011323,40.709385"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "Manhattan", "Art", "Travel", "Voyages", "StreetPhotography", "NYC" }}, "user": { "id": 37471779, "name": "Elisa Nievas", "screen_name": "elisafn", "lang": "es", "location": "Rosario, Argentina", "create_at": date("2009-05-03"), "description": "#Viajes #Photos #Blogs #Culture #LifeStyle 2.0 #Travel #blogging #Luxury #Travel Inspirer Tweets in Spanish French & English Also in http://bit.ly/1QfKv2H", "followers_count": 1570, "friends_count": 816, "statues_count": 61621 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576298298081282, "text": "That wasn't a private message. ABORT MISSION", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2775810638, "name": "Dan C", "screen_name": "danielcury91", "lang": "en", "location": "null", "create_at": date("2014-08-27"), "description": "null", "followers_count": 52, "friends_count": 105, "statues_count": 1087 }, "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-25T10:00:04.000Z"), "id": 669576298386190336, "text": "Partly cloudy this afternoon, high 71 (22 C). Low 56 (13 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712333, "name": "LubbockWeather", "screen_name": "LubbockWeather", "lang": "en", "location": "Lubbock, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Lubbock, Texas", "followers_count": 1167, "friends_count": 93, "statues_count": 7921 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576298478485505, "text": "@Sumdian i think things are slowly getting better but there are peaks and valleys to the shit boy activity in the meantime", "in_reply_to_status": 669575454098108416, "in_reply_to_user": 62060479, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 62060479 }}, "user": { "id": 64490007, "name": "✨snowy miri✨", "screen_name": "heartcatchr", "lang": "en", "location": "seattle", "create_at": date("2009-08-10"), "description": "call me justice/miri!✨she/her♀✨fighting games/FFXIV/RPG✨i stream sometimes✨CA transplant, PNW trashgirl", "followers_count": 414, "friends_count": 236, "statues_count": 61552 }, "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-25T10:00:04.000Z"), "id": 669576298663157760, "text": "@LilMsgss @randyprine @tveitdal @Alex_Verbeek @bennydiego @co_kmaldonado4 @IanMolony\n#climatechange #climate https://t.co/34Q1jheByD", "in_reply_to_status": -1, "in_reply_to_user": 2445365048, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "climatechange", "climate" }}, "user_mentions": {{ 2445365048, 37990581, 75742264, 326665662, 30231573, 837561853, 2443083780 }}, "user": { "id": 474068520, "name": "Joe DeRosa", "screen_name": "derosajoe51", "lang": "en", "location": "USA", "create_at": date("2012-01-25"), "description": "SkyWarn spotter, midnight rambler and procrastinator. An accomplished guitarist, passionate about meteorology, astronomy, geology & photography.", "followers_count": 954, "friends_count": 483, "statues_count": 27848 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576298755297280, "text": "足に怪我したのに怪我しなかったように自転車を乗って行こう!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user": { "id": 916596266, "name": "竹下E本", "screen_name": "takenji_ebooks", "lang": "en", "location": "The Past", "create_at": date("2012-10-31"), "description": "Eh, close enough.", "followers_count": 129, "friends_count": 1, "statues_count": 95701 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "f42a863798156617", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.459696,47.491912 -122.224433,47.734145") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2015-11-25T10:00:04.000Z"), "id": 669576299011248128, "text": "Hmm I have a feeling I need to use twitter more.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 264562593, "name": "A Fox Thing", "screen_name": "tsukune101", "lang": "en", "location": "Columbus", "create_at": date("2011-03-11"), "description": "Fun fox who loves meeting and talking to new furs ^^ You can find the love of my life @RezDaBurr :3", "followers_count": 242, "friends_count": 367, "statues_count": 2003 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576299090870272, "text": "@MariaSansone @Araksya loved the photo of you 2 jump in for joy", "in_reply_to_status": -1, "in_reply_to_user": 15908473, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15908473, 46185434 }}, "user": { "id": 2514831335, "name": "Paul Baker", "screen_name": "paulhobiebaker", "lang": "en", "location": "San Clemente CA", "create_at": date("2014-04-28"), "description": "Im a college graduate and have aspergers syndrome. I enjoy hanging out with friends, loyal fan of Gdla, playing on my xbox, helping out, and an awesome person.", "followers_count": 43, "friends_count": 70, "statues_count": 7121 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576299254386688, "text": "15k who wants 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": 703574234, "name": "meme-y christmissa", "screen_name": "tolmgc", "lang": "en", "location": "null", "create_at": date("2012-07-18"), "description": "° . • . ° not to be fake deep but issa followed me&faved my pinned tweet ° . • . °", "followers_count": 15066, "friends_count": 6310, "statues_count": 11640 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576299464278021, "text": "@Rikkord_Forte Elliot ignored this court order, kept sending insulting messages to people he knew wanted no contact. Who had blocked him.", "in_reply_to_status": 669572979773169664, "in_reply_to_user": 2915348348, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2915348348 }}, "user": { "id": 814435975, "name": "Margaret Pless", "screen_name": "idlediletante", "lang": "en", "location": "New York, NY", "create_at": date("2012-09-09"), "description": "Citizen Journalist, grad student & blogger. Sarah Lawrence College Class of 2012; CCNY-CUNY, 2015 - present. Email: mpless at gm dot slc dot edu. GPG#: 658B7F9D", "followers_count": 555, "friends_count": 298, "statues_count": 13384 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576299690762240, "text": "\"I'm bringing you to the original Versailles baby.\" Lol Melo loves Cuban food like his daddy. Won't stop kicking������ https://t.co/F9ENmkbhDN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 143662724, "name": "vanessa", "screen_name": "vhness119", "lang": "en", "location": "null", "create_at": date("2010-05-13"), "description": "he loved her, of course, but better than that, he chose her, day after day. Choice: that was the thing. 9.12.14❤️", "followers_count": 454, "friends_count": 1199, "statues_count": 6332 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576299766259716, "text": "@TonyGrossi maybe getting to an all time low for this franchise? Even Fox news talks about them now!", "in_reply_to_status": 669570149444411392, "in_reply_to_user": 33424006, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 33424006 }}, "user": { "id": 21465143, "name": "Dan the Man", "screen_name": "originalrrafnad", "lang": "en", "location": "null", "create_at": date("2009-02-20"), "description": "null", "followers_count": 96, "friends_count": 729, "statues_count": 2831 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576299858567169, "text": "@AdamGeib2 all the time bro", "in_reply_to_status": 669575882827255809, "in_reply_to_user": 328212196, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 328212196 }}, "user": { "id": 336675628, "name": "Ronnie Bass", "screen_name": "MRF10_", "lang": "en", "location": "Buffalo, NY", "create_at": date("2011-07-16"), "description": "Go Bills", "followers_count": 375, "friends_count": 242, "statues_count": 31820 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576300051476480, "text": "Temp: 76.8°F | Humidity: 50% | Wind: N @ 4.5 mph | Barometer: 30.27 in | Dewpoint: 56.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": 24, "friends_count": 1, "statues_count": 163297 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576300084989952, "text": "@Joyda55Mathers where's that show at?", "in_reply_to_status": 669160810690801666, "in_reply_to_user": 231253964, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 231253964 }}, "user": { "id": 15531174, "name": "TJ", "screen_name": "tjDetroit", "lang": "en", "location": "The Clem", "create_at": date("2008-07-22"), "description": "Old school heavy metal and Detroit hip-hop. Professional music fan. The Mount Clemens Travis Bickle. Made in Detroit.", "followers_count": 760, "friends_count": 2006, "statues_count": 19819 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576300101656576, "text": "�� Who stays out here in Orange County ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 454700255, "name": "C ▵M ïNSÏDE☪MYV▵RŤX", "screen_name": "PVSSYPWR__", "lang": "en", "location": "null", "create_at": date("2012-01-04"), "description": "drakesbed", "followers_count": 807, "friends_count": 962, "statues_count": 6550 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576300139417600, "text": "@gingerbums oh wait. The mimosas are for you & husbear", "in_reply_to_status": 669573860786745346, "in_reply_to_user": 436097213, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2561848352 }}, "user": { "id": 436097213, "name": "Randy", "screen_name": "PozitiveVizion", "lang": "en", "location": "Bay Area", "create_at": date("2011-12-13"), "description": "HIV+ man, late 50s and AIDS activist with many interests, who also publishes items of interest to other gay men incl v adult items.18+, ADULTS ONLY PLEASE.", "followers_count": 607, "friends_count": 468, "statues_count": 13098 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576300156207104, "text": "what have you done for me lately?�� https://t.co/b1dNvDW1fG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27279153, "name": "Ramona Royale", "screen_name": "_WlkinParadox", "lang": "en", "location": "Behind My Camera", "create_at": date("2009-03-28"), "description": "She stays in the coolest moods. Clearly, women of the year. #AiP", "followers_count": 1427, "friends_count": 597, "statues_count": 243894 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaver Falls, PA", "id": "579173dcb66e54d5", "name": "Beaver Falls", "place_type": "city", "bounding_box": rectangle("-80.335207,40.739691 -80.309549,40.790082") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42007, "countyName": "Beaver", "cityID": 4204792, "cityName": "Beaver Falls" } }
+{ "create_at": datetime("2015-11-25T10:00:04.000Z"), "id": 669576300173107200, "text": "Tew much for me to deal with", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 135353774, "name": "Hadji Galore™", "screen_name": "HadjiTheUpsettr", "lang": "en", "location": "Around ™", "create_at": date("2010-04-20"), "description": "Progressing..|717|FreeQuez|CalU|", "followers_count": 1815, "friends_count": 1104, "statues_count": 150896 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576300185669633, "text": "Just posted a photo @ The Metropolitan Museum of Art, New York https://t.co/u3kO2sMffo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.96297384,40.77912965"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1124482093, "name": "Louise AntoineWatson", "screen_name": "Watson67Louise", "lang": "en", "location": "null", "create_at": date("2013-01-27"), "description": "Works at Manchester Museum, the one with the Dinosaur..", "followers_count": 486, "friends_count": 454, "statues_count": 1901 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576300227612673, "text": "@reIatabIe @JennBressler", "in_reply_to_status": 669565823917273089, "in_reply_to_user": 437030761, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 437030761, 94239026 }}, "user": { "id": 43705985, "name": "Sofia", "screen_name": "_soph28", "lang": "en", "location": "null", "create_at": date("2009-05-31"), "description": "life ain't chess", "followers_count": 253, "friends_count": 248, "statues_count": 36860 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576300378460160, "text": "This life aint promised thats why i show love and do for the people i care about...i wanna leave a lasting impression long after im gone.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4254535093, "name": "YungCoCoLoso", "screen_name": "YungCoCoLosoTYG", "lang": "en", "location": "The Burg/The P/The Chi", "create_at": date("2015-11-15"), "description": "Producer/Artist/Engineer ect. ect. ect. Music of all variety is my life\nIm now taking this more serious than ever its time to do what i love for the bread", "followers_count": 60, "friends_count": 177, "statues_count": 49 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576300479143936, "text": "Every day is a beautiful day when you're in heaven, Jeff. ☺ https://t.co/cNuAxRa6eP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 205331870, "name": "Bryan McLean", "screen_name": "Golfbybryan", "lang": "en", "location": "Snohomish, WA", "create_at": date("2010-10-20"), "description": "Remarkably bad golf, spirits and cigar writer. Golf Architecture nerd. Functional drunk, mostly. Tweets are my own. Follow @golflife for more.", "followers_count": 978, "friends_count": 925, "statues_count": 13733 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576300592537600, "text": "josh is beautiful https://t.co/tyU1DTxoS5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2575971834, "name": "mari's purpose / 80", "screen_name": "sighwhitesides", "lang": "en", "location": "chicago ", "create_at": date("2014-06-18"), "description": "you get prettier every time I see you -dyl", "followers_count": 1856, "friends_count": 1029, "statues_count": 42569 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576300684800001, "text": "@Chabelyb_ ���� https://t.co/BdhUFwDgD8", "in_reply_to_status": 669562338769375232, "in_reply_to_user": 278202228, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 278202228 }}, "user": { "id": 1164683862, "name": "georgina sparks", "screen_name": "fakebagel", "lang": "en", "location": "chicago/omaha/tempe/anywhere", "create_at": date("2013-02-09"), "description": "it'll all be okay; if it's not okay it's not the end", "followers_count": 318, "friends_count": 290, "statues_count": 29444 }, "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-25T10:00:04.000Z"), "id": 669576300848357376, "text": "I've been listening to One Direction 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": 62711456, "name": "Joe", "screen_name": "JoeMichaell_", "lang": "en", "location": "New York City", "create_at": date("2009-08-03"), "description": "I'm great", "followers_count": 1161, "friends_count": 333, "statues_count": 32427 }, "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-25T10:00:04.000Z"), "id": 669576300944736256, "text": "The caribbean women part true though�� https://t.co/Um7FmnL4ZZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2945721116, "name": "Leli❤️", "screen_name": "leli_onfleek", "lang": "en", "location": "null", "create_at": date("2014-12-27"), "description": "Vee❤. & Oriella❤️// Caribbean | John Ehret Volleyball❤️| August 17th♌️", "followers_count": 391, "friends_count": 324, "statues_count": 4856 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576301062127617, "text": "Temp: 47.2°F - Dew Point: 33.6° - Wind: 13.7 mph - Gust: 19.7 - Rain Today: 0.00in. - Pressure: 30.16in, - 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": 9653 }, "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("2015-11-25T10:00:04.000Z"), "id": 669576301087469569, "text": "I'm so excited to start christmas shopping for kyran ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 392137373, "name": "Kara.", "screen_name": "KaraCelena_", "lang": "en", "location": "null", "create_at": date("2011-10-16"), "description": "Lorain, Ohio", "followers_count": 451, "friends_count": 329, "statues_count": 9123 }, "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-25T10:00:04.000Z"), "id": 669576301213126656, "text": "@TheCoffeeBean the winter dream tea latte", "in_reply_to_status": 669574166626988033, "in_reply_to_user": 17967491, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17967491 }}, "user": { "id": 1463650784, "name": "yung shibe", "screen_name": "hahanuhuhh", "lang": "en", "location": "ATX", "create_at": date("2013-05-27"), "description": "raised by keebler elves", "followers_count": 248, "friends_count": 295, "statues_count": 12511 }, "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-25T10:00:05.000Z"), "id": 669576301292875776, "text": "Afternoon showers this afternoon, high 73 (23 C). Low 53 (12 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712562, "name": "ElPasoWX", "screen_name": "ElPasoWX", "lang": "en", "location": "El Paso, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for El Paso, Texas", "followers_count": 525, "friends_count": 93, "statues_count": 7979 }, "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("2015-11-25T10:00:05.000Z"), "id": 669576301330608128, "text": "AIDS N : Why HIV Rates are Soaring in Russia https://t.co/7RldUbWNoD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.2732666,25.77508716"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 191092262, "name": "Stigmabase | NORDIC", "screen_name": "pairsonnalitesN", "lang": "en", "location": "Scot | Inuit | Skandinavien", "create_at": date("2010-09-15"), "description": "Keeping up-to-date on social exclusion worldwide | NORDIC | Scotland | Norsk | Svenska | Русский || Not-for-PROFIT", "followers_count": 1483, "friends_count": 1474, "statues_count": 349555 }, "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("2015-11-25T10:00:05.000Z"), "id": 669576301347545089, "text": "My dreams and real life are starting to collide to where I don't know what's real anymore", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3025423103, "name": "khayo", "screen_name": "KidKhayo", "lang": "en", "location": "Views From The 5", "create_at": date("2015-02-08"), "description": "ΔΤΔ - ΑΚΨ - ΒΑΨ | university of cincinnati | jordanian CirocPapi", "followers_count": 340, "friends_count": 316, "statues_count": 1057 }, "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("2015-11-25T10:00:05.000Z"), "id": 669576301502701569, "text": "Four tips for Thanksgiving air travel https://t.co/shYjqnwcc9 #ROC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ROC" }}, "user": { "id": 17022537, "name": "Rochester D and C", "screen_name": "DandC", "lang": "en", "location": "Rochester, NY", "create_at": date("2008-10-28"), "description": "The best source of local news in Rochester. Our in-depth coverage includes breaking news updates, sports, culture, food, business, education and more.", "followers_count": 49960, "friends_count": 4261, "statues_count": 81800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rochester, NY", "id": "2f1fc0d72969452b", "name": "Rochester", "place_type": "city", "bounding_box": rectangle("-92.558347,43.099801 -77.533421,44.108056") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3663000, "cityName": "Rochester" } }
+{ "create_at": datetime("2015-11-25T10:00:05.000Z"), "id": 669576301611626496, "text": "@LudgerBrinkmann @Lepoulter ⚡️��⚡️��⚡️��", "in_reply_to_status": 669574153033416705, "in_reply_to_user": 455036429, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 455036429, 4099340081 }}, "user": { "id": 2963445000, "name": "ALEX MASON ✖️✖️✖️", "screen_name": "AlexMasonXXX", "lang": "en", "location": "Kansas City", "create_at": date("2015-01-07"), "description": "26 | Go-Go Dancer | Bartender | Porn Model | Vers Bottom | Athlete | Runner | Embrace Fitness |", "followers_count": 4517, "friends_count": 1014, "statues_count": 1068 }, "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("2015-11-25T10:00:05.000Z"), "id": 669576301758447616, "text": "@MiracleeNoWhip @_ShellyMac365 damn. I forgot", "in_reply_to_status": 669569195173003264, "in_reply_to_user": 497437603, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 497437603, 2449929680 }}, "user": { "id": 1563308737, "name": "✨RestEazyTookie✨", "screen_name": "_ChillBb__", "lang": "en", "location": "C h I C a g O ☔️❄️⛅️", "create_at": date("2013-07-02"), "description": "wondering what it feels like to be in love? •09|26 \n#BallingLikeTookie \n forever in my heart cuddy❤", "followers_count": 652, "friends_count": 619, "statues_count": 16668 }, "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-25T10:00:05.000Z"), "id": 669576301934682112, "text": "@starryeyedkid I want him to kill me... like saying that is the biggest compliment I can pay someone", "in_reply_to_status": 669575687032971265, "in_reply_to_user": 18108408, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18108408 }}, "user": { "id": 2442887570, "name": "stevebucky stina♥✨", "screen_name": "buckybarnesing", "lang": "en", "location": "DC/Palo Alto/stinaxx on insta", "create_at": date("2014-04-13"), "description": "20, she/her • queer • i like musicals (esp. bom (tbh i'm TRASH) + hamilton), turn amc, sebastian stan, marvel • i have no self restraint + i cry a LOT • |-/", "followers_count": 734, "friends_count": 878, "statues_count": 90185 }, "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("2015-11-25T10:00:05.000Z"), "id": 669576302438047744, "text": "In everything you do, do it with love.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2473193479, "name": "§oleil ß.", "screen_name": "soulayyy", "lang": "en", "location": "The end of Lucy", "create_at": date("2014-05-01"), "description": "Your dreams are in reach. Step out of your comfort zone. OVERCOME ALL OBSTACLES.", "followers_count": 375, "friends_count": 672, "statues_count": 3721 }, "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("2015-11-25T10:00:05.000Z"), "id": 669576302593077248, "text": "I'm probably the only person in my dorm 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": 2299091833, "name": "b", "screen_name": "_GutierrezBryan", "lang": "en", "location": "#GetBetterTeam", "create_at": date("2014-01-18"), "description": "IPromise. God & Fam. WC c/o 2019. #Shhhh", "followers_count": 147, "friends_count": 53, "statues_count": 18201 }, "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-25T10:00:05.000Z"), "id": 669576302790340608, "text": "Vámonos a casa ���� ���� #IVLP (@ Orlando International Airport (MCO) - @mco in Orlando, FL) https://t.co/dgxoiarJFS https://t.co/jM1qQi0RVo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.30840898,28.43090203"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "IVLP" }}, "user_mentions": {{ 366027659 }}, "user": { "id": 230568927, "name": "Miriam Carrillo B.", "screen_name": "Mirky_MB", "lang": "es", "location": "Mexico, City", "create_at": date("2010-12-25"), "description": "Scientist: Physicist & Museologist |Educational Curator at @UniversumMuseo |Focusing on #Astronomy #Education #ScienceCommunication #STEM |#ExchangeAlumni #IVLP", "followers_count": 901, "friends_count": 1883, "statues_count": 7881 }, "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("2015-11-25T10:00:05.000Z"), "id": 669576302953918464, "text": "Temp 52.3°F RH 41% Wind 4.0 E Gust 11.0 E SLP 30.606 in Falling slowly Rain 0.00 in Solar 396 UV 2.3 #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": 103, "friends_count": 65, "statues_count": 31327 }, "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("2015-11-25T10:00:05.000Z"), "id": 669576303000100865, "text": "just why?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2308508900, "name": "megann", "screen_name": "_megann19", "lang": "en", "location": "null", "create_at": date("2014-01-24"), "description": "6/16/15❤", "followers_count": 269, "friends_count": 257, "statues_count": 2143 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albemarle, NC", "id": "493594fab29664d6", "name": "Albemarle", "place_type": "city", "bounding_box": rectangle("-80.246242,35.327705 -80.137714,35.427345") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37167, "countyName": "Stanly", "cityID": 3700680, "cityName": "Albemarle" } }
+{ "create_at": datetime("2015-11-25T10:00:05.000Z"), "id": 669576303155134464, "text": "It's hot asf ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2888938726, "name": "nicole ❤️", "screen_name": "xxnniikkaayy__", "lang": "en", "location": "null", "create_at": date("2014-11-23"), "description": "479 | sc; x.nikk | 18 |", "followers_count": 216, "friends_count": 201, "statues_count": 5435 }, "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("2015-11-25T10:00:05.000Z"), "id": 669576303167705089, "text": "Now I 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": 2894187420, "name": "ciaaannnaaa_", "screen_name": "Ciaannnaaa_1", "lang": "en", "location": "null", "create_at": date("2014-11-08"), "description": "bhs", "followers_count": 387, "friends_count": 372, "statues_count": 2027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bishop, TX", "id": "86a8191a31584253", "name": "Bishop", "place_type": "city", "bounding_box": rectangle("-97.811344,27.572012 -97.783257,27.599363") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4808392, "cityName": "Bishop" } }
+{ "create_at": datetime("2015-11-25T10:00:05.000Z"), "id": 669576303260102657, "text": "Today. Weirdly. Is the biggest drinking day of the year.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 382851604, "name": "Dylan Scrimmy", "screen_name": "ScrimmyFS", "lang": "en", "location": "null", "create_at": date("2011-09-30"), "description": ".", "followers_count": 1125, "friends_count": 540, "statues_count": 49432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kirtland, OH", "id": "83fb00e66729dcd3", "name": "Kirtland", "place_type": "city", "bounding_box": rectangle("-81.391497,41.543257 -81.309644,41.640817") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39085, "countyName": "Lake", "cityID": 3940642, "cityName": "Kirtland" } }
+{ "create_at": datetime("2015-11-25T10:00:05.000Z"), "id": 669576303486435329, "text": "Amanecí con mocos :(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 3240432476, "name": "Lil", "screen_name": "lilscorral", "lang": "en", "location": "null", "create_at": date("2015-06-08"), "description": "it'll be fine", "followers_count": 192, "friends_count": 151, "statues_count": 2079 }, "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("2015-11-25T10:00:05.000Z"), "id": 669576303826309120, "text": "On a scale of 1-10, she about a 12 and a half.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2211252501, "name": "pine needle", "screen_name": "TubesockSquad", "lang": "en", "location": "#superlowkeylegend", "create_at": date("2013-12-06"), "description": "I'm funny af if anyone would check my tweets insta: fuglygang // snapchat : jayso_n // I OWN DICK CREAM BOI ↓↓↓↓↓↓ FWM", "followers_count": 370, "friends_count": 1071, "statues_count": 3903 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berea, KY", "id": "79331dfa19658042", "name": "Berea", "place_type": "city", "bounding_box": rectangle("-84.352921,37.526946 -84.236204,37.657644") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21151, "countyName": "Madison", "cityID": 2105842, "cityName": "Berea" } }
+{ "create_at": datetime("2015-11-25T10:00:05.000Z"), "id": 669576303910219776, "text": "Sigh https://t.co/p5yKRPmOZZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2946664546, "name": "Jord(H)an(ukkah)", "screen_name": "JordanKatzby", "lang": "en", "location": "null", "create_at": date("2014-12-28"), "description": "Although the butterfly and caterpillar are completely different, they are one and the same.", "followers_count": 214, "friends_count": 205, "statues_count": 9707 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Savage, MD", "id": "013e2fa93b1bd0be", "name": "Savage", "place_type": "city", "bounding_box": rectangle("-76.852175,39.120076 -76.776884,39.179931") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24027, "countyName": "Howard", "cityID": 2470475, "cityName": "Savage" } }
+{ "create_at": datetime("2015-11-25T10:00:05.000Z"), "id": 669576304069615619, "text": "But I guess least terrible also means best", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3037878478, "name": "emo grandmother ll.", "screen_name": "SuarezSucks", "lang": "en", "location": "The Black Box", "create_at": date("2015-02-14"), "description": "¡zotorro! // Shameless Morrissey Wannabe\nthey/them", "followers_count": 729, "friends_count": 684, "statues_count": 9150 }, "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("2015-11-25T10:00:05.000Z"), "id": 669576304161890305, "text": "I'm a big fan of people being exactly who they are.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 455113055, "name": "Tone", "screen_name": "_Audio_", "lang": "en", "location": "Copiague, NY", "create_at": date("2012-01-04"), "description": "null", "followers_count": 7944, "friends_count": 5790, "statues_count": 48218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Copiague, NY", "id": "72b5e96d44aaac21", "name": "Copiague", "place_type": "city", "bounding_box": rectangle("-73.417569,40.655057 -73.366646,40.694185") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3618146, "cityName": "Copiague" } }
+{ "create_at": datetime("2015-11-25T10:00:05.000Z"), "id": 669576304195301376, "text": "sound bout right .... https://t.co/xtZqjFCbXF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1075818836, "name": "ZAE DOGG. ™", "screen_name": "A1_Zae", "lang": "en", "location": "IG: A1Zae | Macon Ga", "create_at": date("2013-01-09"), "description": "A1$D1.⛽️♣️ BMG1017Bandz.", "followers_count": 1617, "friends_count": 1405, "statues_count": 46120 }, "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("2015-11-25T10:00:05.000Z"), "id": 669576304300130304, "text": "@SETXathlete , Too Bad I Won't ��", "in_reply_to_status": 669576155964403712, "in_reply_to_user": 1080705955, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1080705955 }}, "user": { "id": 61707275, "name": "8 Weeks ❣", "screen_name": "QueenKieee_", "lang": "en", "location": "null", "create_at": date("2009-07-30"), "description": "Aubrey Janelle ✨ Tory Lanez Wife ❤️", "followers_count": 1198, "friends_count": 581, "statues_count": 54971 }, "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("2015-11-25T10:00:05.000Z"), "id": 669576304682016770, "text": "Being thankful is good for the body and the mind, research suggests https://t.co/tluJBRJuky", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3047925973, "name": "Ian Weissman, DO", "screen_name": "DrIanWeissman", "lang": "en", "location": "Milwaukee, WI", "create_at": date("2015-02-27"), "description": "Radiologist. Advocate of improving the patient experience, healthcare innovation, leadership best practices, value-based medicine. Opinions are my own.", "followers_count": 2965, "friends_count": 2999, "statues_count": 5559 }, "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("2015-11-25T10:00:05.000Z"), "id": 669576305109803016, "text": "Lou's ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2770622650, "name": "Sèrria✨", "screen_name": "b0mbri", "lang": "en", "location": "Bloomfield College.", "create_at": date("2014-09-14"), "description": "effb0mbgorg...", "followers_count": 406, "friends_count": 445, "statues_count": 7661 }, "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("2015-11-25T10:00:05.000Z"), "id": 669576305415852033, "text": "Wind 0.7 mph N. Barometer 29.92 in, Rising slowly. Temperature 55.6 °F. Rain today 0.05 in. Humidity 48%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.13472222,33.81666667"), "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": 20, "friends_count": 92, "statues_count": 12805 }, "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("2015-11-25T10:00:05.000Z"), "id": 669576305424232450, "text": "Black Friday and Cyber Monday Specials!...https://t.co/kE52KF1c4F", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.1333,40.1806"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2409824671, "name": "Elements of Horsham", "screen_name": "ElementsHorsham", "lang": "en", "location": "Horsham, PA", "create_at": date("2014-03-24"), "description": "Not All Massage Is Created Equal. At Elements, we focus on providing a superior therapeutic experience. Call 267-282-4215.", "followers_count": 686, "friends_count": 632, "statues_count": 320 }, "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("2015-11-25T10:00:06.000Z"), "id": 669576305608818688, "text": "@sevyn killed say it ���� ��", "in_reply_to_status": -1, "in_reply_to_user": 46564873, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 46564873 }}, "user": { "id": 35988102, "name": "J-FORDÈ 08/30", "screen_name": "UR_Ambassador_J", "lang": "en", "location": "FL - DC - ATL ", "create_at": date("2009-04-27"), "description": "- He Who Kneels Before God Can Stand Before Anyone - : #⃣Go Rattlers IG: Chic_Tactics #TeamTwirl #TeamKenya #FutureAttorney #BEYHIVE", "followers_count": 1419, "friends_count": 1649, "statues_count": 74239 }, "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("2015-11-25T10:00:06.000Z"), "id": 669576305789108224, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 559952962, "name": "saul", "screen_name": "XBPC_", "lang": "en", "location": "null", "create_at": date("2012-04-21"), "description": "♔ HHS ♔ C/O ➀➅", "followers_count": 572, "friends_count": 405, "statues_count": 24577 }, "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("2015-11-25T10:00:06.000Z"), "id": 669576305822867456, "text": "Cam that nigga. https://t.co/Jmq0vDKAzc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 426424139, "name": "SushiBoy5⃣⭕️5⃣Mexico", "screen_name": "sushiboyMEXICO", "lang": "en", "location": "Atlanta, GA", "create_at": date("2011-12-01"), "description": "Follow @sushiboyzdie #HSE #LostSovereignty #sushiboyz thats it bruh bruh ..", "followers_count": 708, "friends_count": 270, "statues_count": 70711 }, "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("2015-11-25T10:00:06.000Z"), "id": 669576305919266816, "text": "I was hoping to skip the Xanax this thanksgiving since everyone's out of town. But, I'm dog-sitting this crazy chihuahua, so...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3239557449, "name": "HippienPearls", "screen_name": "hippienpearls", "lang": "en", "location": "Charleston, SC", "create_at": date("2015-05-06"), "description": "I'd rather have flowers in my hair than diamonds around my neck...", "followers_count": 75, "friends_count": 87, "statues_count": 2356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Pleasant, SC", "id": "37bcd9d2fe172622", "name": "Mount Pleasant", "place_type": "city", "bounding_box": rectangle("-79.922434,32.766578 -79.75221,32.929315") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45019, "countyName": "Charleston", "cityID": 4548535, "cityName": "Mount Pleasant" } }
+{ "create_at": datetime("2015-11-25T10:00:06.000Z"), "id": 669576306154147840, "text": "�� https://t.co/JScfCwnIvu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 313336365, "name": "Francisco.", "screen_name": "HelloFrancisco_", "lang": "en", "location": "Columbus, OH", "create_at": date("2011-06-08"), "description": "lol @ my life...19 // @Semiperfect", "followers_count": 1306, "friends_count": 331, "statues_count": 35117 }, "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("2015-11-25T10:00:06.000Z"), "id": 669576306741391361, "text": "@kailaaa_xo CONGRATS�������������������� so happy to see you accomplish your goals", "in_reply_to_status": 669574032522653696, "in_reply_to_user": 2332009948, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2332009948 }}, "user": { "id": 357503975, "name": "Valentina Munoz", "screen_name": "_Valentinamunoz", "lang": "en", "location": "Miami✈️CT", "create_at": date("2011-08-18"), "description": "沖縄 • Miami • CT // FHS Sen16r", "followers_count": 322, "friends_count": 180, "statues_count": 9093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Hill, CT", "id": "242937dfd73dc7cc", "name": "Long Hill", "place_type": "city", "bounding_box": rectangle("-72.074221,41.341193 -72.035723,41.364499") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9011, "countyName": "New London", "cityID": 943790, "cityName": "Long Hill" } }
+{ "create_at": datetime("2015-11-25T10:00:06.000Z"), "id": 669576306871275520, "text": "intelligence is so attractive", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3189475836, "name": "Chase Miller", "screen_name": "ChaseMillllllll", "lang": "en", "location": "null", "create_at": date("2015-05-09"), "description": "#waka2016 #freemigos #freegucci", "followers_count": 42, "friends_count": 63, "statues_count": 770 }, "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("2015-11-25T10:00:06.000Z"), "id": 669576307047550976, "text": "I stress about the littlest things ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 612249910, "name": "‼️ sami newsome ‼️", "screen_name": "SamiPaigee", "lang": "en", "location": "Dalton, GA", "create_at": date("2012-06-18"), "description": "instagram-samipaige12 / sc-samipaigeee / @austinlong_25 ♥️", "followers_count": 1035, "friends_count": 736, "statues_count": 11486 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dalton, GA", "id": "85ee217f155e3ad8", "name": "Dalton", "place_type": "city", "bounding_box": rectangle("-85.037178,34.701611 -84.914986,34.833507") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13313, "countyName": "Whitfield", "cityID": 1321380, "cityName": "Dalton" } }
+{ "create_at": datetime("2015-11-25T10:00:06.000Z"), "id": 669576307160690688, "text": "Someone wanna give me a ride some where tonight, I got gas money ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2845631961, "name": "KayKay", "screen_name": "kayannalawler19", "lang": "en", "location": "the valley", "create_at": date("2014-10-26"), "description": "♓️♓️", "followers_count": 295, "friends_count": 229, "statues_count": 3513 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Apple Valley, MN", "id": "2dbc9973dd80d3a3", "name": "Apple Valley", "place_type": "city", "bounding_box": rectangle("-93.248871,44.717447 -93.156006,44.775899") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27037, "countyName": "Dakota", "cityID": 2701900, "cityName": "Apple Valley" } }
+{ "create_at": datetime("2015-11-25T10:00:06.000Z"), "id": 669576307324272640, "text": "don't get too attached, cause people change like seasons", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2400030589, "name": "madison", "screen_name": "_simplymadison_", "lang": "en", "location": "null", "create_at": date("2014-03-20"), "description": "WHITE GUUURRLLL", "followers_count": 366, "friends_count": 929, "statues_count": 2690 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Heath, TX", "id": "e206c812e7a2e03b", "name": "Heath", "place_type": "city", "bounding_box": rectangle("-96.508391,32.813385 -96.43361,32.877147") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48397, "countyName": "Rockwall", "cityID": 4832984, "cityName": "Heath" } }
+{ "create_at": datetime("2015-11-25T10:00:06.000Z"), "id": 669576307362041856, "text": "Sunny this afternoon, high 73 (23 C). Low 61 (16 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712530, "name": "MidlandWeather", "screen_name": "MidlandWeather", "lang": "en", "location": "Midland/Odessa, TX", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Midland/Odessa, TX", "followers_count": 863, "friends_count": 93, "statues_count": 8002 }, "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("2015-11-25T10:00:06.000Z"), "id": 669576307374620673, "text": "Hungry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 356003786, "name": "Danika", "screen_name": "Danika_DP", "lang": "en", "location": "Superior/Duluth", "create_at": date("2011-08-15"), "description": "follow me if youre creepin •insta/snap danika_dp", "followers_count": 1286, "friends_count": 268, "statues_count": 42603 }, "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("2015-11-25T10:00:06.000Z"), "id": 669576307525554176, "text": "@MightyMolasses @StevieEvilCat @Lazlovely @PeacecatOM *goes2da woods2collect more mistletoe 4 xmas blues removal*I'll b back. Sit on him Mo", "in_reply_to_status": 669287875716911104, "in_reply_to_user": 2202607844, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2202607844, 2226701918, 2364568038, 3027659001 }}, "user": { "id": 1020050376, "name": "Diego #RIPLani+Buddy", "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 and header by Hana @paseri021", "followers_count": 3577, "friends_count": 1011, "statues_count": 50453 }, "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-25T10:00:06.000Z"), "id": 669576307550760960, "text": "High above the valley in the sky, floats a man on a line without measure of time. Be free. \nCOS… https://t.co/wMX3uQLf6H", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.593,37.756"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1489927855, "name": "KeepItContagious", "screen_name": "KICwithCOS", "lang": "en", "location": "Salt Lake City, Utah", "create_at": date("2013-06-07"), "description": "Human-powered Outdoor Sporting | Featuring 'The Most' Epic Adventurers! | Adventure . . . It's Contagious! | Join Our Community #KeepItContagious® |", "followers_count": 300, "friends_count": 387, "statues_count": 1131 }, "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": 6039, "countyName": "Madera" } }
+{ "create_at": datetime("2015-11-25T10:00:06.000Z"), "id": 669576307588509696, "text": "pretty sure the doctor doing my blood work just thought I was a heroin addict because of the biolife scar on my veins on my right arm..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 434256220, "name": "hey lo", "screen_name": "lalalaurennn___", "lang": "en", "location": "california", "create_at": date("2011-12-11"), "description": "| boys have it easier but girls have it better |", "followers_count": 704, "friends_count": 544, "statues_count": 16515 }, "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("2015-11-25T10:00:06.000Z"), "id": 669576307810811905, "text": "Where's the plug in Gonzales when you need one? ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2732149414, "name": "Dessss.", "screen_name": "ramirez_desaray", "lang": "en", "location": "null", "create_at": date("2014-08-02"), "description": "4:20 is my fav time of the day college student | 1996. | Austin, Texas", "followers_count": 848, "friends_count": 712, "statues_count": 9099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gonzales, TX", "id": "f2f7802136491bdc", "name": "Gonzales", "place_type": "city", "bounding_box": rectangle("-97.46785,29.487551 -97.412051,29.535813") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48177, "countyName": "Gonzales", "cityID": 4830116, "cityName": "Gonzales" } }
+{ "create_at": datetime("2015-11-25T10:00:06.000Z"), "id": 669576308301561856, "text": "Omg https://t.co/5KvWffRZah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 189771952, "name": "©-swizzle", "screen_name": "casey_gallegos", "lang": "en", "location": "Nowhere USA", "create_at": date("2010-09-11"), "description": "Cant talk right now I'm busy looking for a cool bio. 20.Heather.", "followers_count": 822, "friends_count": 564, "statues_count": 37120 }, "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("2015-11-25T10:00:06.000Z"), "id": 669576308632891392, "text": "This is what I call breakfast un bed #chillingout #champagnebreakfast #breakfastinbed… https://t.co/JU0EZvSnb8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.26833333,42.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "chillingout", "champagnebreakfast", "breakfastinbed" }}, "user": { "id": 114857237, "name": "11TH STREET CAFE", "screen_name": "11THSTREETCAFE", "lang": "en", "location": "327 west 11th street ", "create_at": date("2010-02-16"), "description": "At 11th street cafe, it's all about good coffee, great wine, fantastic food, amazing company,friendly staff, neighborhood\r\natmosphere.....:)", "followers_count": 189, "friends_count": 198, "statues_count": 1078 }, "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.886811") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25003, "countyName": "Berkshire" } }
+{ "create_at": datetime("2015-11-25T10:00:06.000Z"), "id": 669576308863729666, "text": "Everyone tries to do something to be remembered forever. Why?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174840636, "name": "James", "screen_name": "ryk_er", "lang": "en", "location": "Hyrule", "create_at": date("2010-08-04"), "description": "21: Inked with Memories. Dreamer. Beholder of Worlds. \n\nKing to a Queen.", "followers_count": 2977, "friends_count": 104, "statues_count": 13562 }, "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("2015-11-25T10:00:06.000Z"), "id": 669576308867915776, "text": "@luvtko123 yes, it was genius.", "in_reply_to_status": 669575889244594176, "in_reply_to_user": 2802578504, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2802578504 }}, "user": { "id": 3250712323, "name": "working for #TheDons", "screen_name": "sophia0269", "lang": "en", "location": "null", "create_at": date("2015-06-20"), "description": "Met Big Sean 7/31/15, he followed and retweeted me that same day. Zeno followed also. Meeting new people. #findingParadise 11/06/2015", "followers_count": 774, "friends_count": 393, "statues_count": 8309 }, "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-25T10:00:06.000Z"), "id": 669576309002149888, "text": "Hate when rose FaceTimes me when she's driving bc there's ALWAYS cars beeping at her", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1035942482, "name": "$@〽️", "screen_name": "samrobberson", "lang": "en", "location": "Allenwood, NJ", "create_at": date("2012-12-25"), "description": "Some of us are lovers, most of y'all haters snapchat - sammm017", "followers_count": 346, "friends_count": 413, "statues_count": 7833 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Allenwood, NJ", "id": "42b2218bccd763e2", "name": "Allenwood", "place_type": "city", "bounding_box": rectangle("-74.139007,40.116937 -74.084153,40.161872") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3400790, "cityName": "Allenwood" } }
+{ "create_at": datetime("2015-11-25T10:00:06.000Z"), "id": 669576309111185409, "text": "Cualquiera de los dos muy recomendables ��... #ConLaFestejada (@ Sushi Madre in Laredo, TX) https://t.co/mlwC9y4LIc https://t.co/hnzQsnfMae", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-99.44487845,27.56016403"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "ConLaFestejada" }}, "user": { "id": 125528725, "name": "iTommito Official", "screen_name": "iTGarcia1", "lang": "es", "location": "Laredo Tx", "create_at": date("2010-03-22"), "description": "Ing. Industrial egresado del ITNL, Miembro de ICM Laredo Tx. Practicando Ciclismo MTB. Respetuoso de Ideologías y convicciones, respeta las mías,Yo Soy @Chivas", "followers_count": 1141, "friends_count": 1069, "statues_count": 2536 }, "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("2015-11-25T10:00:06.000Z"), "id": 669576309266321410, "text": "@erikaude @erikaudefans thank you Erik, your story has touched the lives of all your fans and countless others. #GodBless", "in_reply_to_status": 669454843132682241, "in_reply_to_user": 28015334, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GodBless" }}, "user_mentions": {{ 28015334, 4265379598 }}, "user": { "id": 1180056812, "name": "At.Long.Last.Apak", "screen_name": "TheGreat_Leader", "lang": "en", "location": "null", "create_at": date("2013-02-14"), "description": "2 Weeks Ago A Man Named Erik Aude Changed My Life, And He Can Change Yours Too...", "followers_count": 271, "friends_count": 157, "statues_count": 2924 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stoughton, WI", "id": "7798cc832f30e315", "name": "Stoughton", "place_type": "city", "bounding_box": rectangle("-89.29946,42.904139 -89.190529,42.981242") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55025, "countyName": "Dane", "cityID": 5577675, "cityName": "Stoughton" } }
+{ "create_at": datetime("2015-11-25T10:00:06.000Z"), "id": 669576309299789824, "text": "@dcuddz But u never did muahaha also I never went ):", "in_reply_to_status": 669576055871565824, "in_reply_to_user": 61123149, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 61123149 }}, "user": { "id": 1651125566, "name": "Tamara♡", "screen_name": "Vaniadarlingg_", "lang": "en", "location": "San Diego, CA", "create_at": date("2013-08-06"), "description": "Vania tamara ♡ Problematic fave ♡ I love creepy things & cute things~\nI'm pink, cute and moody like jigglypuff \\(-^3^-)/\nSnapchat: darlingvania", "followers_count": 97, "friends_count": 52, "statues_count": 3380 }, "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("2015-11-25T10:00:06.000Z"), "id": 669576309434019841, "text": "We're #hiring! Click to apply: Shift Supervisor (US) - https://t.co/3aTwUESa3b #Hospitality #Veterans #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.9241296,37.3343901"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Hospitality", "Veterans", "Job", "Jobs", "CareerArc" }}, "user": { "id": 22489947, "name": "TMJ-SJC HRTA Jobs", "screen_name": "tmj_sjc_hrta", "lang": "en", "location": "San Jose, CA", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in San Jose, CA. Need help? Tweet us at @CareerArc!", "followers_count": 356, "friends_count": 295, "statues_count": 714 }, "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-25T10:00:06.000Z"), "id": 669576309622747136, "text": "#MTVStars 5 seconds of summer https://t.co/nVM0eaL9rW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MTVStars" }}, "user": { "id": 3238499402, "name": "PENGUIN LOVER 13", "screen_name": "destinyware7642", "lang": "en", "location": "null", "create_at": date("2015-06-06"), "description": "l like calum hood cuz he hot. #MTVStars 5 seconds of summer", "followers_count": 31, "friends_count": 46, "statues_count": 382 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576309694033920, "text": "National Security Agency #internship #Job: CAE in Cyber Operations Summer Intern Program (#Honolulu, HI) https://t.co/5nGfXa6j4C #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "internship", "Job", "Honolulu", "Jobs" }}, "user": { "id": 154601054, "name": "TMJ-HON Intern Jobs", "screen_name": "tmj_HON_intern", "lang": "en", "location": "Honolulu, HI", "create_at": date("2010-06-11"), "description": "Follow this account for geo-targeted Internships job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 263, "friends_count": 256, "statues_count": 3 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576309731815424, "text": "Breakfast date with my beautiful mama ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2643741626, "name": "Jorden", "screen_name": "jbarnesssss", "lang": "en", "location": "OC ☀️", "create_at": date("2014-07-13"), "description": "pray. slay. vibe.", "followers_count": 407, "friends_count": 453, "statues_count": 4166 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576309773885441, "text": "https://t.co/cszhfZJ9Mx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3529721296, "name": "Tori!!!!", "screen_name": "yokaiboyfriends", "lang": "en", "location": "Rhode Island, USA", "create_at": date("2015-09-02"), "description": "♥ tori ♥ 22 years old ♥ more yokai than taters in idaho ♥ she/her ♥ meramelion is my adorable son ♥ jinmenken will die by my hand ♥ (nsfw sometimes) ♥", "followers_count": 140, "friends_count": 95, "statues_count": 16638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cranston, RI", "id": "4015afc69a8a8506", "name": "Cranston", "place_type": "city", "bounding_box": rectangle("-71.550463,41.730422 -71.377473,41.807454") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4419180, "cityName": "Cranston" } }
+{ "create_at": datetime("2015-11-25T10:00:07.000Z"), "id": 669576309786304512, "text": "I stack my money just to spend it cause when I'm goin I cannot take it 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": 2316058363, "name": "FreeDeeFreeTFoye", "screen_name": "ChampLiveBang", "lang": "en", "location": "null", "create_at": date("2014-01-28"), "description": "im coolin with my lil niggah fck tha rest", "followers_count": 687, "friends_count": 459, "statues_count": 14997 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576309975048192, "text": "a team that prays ���� together stays together. ��. ��lood ��rothers https://t.co/EDA3uAc7bE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 290803005, "name": "J'Chris 5⃣1⃣❌", "screen_name": "__Top_guy__", "lang": "en", "location": "null", "create_at": date("2011-04-30"), "description": "null", "followers_count": 421, "friends_count": 705, "statues_count": 3243 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576310105083904, "text": "Morning showers this afternoon, high 76 (24 C). Low 68 (20 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712593, "name": "CorpusChristiWX", "screen_name": "CorpusChristiWX", "lang": "en", "location": "Corpus Christi, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Corpus Christi, Texas", "followers_count": 1004, "friends_count": 93, "statues_count": 7994 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576310197358592, "text": "Service blows again!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.921686,42.99849"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2757164168, "name": "Lisa Henry", "screen_name": "frecklip", "lang": "en", "location": "null", "create_at": date("2014-08-22"), "description": "null", "followers_count": 40, "friends_count": 94, "statues_count": 35 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mama Lena's Pizzeria", "id": "07d9f348db886000", "name": "Mama Lena's Pizzeria", "place_type": "poi", "bounding_box": rectangle("-70.92168609999999,42.998489899999996 -70.921686,42.99849") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33015, "countyName": "Rockingham" } }
+{ "create_at": datetime("2015-11-25T10:00:07.000Z"), "id": 669576310407241729, "text": "it really sucks not having 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": 768158186, "name": "ᗰᗩᖇIᗩ ☯", "screen_name": "mariaqdoezit_", "lang": "en", "location": "null", "create_at": date("2012-08-19"), "description": "if you never try, you'll never know", "followers_count": 530, "friends_count": 508, "statues_count": 9383 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vernon Hills, IL", "id": "01db8b750a1c4fe4", "name": "Vernon Hills", "place_type": "city", "bounding_box": rectangle("-87.9943,42.20041 -87.898682,42.269654") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1777694, "cityName": "Vernon Hills" } }
+{ "create_at": datetime("2015-11-25T10:00:07.000Z"), "id": 669576310516260864, "text": "Tyrell I love you so very much Thanksgiving is going to be so much love makeing im ready for you https://t.co/4PZyrzuJ5E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.5877015,34.2691228"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2978280953, "name": "Peaches Newton", "screen_name": "BeyonceNewton", "lang": "en", "location": "null", "create_at": date("2015-01-14"), "description": "null", "followers_count": 1140, "friends_count": 2011, "statues_count": 12168 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newberry, SC", "id": "a83e4dd26ae7b004", "name": "Newberry", "place_type": "city", "bounding_box": rectangle("-81.657544,34.255599 -81.571202,34.315219") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45071, "countyName": "Newberry", "cityID": 4549570, "cityName": "Newberry" } }
+{ "create_at": datetime("2015-11-25T10:00:07.000Z"), "id": 669576310520434690, "text": "Thank you, @bama_tweets! I hope you and your family have a wonderful, blessed Thanksgiving!", "in_reply_to_status": 669492048832143361, "in_reply_to_user": 474877249, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 474877249 }}, "user": { "id": 55397102, "name": "Debby Hatcher", "screen_name": "DebbyHatcher", "lang": "en", "location": "Alabama", "create_at": date("2009-07-09"), "description": "Be nice. Seriously. It's not that difficult. Just be nice.", "followers_count": 291, "friends_count": 371, "statues_count": 363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albertville, AL", "id": "311c9292b4055a54", "name": "Albertville", "place_type": "city", "bounding_box": rectangle("-86.266183,34.220164 -86.146056,34.306364") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1095, "countyName": "Marshall", "cityID": 100988, "cityName": "Albertville" } }
+{ "create_at": datetime("2015-11-25T10:00:07.000Z"), "id": 669576310600163329, "text": "You and yours VS Me and mine ain't even fair b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 204288097, "name": "Young Fre$h ♊️©", "screen_name": "Whiteb0i_Tatted", "lang": "en", "location": "Orange,NJ", "create_at": date("2010-10-18"), "description": "Golfer⛳️ Live for today because tomorrow is never promised #LongLiveSlimBz", "followers_count": 207, "friends_count": 187, "statues_count": 7936 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Holiday City South, NJ", "id": "013f90ff78ba4f7b", "name": "Holiday City South", "place_type": "city", "bounding_box": rectangle("-74.329643,39.879125 -74.190235,39.987032") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean", "cityID": 3432418, "cityName": "Holiday City South" } }
+{ "create_at": datetime("2015-11-25T10:00:07.000Z"), "id": 669576311065677826, "text": "@DanburyFair brother is working 2-til 10pm *dennis Keefe* picture people. mess with computer system n phones plz (:", "in_reply_to_status": -1, "in_reply_to_user": 40071207, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 40071207 }}, "user": { "id": 1437760195, "name": "Elizaßêth .", "screen_name": "KnicKindaGirl5", "lang": "en", "location": "null", "create_at": date("2013-05-18"), "description": "NYK|NYR| ¤ Denver -8- ƒriendly -ZAB. ♢ Coffeêholic • Gamer. Tech Savvy . Ct bound sometimes ¦ 620 } 26 years young ø lilard. |s2E5 ZAB.|", "followers_count": 404, "friends_count": 1187, "statues_count": 18384 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Danbury, CT", "id": "a5b6bdd8008412b1", "name": "Danbury", "place_type": "city", "bounding_box": rectangle("-73.545122,41.352784 -73.400835,41.463455") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 918430, "cityName": "Danbury" } }
+{ "create_at": datetime("2015-11-25T10:00:07.000Z"), "id": 669576311489212417, "text": "Happy Thanksgiving!! I am so grateful for all my wonderful family and amazing friends! Cheers!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 310609375, "name": "Mimi Patterson", "screen_name": "mimi_patterson", "lang": "en", "location": "null", "create_at": date("2011-06-03"), "description": "null", "followers_count": 23, "friends_count": 261, "statues_count": 74 }, "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-25T10:00:07.000Z"), "id": 669576311711637505, "text": "@savannah111200 @mcr121110 me too....oh Morgan", "in_reply_to_status": 669576193688117248, "in_reply_to_user": 2277243658, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2277243658, 708473809 }}, "user": { "id": 323609058, "name": "Jen", "screen_name": "jennacwynn", "lang": "en", "location": "Fenton, MO", "create_at": date("2011-06-24"), "description": "cool, calm, and collected", "followers_count": 304, "friends_count": 234, "statues_count": 4460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fenton, MO", "id": "00b12a4d4b1e946f", "name": "Fenton", "place_type": "city", "bounding_box": rectangle("-90.49599,38.461929 -90.405045,38.55905") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2923950, "cityName": "Fenton" } }
+{ "create_at": datetime("2015-11-25T10:00:07.000Z"), "id": 669576311749263360, "text": "Lol don't play me https://t.co/yhvnlr7ZJR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 870963180, "name": "Queen", "screen_name": "_yvoonee", "lang": "en", "location": "null", "create_at": date("2012-10-09"), "description": "null", "followers_count": 708, "friends_count": 422, "statues_count": 32704 }, "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-25T10:00:07.000Z"), "id": 669576311787032576, "text": "so many th��tties at the union right now ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-106.503323,31.771646"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 211398067, "name": "®", "screen_name": "airthot", "lang": "en", "location": "el paso", "create_at": date("2010-11-02"), "description": "who cares", "followers_count": 195, "friends_count": 76, "statues_count": 3733 }, "place": { "country": "United States", "country_code": "United States", "full_name": "UTEP Union East", "id": "07d9e3ca10083000", "name": "UTEP Union East", "place_type": "poi", "bounding_box": rectangle("-106.50332309999999,31.7716459 -106.503323,31.771646") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2015-11-25T10:00:07.000Z"), "id": 669576311866859520, "text": "I had the best mug! Lmao https://t.co/GYuGoVeDyS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 529035537, "name": "Jawan", "screen_name": "FameOverFailure", "lang": "en", "location": "Cleveland, Ohio", "create_at": date("2012-03-18"), "description": "#OSU17 | Musician\n| GFC is fam |\rCleveland breed |", "followers_count": 513, "friends_count": 525, "statues_count": 19975 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576312013484032, "text": "@MoFitz412 Looks like that will be necessary, then! Kinda dope looking, even.", "in_reply_to_status": 669567548648587264, "in_reply_to_user": 380680050, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 380680050 }}, "user": { "id": 22297984, "name": "Stoots", "screen_name": "stoots_will", "lang": "en", "location": "Denver", "create_at": date("2009-02-28"), "description": "#Selffulfillingprophet #Sober #SneakerMisfit #Maven #Accountant", "followers_count": 447, "friends_count": 669, "statues_count": 17845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Speer, Denver", "id": "35b1272b4ab2b4b2", "name": "Speer", "place_type": "neighborhood", "bounding_box": rectangle("-104.987584,39.71109 -104.972902,39.727313") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2015-11-25T10:00:07.000Z"), "id": 669576312231563264, "text": "@abby_lutz10 https://t.co/qclHnGMRbo", "in_reply_to_status": -1, "in_reply_to_user": 627882069, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 627882069 }}, "user": { "id": 3149849144, "name": "ponyo", "screen_name": "kmsgxx", "lang": "en", "location": "water[s]", "create_at": date("2015-04-11"), "description": "null", "followers_count": 759, "friends_count": 347, "statues_count": 5852 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576312235753472, "text": "Niggas don't know how to be themselves", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2377817904, "name": "Ton", "screen_name": "MoneyTeamTon", "lang": "en", "location": "Dallas ✈️ Commerce", "create_at": date("2014-03-07"), "description": "21 | Black & Educated | Texas A&M Commerce #MoneyTeam #LongLiveClyde IG:moneyteamton Snapchat: liltondadon", "followers_count": 32227, "friends_count": 543, "statues_count": 9418 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576312541982721, "text": ":p...............1.2!..............:p https://t.co/jHpPw6935O", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 18449315, "name": "michaeltbee", "screen_name": "michaeltbee", "lang": "en", "location": ".", "create_at": date("2008-12-29"), "description": ":)maker (raspberry pi), lover of people, literature, poetry, transforming society, promoting equality for LGBT, women, different cultures", "followers_count": 141, "friends_count": 363, "statues_count": 2306 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minnetonka, MN", "id": "555209e7a34626a2", "name": "Minnetonka", "place_type": "city", "bounding_box": rectangle("-93.523339,44.890915 -93.398853,44.978956") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743252, "cityName": "Minnetonka" } }
+{ "create_at": datetime("2015-11-25T10:00:07.000Z"), "id": 669576312571301888, "text": "I'm thankful I live in an age of food that resembles food #Thanksgiving https://t.co/0eUSb7jb8D", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Thanksgiving" }}, "user": { "id": 892256078, "name": "Yi Yu", "screen_name": "yixxyu", "lang": "en", "location": "Manhattan", "create_at": date("2012-10-19"), "description": "null", "followers_count": 72, "friends_count": 125, "statues_count": 361 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisville, CO", "id": "5d1bffd975c6ff73", "name": "Louisville", "place_type": "city", "bounding_box": rectangle("-105.183643,39.939576 -105.099812,39.998224") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8013, "countyName": "Boulder", "cityID": 846355, "cityName": "Louisville" } }
+{ "create_at": datetime("2015-11-25T10:00:07.000Z"), "id": 669576312680411137, "text": "Cloudy this afternoon, high 65 (18 C). Low 60 (16 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712706, "name": "TulsaWX", "screen_name": "TulsaWX", "lang": "en", "location": "Tulsa, Oklahoma", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Tulsa, Oklahoma", "followers_count": 510, "friends_count": 93, "statues_count": 8056 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576312965758977, "text": "Where is Youngstown on the map? Play the game at https://t.co/188jL3m06w #Youngstown", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.6495,41.0998"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Youngstown" }}, "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": 935, "friends_count": 312, "statues_count": 2436681 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576313015939072, "text": "Presale for @thecure & @thetwilightsad in Vancouver starts now. Codes: thecure2016, TRACKS, COMMUNITY. https://t.co/xBeIKrAirs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 474833666, 23094396 }}, "user": { "id": 20106036, "name": "Chain of Flowers", "screen_name": "CraigatCoF", "lang": "en", "location": "New Orleans", "create_at": date("2009-02-04"), "description": "19 years of news, rumors, minutiae about The Cure.", "followers_count": 4532, "friends_count": 449, "statues_count": 29682 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576313020284928, "text": "Niggas always wanna fight, I don't give a fuck about y'all! I just enjoy the show", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 256057189, "name": "California's Dad", "screen_name": "MarryJuwana", "lang": "en", "location": "control your environment", "create_at": date("2011-02-22"), "description": "mind over matter | ju2xent@gmail.com | follow @celpadrai #freemydad", "followers_count": 1149, "friends_count": 438, "statues_count": 51541 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576313066270720, "text": "I just my mom to come home", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1401374996, "name": "10|02 ♎", "screen_name": "kmhxx_", "lang": "en", "location": "Killeen, TX", "create_at": date("2013-05-03"), "description": "Rita Phillips ❤", "followers_count": 523, "friends_count": 534, "statues_count": 21258 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576313125134336, "text": "N4TRQ-8 Near Famous Uncle Al's Hotdogs, Chesapeake, VA https://t.co/Jebzp2ND2R #APRS 13b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.23267,36.77633"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "APRS" }}, "user": { "id": 56001919, "name": "Arthur", "screen_name": "n4trqjedi", "lang": "en", "location": "APRS N of Chesapeake, VA", "create_at": date("2009-07-11"), "description": "Electronics Technician and Amateur Radio Operator N4TRQ\r\nhttp://n4trq.wordpress.com", "followers_count": 419, "friends_count": 681, "statues_count": 46493 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576313129316352, "text": "@thesadtitty then ur not gonna kickstart my heart", "in_reply_to_status": 669576168287399938, "in_reply_to_user": 2156818871, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2156818871 }}, "user": { "id": 2822281770, "name": "cinnamon girl", "screen_name": "dmxfiles", "lang": "en", "location": "Carmel, IN", "create_at": date("2014-09-20"), "description": "hemorrhoid demons...COME OUT! @thebigtitty", "followers_count": 212, "friends_count": 182, "statues_count": 35358 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576313179508737, "text": "good morning I just walked into a glass door", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1433771815, "name": "soda", "screen_name": "bbysoda", "lang": "en", "location": "ig:@angelasoda", "create_at": date("2013-05-16"), "description": "null", "followers_count": 870, "friends_count": 506, "statues_count": 11281 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576313250803712, "text": "We're #hiring! Read about our latest #job opening here: Patient Transporter - https://t.co/3vYSatgTXQ #Springfield, MO #CustomerService", "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": {{ "hiring", "job", "Springfield", "CustomerService" }}, "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": 280, "friends_count": 4, "statues_count": 4215 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576313326460933, "text": "Mon 146.970 on FT1DR (@ Famous Uncle Al's Hotdogs in Chesapeake, VA) https://t.co/OJ6JgUKdwO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.23257061,36.77578029"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 56001919, "name": "Arthur", "screen_name": "n4trqjedi", "lang": "en", "location": "APRS N of Chesapeake, VA", "create_at": date("2009-07-11"), "description": "Electronics Technician and Amateur Radio Operator N4TRQ\r\nhttp://n4trq.wordpress.com", "followers_count": 419, "friends_count": 681, "statues_count": 46493 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576313599037440, "text": "Happy Thanksgiving to everyone!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46642435, "name": "Terry True", "screen_name": "terrytrue77", "lang": "en", "location": "Millington, TN", "create_at": date("2009-06-12"), "description": "Single", "followers_count": 102, "friends_count": 847, "statues_count": 631 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576313653567489, "text": "�� https://t.co/WfgvMmfeDS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1658600425, "name": "a milli", "screen_name": "fvckpictures", "lang": "en", "location": "Broward County, FL ", "create_at": date("2013-08-09"), "description": "Police explorer | 18", "followers_count": 1616, "friends_count": 1623, "statues_count": 49053 }, "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("2015-11-25T10:00:07.000Z"), "id": 669576313737383936, "text": "there's no time limit for pain", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2484569708, "name": "madeleine✨", "screen_name": "mAdeleinenaquin", "lang": "en", "location": "probably reading", "create_at": date("2014-05-08"), "description": "mess of broken veins♊️", "followers_count": 146, "friends_count": 149, "statues_count": 1038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Belle Chasse, LA", "id": "018b34730580d977", "name": "Belle Chasse", "place_type": "city", "bounding_box": rectangle("-90.068239,29.769816 -89.975552,29.904303") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22075, "countyName": "Plaquemines", "cityID": 2206120, "cityName": "Belle Chasse" } }
+{ "create_at": datetime("2015-11-25T10:00:07.000Z"), "id": 669576313817067520, "text": "I'm hungry someone bring me food ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1602238572, "name": "Nia Ellen", "screen_name": "_Ihateevrybody", "lang": "en", "location": "California, USA", "create_at": date("2013-07-17"), "description": "Nothing worth it comes easy #LongliveMeech", "followers_count": 255, "friends_count": 277, "statues_count": 5316 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576313850699777, "text": "@JustTishh_ omg this just made me even more excited", "in_reply_to_status": 669346182309744640, "in_reply_to_user": 95962569, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 95962569 }}, "user": { "id": 235296714, "name": "Ebster", "screen_name": "x_EbbyJay", "lang": "en", "location": "null", "create_at": date("2011-01-07"), "description": "Ebby EHS cheerleader PantherNation.✨", "followers_count": 657, "friends_count": 458, "statues_count": 13475 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576313880072192, "text": "A man just screamed \"you look like a lot of FUN\" in my face on 8th Ave so on that note, headed to the burbs for a 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": 32234270, "name": "A$AP R$ilvz", "screen_name": "rsilvz", "lang": "en", "location": "NYC", "create_at": date("2009-04-16"), "description": "fries on a salad? A+", "followers_count": 307, "friends_count": 278, "statues_count": 1966 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576313913479168, "text": "In route... @ I10 Florida https://t.co/pJtxnIMUE0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.44979192,30.74551449"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 39839511, "name": "Annas Army", "screen_name": "abarton87", "lang": "en", "location": "GA/NY", "create_at": date("2009-05-13"), "description": "Promotion - Annas Army", "followers_count": 519, "friends_count": 789, "statues_count": 5927 }, "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": 12133, "countyName": "Washington" } }
+{ "create_at": datetime("2015-11-25T10:00:08.000Z"), "id": 669576313947074560, "text": "Malik funny ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3289466419, "name": "MannLongWayy❤️", "screen_name": "PoppaPete_", "lang": "en", "location": "318✈️✈️337", "create_at": date("2015-07-23"), "description": "Student-Athlete @ Mcneese State Father of Kamryn Dewayne Kimble!! #GoPokes Ig: _petey SC: Petey_5 #SamStrong #BlackLivesMatter", "followers_count": 808, "friends_count": 860, "statues_count": 6629 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576314123239424, "text": "I’ve picked Iowa to be my @DrPepper #OneFinalTeam. https://t.co/UWiWtTtGCG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "OneFinalTeam" }}, "user_mentions": {{ 19720019 }}, "user": { "id": 1244773423, "name": "Jé The Third", "screen_name": "jayhawk_insIII", "lang": "en", "location": "The LX", "create_at": date("2013-03-05"), "description": "Ferris Bueller, you're my hero", "followers_count": 481, "friends_count": 399, "statues_count": 1636 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576314475560962, "text": "@3Gomez2 you coming to nebraska this weekend ��", "in_reply_to_status": 669574761874366464, "in_reply_to_user": 75817782, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 75817782 }}, "user": { "id": 228460951, "name": "Jackieℓynn♡†️", "screen_name": "JackieLynnn_", "lang": "en", "location": "Guadalajara, Jalisco", "create_at": date("2010-12-19"), "description": "null", "followers_count": 634, "friends_count": 557, "statues_count": 24582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hastings, NE", "id": "ac9830fe327eb63f", "name": "Hastings", "place_type": "city", "bounding_box": rectangle("-98.467334,40.561183 -98.347736,40.646339") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31001, "countyName": "Adams", "cityID": 3121415, "cityName": "Hastings" } }
+{ "create_at": datetime("2015-11-25T10:00:08.000Z"), "id": 669576314504900608, "text": "@MSorrinni ...propane. sounds the same. Library sometimes sounds like Liiiibrary though :)", "in_reply_to_status": 669573085704536064, "in_reply_to_user": 2275021952, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2275021952 }}, "user": { "id": 535826039, "name": "Judith Slaughter", "screen_name": "txfoo", "lang": "en", "location": "S a n A n t o n i o T e x a s", "create_at": date("2012-03-24"), "description": "Cherry chapstick. Aquarius. Shoegazer. Paid to hang out in public libraries since 2007.", "followers_count": 97, "friends_count": 348, "statues_count": 3307 }, "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-25T10:00:08.000Z"), "id": 669576314521853952, "text": "Cause your love give me goosebumps", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 42539518, "name": "Eva Athena", "screen_name": "greek_freek", "lang": "en", "location": "CT -- London", "create_at": date("2009-05-25"), "description": "Turn your block club to my fan club", "followers_count": 233, "friends_count": 302, "statues_count": 18972 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cranbury, NJ", "id": "010cea6f5a910a2b", "name": "Cranbury", "place_type": "city", "bounding_box": rectangle("-74.54946,40.288358 -74.476519,40.338744") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3415520, "cityName": "Cranbury" } }
+{ "create_at": datetime("2015-11-25T10:00:08.000Z"), "id": 669576314580439041, "text": "Lol at the ones I found in my purse thanks to @daniellyferris ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 840747474 }}, "user": { "id": 2353608223, "name": "Katie-Rose", "screen_name": "katierosed94", "lang": "en", "location": "CA ✈️ TX", "create_at": date("2014-02-20"), "description": "SHSU coed cheer•AGFL•SFCF•Future teacher•Blessed•RIP Jeff ❤️", "followers_count": 336, "friends_count": 321, "statues_count": 4597 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576314605694976, "text": "Ripley SW Limestone Co. Temp: 64.6°F Wind:8.3mph Pressure: 1008.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": 37, "friends_count": 33, "statues_count": 44702 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576314639224832, "text": "I am thankful that God gave us His word.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 817710866, "name": "Stephen Bowen ن", "screen_name": "countryboy7477", "lang": "en", "location": "Alabama to Satellite Beach,FL", "create_at": date("2012-09-11"), "description": "I love Jesus, my wife, & kids. HobeSoundBibleCollege. My goal is serving the Lord. Matthew 6:33 & 2 Corinthians 5:17. Visit http://hbcmccalla.com. #2A #Hunt", "followers_count": 8420, "friends_count": 7451, "statues_count": 84305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Patrick Shores, FL", "id": "e5cecfb170a4cbac", "name": "South Patrick Shores", "place_type": "city", "bounding_box": rectangle("-80.620244,28.186323 -80.593197,28.212852") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12009, "countyName": "Brevard", "cityID": 1267725, "cityName": "South Patrick Shores" } }
+{ "create_at": datetime("2015-11-25T10:00:08.000Z"), "id": 669576314664431616, "text": "when i was 13, I thought when i turn 16 I would for sure get my life together.Cuz ppl at that age were driving, had jobs, going out etc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 393927335, "name": "Bɛhiŋd The Mādŋɛssʚɞ", "screen_name": "LadyPrettyWings", "lang": "en", "location": "in Forest Hills Drive w/ bae♛", "create_at": date("2011-10-19"), "description": "African seed. Arab roots. Sudanese flower. watered by islam. |you can read between the lines, but i promise that you will not understand..| #AlwaysWithKidrauhl", "followers_count": 2952, "friends_count": 657, "statues_count": 31726 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576314672816128, "text": "@jimmyCZ50 @parsons412 @NateWade412 @BryanKane11 @BleacherReport $50 they don't", "in_reply_to_status": 669576233391366145, "in_reply_to_user": 521327202, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 521327202, 2204112285, 3496344617, 3408126069, 890891 }}, "user": { "id": 1397178678, "name": "Caramel Delight", "screen_name": "jahova11", "lang": "en", "location": "null", "create_at": date("2013-05-02"), "description": "free bro", "followers_count": 193, "friends_count": 114, "statues_count": 11621 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576314798612480, "text": "I hate that I know what this surprise is that's gone be for my mom�� I want to tell her she's gone be 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": 3142621838, "name": "Ka.", "screen_name": "bossskay", "lang": "en", "location": "null", "create_at": date("2015-04-06"), "description": "STILL no fucks given..", "followers_count": 256, "friends_count": 216, "statues_count": 2400 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576314903375872, "text": "#CareerArc #CustomerService #Job alert: Meat Part Time Customer Service | Wegmans | #Burlington, MA https://t.co/ynsNByWMC4 #Jobs #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": {{ "CareerArc", "CustomerService", "Job", "Burlington", "Jobs", "Hiring" }}, "user": { "id": 71719032, "name": "TMJ-BOS CstSrv Jobs", "screen_name": "tmj_bos_cstsrv", "lang": "en", "location": "Boston, MA", "create_at": date("2009-09-04"), "description": "Follow this account for geo-targeted Customer Service job tweets in Boston, MA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 320, "friends_count": 287, "statues_count": 176 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576315482296320, "text": "@PAPIIZEEDOE Goodevening Hope You Enjoy Your Day - Add Me On snapchat : Unclesnoopp", "in_reply_to_status": -1, "in_reply_to_user": 2503147280, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2503147280 }}, "user": { "id": 400961655, "name": "TyrekBrown", "screen_name": "Unclesnoop_", "lang": "en", "location": "Marydale,louisiana", "create_at": date("2011-10-29"), "description": "✈️ High Granny , Rip to The Real Niggas , Free My Uncle , Solid To All My Bros ✊ Tyrek Unclesnoop Atm L.N.O", "followers_count": 2382, "friends_count": 2717, "statues_count": 31416 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576315507486720, "text": "First it was #freeboosie na badazz got cancer. You know I'm praying #PrayForBoosie ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "freeboosie", "PrayForBoosie" }}, "user": { "id": 3283161792, "name": "504Neno", "screen_name": "NenoFarian", "lang": "en", "location": "Violet, LA", "create_at": date("2015-07-18"), "description": "#RestVell imma miss you bro RIP TRUY REST IN PEACE TODDRICK #LEBRON #CAVS #saints #GUEAXTIGERS #DUKENATION #BLUEDEVILS Khalil over everything", "followers_count": 233, "friends_count": 429, "statues_count": 5882 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576315536736256, "text": "Sunrise Senior Living #Finance #Job: Resident Billing Associate (#McLean, VA) https://t.co/jWTw3edPZg #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.1774801,38.9342776"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Finance", "Job", "McLean", "Jobs", "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": 296, "friends_count": 9, "statues_count": 2880 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576315549302785, "text": "Wish I was home!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160474356, "name": "Jasmine Reneé", "screen_name": "jasreneewms", "lang": "en", "location": "Dallas TX - Baton Rouge LA", "create_at": date("2010-06-28"), "description": "LSU + Marketing and Sports Studies", "followers_count": 1216, "friends_count": 628, "statues_count": 25782 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576315591372801, "text": "Bae https://t.co/MgKQU6S91s", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 900111098, "name": "Tay", "screen_name": "tlight__", "lang": "en", "location": "502 come up", "create_at": date("2012-10-23"), "description": "high on life, that's what it does for me", "followers_count": 962, "friends_count": 651, "statues_count": 18121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Springboro, OH", "id": "6ae9b8002ff77818", "name": "Springboro", "place_type": "city", "bounding_box": rectangle("-84.277031,39.520846 -84.187373,39.586883") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39165, "countyName": "Warren", "cityID": 3974076, "cityName": "Springboro" } }
+{ "create_at": datetime("2015-11-25T10:00:08.000Z"), "id": 669576316140830720, "text": "UTA Spotlight: Student Affairs Celebrates Japanese Culture https://t.co/vwelwGP3Dn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.1339035,32.69229889"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1322875490, "name": "Arlington, TX News", "screen_name": "BLifeArlington", "lang": "en", "location": "Arlington, Texas", "create_at": date("2013-04-02"), "description": "Arlington BubbleLife features community news, photos and events. Share your business, organization or personal news and events at https://t.co/bGTis0e6jp.", "followers_count": 818, "friends_count": 1558, "statues_count": 49935 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576316384108545, "text": "Oh lord. My bae❤️❤️❤️❤️❤️❤️❤️ https://t.co/S36nRxJYFS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3002273115, "name": "Ya Squared", "screen_name": "yayaj17", "lang": "en", "location": "null", "create_at": date("2015-01-27"), "description": "I'm not inclined to resign to maturity", "followers_count": 384, "friends_count": 368, "statues_count": 1845 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576316417626112, "text": "We got some cloudy at 34ºF going on. #MorrisWeather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.9,45.59"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MorrisWeather" }}, "user": { "id": 1931463259, "name": "Morris Weather", "screen_name": "MorrisMNWeather", "lang": "en", "location": "Morris, MN", "create_at": date("2013-10-03"), "description": "A place to find the current weather in Morris, MN. Run by @bman4789\n\nScheduled tweets are at 7 AM, 12 PM, 3 PM, 6 PM, and 10 PM. Daily forecast at 6 AM.", "followers_count": 20, "friends_count": 1, "statues_count": 1498 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morris, MN", "id": "dc943406a248a125", "name": "Morris", "place_type": "city", "bounding_box": rectangle("-95.933929,45.564348 -95.883709,45.607334") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27149, "countyName": "Stevens", "cityID": 2744242, "cityName": "Morris" } }
+{ "create_at": datetime("2015-11-25T10:00:08.000Z"), "id": 669576316459577344, "text": "I want a pair of these so bad! ���� @eSskateboarding https://t.co/LY2fIjBBwc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21929526 }}, "user": { "id": 117693147, "name": "Lando", "screen_name": "RyanLando", "lang": "en", "location": "Inver Grove Heights, MN", "create_at": date("2010-02-26"), "description": "@DownstreamUSA // snapchat: brolando27 & NoContentSnaps // http://NoContent.org // R.I.P. Alex Hondi Paredes 1993-2015 #Hondi4ever", "followers_count": 339, "friends_count": 703, "statues_count": 9744 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Apple Valley, MN", "id": "2dbc9973dd80d3a3", "name": "Apple Valley", "place_type": "city", "bounding_box": rectangle("-93.248871,44.717447 -93.156006,44.775899") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27037, "countyName": "Dakota", "cityID": 2701900, "cityName": "Apple Valley" } }
+{ "create_at": datetime("2015-11-25T10:00:08.000Z"), "id": 669576316539113472, "text": "Quickly boost holiday/ Christmas sales with a local Google Partner agency. Call 501.505.6278. https://t.co/fCsOjK2M3g", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 44725242, "name": "Jeff A. Turnbow", "screen_name": "reachturnbow", "lang": "en", "location": "Conway, Arkansas", "create_at": date("2009-06-04"), "description": "Internet/Digital Marketing Expert", "followers_count": 167, "friends_count": 259, "statues_count": 158 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576316623040512, "text": "I wish mlb was for xbox ��", "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": "Jakobe", "screen_name": "jakobe980", "lang": "en", "location": "sc: jakobe.8", "create_at": date("2014-10-27"), "description": "fuck it", "followers_count": 331, "friends_count": 158, "statues_count": 17871 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576316652535808, "text": "I LOVE DOGS ������ https://t.co/QDdHeoocXP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1956547172, "name": "Lillie", "screen_name": "lildellapenna", "lang": "en", "location": "sunshine state", "create_at": date("2013-10-12"), "description": "whats good", "followers_count": 1021, "friends_count": 395, "statues_count": 7853 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Largo, FL", "id": "e54569d90c7032c0", "name": "Largo", "place_type": "city", "bounding_box": rectangle("-82.849182,27.865251 -82.696478,27.943275") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1239425, "cityName": "Largo" } }
+{ "create_at": datetime("2015-11-25T10:00:08.000Z"), "id": 669576316790902784, "text": "My Cousin Is Fuckin Hilarious Lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 298287932, "name": ":)", "screen_name": "txxlxrr", "lang": "en", "location": "Washington, DC", "create_at": date("2011-05-13"), "description": "null", "followers_count": 4976, "friends_count": 3053, "statues_count": 97499 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waldorf, MD", "id": "4ddec4770a850303", "name": "Waldorf", "place_type": "city", "bounding_box": rectangle("-76.985159,38.608878 -76.841929,38.660379") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24017, "countyName": "Charles", "cityID": 2481175, "cityName": "Waldorf" } }
+{ "create_at": datetime("2015-11-25T10:00:08.000Z"), "id": 669576317365567488, "text": "Waiting... ⌚️ @ San Jose International Airport (SJC) https://t.co/N79oTnCNxW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.92666642,37.36707907"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 281268314, "name": "Marden Zelaya", "screen_name": "mardenjosue", "lang": "en", "location": "null", "create_at": date("2011-04-12"), "description": "Made in El Salvador ✈️| Living in Cali | God is my everything | Failures are finger posts on the road to achievement. C.S. Lewis✌️", "followers_count": 307, "friends_count": 1269, "statues_count": 6716 }, "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-25T10:00:08.000Z"), "id": 669576317403340800, "text": "A surprising number of pills I'm prescribed for migraines clearly state \"may increase severity of migraines\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 73014749, "name": "Bükem", "screen_name": "bukemr", "lang": "en", "location": "null", "create_at": date("2009-09-09"), "description": "Writer / Managing Editor at Cosmonauts Avenue / Puppy Enthusiast / Llama Apologist / Religion: Beyoncé / Week of the Loner", "followers_count": 1243, "friends_count": 1320, "statues_count": 1183 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Amherst, MA", "id": "c6dc3c4bfa85ba10", "name": "South Amherst", "place_type": "city", "bounding_box": rectangle("-72.540778,42.30165 -72.488846,42.357144") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25015, "countyName": "Hampshire", "cityID": 2562675, "cityName": "South Amherst" } }
+{ "create_at": datetime("2015-11-25T10:00:08.000Z"), "id": 669576317482852356, "text": "Want to work at Community Hospital San Bernardino? We're #hiring in #SanBernardino, CA! Click for details: https://t.co/B4iOP1nGIj #Job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.3205123,34.1242701"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "SanBernardino", "Job" }}, "user": { "id": 22533900, "name": "TMJ-CA-US Nursing", "screen_name": "tmj_CA_NURSING", "lang": "en", "location": "California", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 464, "friends_count": 308, "statues_count": 2718 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576317705125888, "text": "Aim NationaLease #Sales #Job: B2B Outside Sales Manager (#Gainesville, GA) https://t.co/3WrgOABZGJ #Jobs #Hiring #CareerArc", "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": {{ "Sales", "Job", "Gainesville", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 21777778, "name": "TMJ-GA Sales Jobs", "screen_name": "tmj_ga_sales", "lang": "en", "location": "Georgia", "create_at": date("2009-02-24"), "description": "Follow this account for geo-targeted Sales job tweets in Georgia Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 383, "friends_count": 305, "statues_count": 427 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576317835214848, "text": "I'm sending my mom pictures of street bikes hoping she'll catch on", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2697573321, "name": "tracey", "screen_name": "Tracey_smileyy", "lang": "en", "location": "null", "create_at": date("2014-07-10"), "description": "null", "followers_count": 211, "friends_count": 321, "statues_count": 2657 }, "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("2015-11-25T10:00:08.000Z"), "id": 669576318007185408, "text": "Wind 2.7 mph ESE. Barometer 30.125 in, Falling slowly. Temperature 69.6 °F. Rain today 0.00in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.11555556,29.6725"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25210383, "name": "New Braunfels WX", "screen_name": "jeffreycentexwx", "lang": "en", "location": "New Braunfels, TX", "create_at": date("2009-03-18"), "description": "Weather Station located on County Line Road in New Braunfels, TX.\r\n\r\nRealtime Weather @ http://www.nbtx-wx.com\r\nFollow me @ http://twitter.com/jeffreycentex", "followers_count": 369, "friends_count": 21, "statues_count": 156210 }, "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("2015-11-25T10:00:09.000Z"), "id": 669576318179258369, "text": "Ya me canse de esta vida", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.1478719,41.6913379"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2934182116, "name": "Ángeel.", "screen_name": "PaganAngeel", "lang": "es", "location": "null", "create_at": date("2014-12-20"), "description": "null", "followers_count": 328, "friends_count": 294, "statues_count": 24022 }, "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("2015-11-25T10:00:09.000Z"), "id": 669576318191714304, "text": "Wasted my time son", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2722814067, "name": "Caam", "screen_name": "caamster_", "lang": "en", "location": "new orleans ", "create_at": date("2014-07-24"), "description": "I just wanted people to like me more", "followers_count": 389, "friends_count": 506, "statues_count": 10326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ama, LA", "id": "3b00d3def130ae89", "name": "Ama", "place_type": "city", "bounding_box": rectangle("-90.32121,29.931019 -90.275633,29.961727") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22089, "countyName": "St. Charles", "cityID": 2201780, "cityName": "Ama" } }
+{ "create_at": datetime("2015-11-25T10:00:09.000Z"), "id": 669576318250409984, "text": "Want to work at CVS Health? We're #hiring in #Norwood, MA! Click for details: https://t.co/1C2B3eR8OD #Retail #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.1964952,42.1915258"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Norwood", "Retail", "Job", "Jobs", "CareerArc" }}, "user": { "id": 28442780, "name": "TMJ-BOS Retail Jobs", "screen_name": "tmj_bos_retail", "lang": "en", "location": "Boston, MA", "create_at": date("2009-04-02"), "description": "Follow this account for geo-targeted Retail job tweets in Boston, MA. Need help? Tweet us at @CareerArc!", "followers_count": 408, "friends_count": 318, "statues_count": 1198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwood, MA", "id": "b7d826f55dca45e9", "name": "Norwood", "place_type": "city", "bounding_box": rectangle("-71.238551,42.149887 -71.153926,42.213847") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25021, "countyName": "Norfolk", "cityID": 2550285, "cityName": "Norwood" } }
+{ "create_at": datetime("2015-11-25T10:00:09.000Z"), "id": 669576318376239105, "text": "Cloudy this afternoon, high 75 (24 C). Low 67 (19 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712488, "name": "SanAntonioWX", "screen_name": "SanAntonioWX", "lang": "en", "location": "San Antonio, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for San Antonio, Texas", "followers_count": 245, "friends_count": 93, "statues_count": 8073 }, "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-25T10:00:09.000Z"), "id": 669576318447583232, "text": "Interested in a #Labor #job near #Honolulu, HI? This could be a great fit: https://t.co/RXHXQIG4aW #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Labor", "job", "Honolulu", "Hiring" }}, "user": { "id": 22777294, "name": "TMJ-HON Labor Jobs", "screen_name": "tmj_hon_labor", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-04"), "description": "Follow this account for geo-targeted General Labor job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 299, "friends_count": 291, "statues_count": 4 }, "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("2015-11-25T10:00:09.000Z"), "id": 669576318695006208, "text": "happy birthday to my bro ✊�� @isiah_clement", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 863602406 }}, "user": { "id": 368616142, "name": "tate", "screen_name": "ayeitstate", "lang": "en", "location": "CA", "create_at": date("2011-09-05"), "description": "null", "followers_count": 519, "friends_count": 304, "statues_count": 20917 }, "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("2015-11-25T10:00:09.000Z"), "id": 669576318862823424, "text": "@drwho481 Oh yeah she did! LOL!", "in_reply_to_status": 669575966407180288, "in_reply_to_user": 2172733325, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2172733325 }}, "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": 1677, "friends_count": 915, "statues_count": 29184 }, "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-25T10:00:09.000Z"), "id": 669576318997176320, "text": "������ this boy funny like 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": 249940727, "name": "Diamond", "screen_name": "shitJUICY", "lang": "en", "location": "Montgomery, AL", "create_at": date("2011-02-09"), "description": "21 and about her Buisness !", "followers_count": 3251, "friends_count": 1565, "statues_count": 61328 }, "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("2015-11-25T10:00:09.000Z"), "id": 669576319110369281, "text": "I'm at @FootLocker in Somerset, NJ https://t.co/nKVhQGvpsw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.48326349,40.48771423"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 22030851 }}, "user": { "id": 105950874, "name": "Oso MaRvElOuS", "screen_name": "iamosomarvelous", "lang": "en", "location": "New York, NY", "create_at": date("2010-01-17"), "description": "Oso MaRvLoUs : Dj •Music Producer• Song Writer •Entertainer ! [Real~Deal] .!\n #iamOsoMaRvEloUs-X-Plode - Single https://itun.es/i6SD8ZX #iTunes", "followers_count": 4028, "friends_count": 50, "statues_count": 2844 }, "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("2015-11-25T10:00:09.000Z"), "id": 669576319286550528, "text": "Who's an active? :)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1471554756, "name": "Fwiz - Alex", "screen_name": "ObeyFwiz", "lang": "en", "location": "Minoqua, WI", "create_at": date("2013-05-31"), "description": "☆ Turn My Notifications On ☆ ☆ Former Manager For @TheObeyAlliance ☆", "followers_count": 68053, "friends_count": 3561, "statues_count": 41333 }, "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("2015-11-25T10:00:09.000Z"), "id": 669576319319994369, "text": "I like how he text me \"good morning��\" ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2311048068, "name": "ماريا", "screen_name": "mariaa_234", "lang": "en", "location": "null", "create_at": date("2014-01-25"), "description": "IG: & SC : mariaa_234", "followers_count": 205, "friends_count": 178, "statues_count": 1541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:09.000Z"), "id": 669576319475150848, "text": "@shhainav aw thank you ��", "in_reply_to_status": 669562233366556672, "in_reply_to_user": 849729368, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 849729368 }}, "user": { "id": 339455504, "name": "ashy", "screen_name": "ashypls", "lang": "en", "location": "null", "create_at": date("2011-07-20"), "description": "accept me into college", "followers_count": 583, "friends_count": 494, "statues_count": 14117 }, "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-25T10:00:09.000Z"), "id": 669576319496265730, "text": "@AbzarChoudhury Aye, thanks for following! I'm a music producer. Feel free to take a listen via my catalog @ https://t.co/wLJiWth2W9", "in_reply_to_status": -1, "in_reply_to_user": 176096741, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 176096741 }}, "user": { "id": 2694967513, "name": "The B-ST | HQ Beats", "screen_name": "ItsTheBE4ST", "lang": "en", "location": "The Studio, Fam! Where U @?", "create_at": date("2014-07-30"), "description": "#ProudlyMuslim... #HipHop | #Trap | #RnB Music Producer. Email - thebst@outlook.com | My SC - https://soundcloud.com/the_b-st | My Beats ⬇", "followers_count": 2797, "friends_count": 2333, "statues_count": 5914 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Exeter, NH", "id": "0a5eca4e60fe6f35", "name": "Exeter", "place_type": "city", "bounding_box": rectangle("-71.001194,42.956811 -70.887799,42.997041") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33015, "countyName": "Rockingham", "cityID": 3325300, "cityName": "Exeter" } }
+{ "create_at": datetime("2015-11-25T10:00:09.000Z"), "id": 669576319634702336, "text": "Lol I feel her. I told y'all I only fix plates for people under age 5.. And I don't invite family over who have kids that young.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287882043, "name": "Atira ", "screen_name": "AtiRANDOMness", "lang": "en", "location": "East Tennessee ", "create_at": date("2011-04-25"), "description": "Founder of sarcasm and laughing at people who don't get it. I live for science, football and wine.. Blurring the lines between fanbase & cult. #VolTwitter #PATS", "followers_count": 1248, "friends_count": 819, "statues_count": 168805 }, "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("2015-11-25T10:00:09.000Z"), "id": 669576319701819394, "text": "Thanks for the follow, @pjbracke! Didn't know you were in the Twittersphere!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 12363962 }}, "user": { "id": 10253572, "name": "Alton 'Tony' Zanders", "screen_name": "zanders", "lang": "en", "location": "New Orleans", "create_at": date("2007-11-14"), "description": "Husband/Dad. Global VP @EBSCO. Advisor @BooksThatGrow & @CmlbackVentures. Board Member @RootedSchool. Committed to Christian discipleship & Black excellence.", "followers_count": 883, "friends_count": 250, "statues_count": 13101 }, "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("2015-11-25T10:00:09.000Z"), "id": 669576319835881472, "text": "Starbucks #Hospitality #Job: Barista (US) https://t.co/bNcgWuIVFS #Veterans #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.6957985,28.0494571"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "Job", "Veterans", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22457395, "name": "FL Hospitality Jobs", "screen_name": "tmj_tpa_hrta", "lang": "en", "location": "Tampa Bay, FL", "create_at": date("2009-03-01"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Tampa Bay, FL. Need help? Tweet us at @CareerArc!", "followers_count": 474, "friends_count": 293, "statues_count": 533 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oldsmar, FL", "id": "61eefa163263bef0", "name": "Oldsmar", "place_type": "city", "bounding_box": rectangle("-82.710115,27.999141 -82.648349,28.066154") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1251350, "cityName": "Oldsmar" } }
+{ "create_at": datetime("2015-11-25T10:00:09.000Z"), "id": 669576319932452864, "text": "@ArtistsOfRed could I take a sort of internship at red ?", "in_reply_to_status": -1, "in_reply_to_user": 2963490861, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2963490861 }}, "user": { "id": 4013598863, "name": "Colton Williams", "screen_name": "TimeX_Arts", "lang": "en", "location": "Grove City, OH", "create_at": date("2015-10-21"), "description": "Love to make YouTube videos and make people laugh freelance designer. I do thumbnails for free all you need to do is say I made your thumbnail.Snake enthusiasts", "followers_count": 165, "friends_count": 422, "statues_count": 177 }, "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("2015-11-25T10:00:09.000Z"), "id": 669576319936540672, "text": "Interested in a #SkilledTrade #job near #Sacramento, CA? This could be a great fit: https://t.co/czpEQ8Mber #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.4943996,38.5815719"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SkilledTrade", "job", "Sacramento", "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": 27, "friends_count": 13, "statues_count": 2010 }, "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-25T10:00:09.000Z"), "id": 669576320091873280, "text": "@MetalHead6112 is there a good place to stream it from my phone that you know of? SoundCloud maybe?", "in_reply_to_status": 669574148419514368, "in_reply_to_user": 2657402613, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2657402613 }}, "user": { "id": 178016785, "name": "HE | Mister Eric", "screen_name": "MisterbeepEric", "lang": "en", "location": "Washington, DC", "create_at": date("2010-08-13"), "description": "Smash 4 competitor | ROB main | Sponsored by @Haven_eSports | #ROBSquad #SmashBros", "followers_count": 1171, "friends_count": 798, "statues_count": 9263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reston, VA", "id": "ec212eb1116b92e2", "name": "Reston", "place_type": "city", "bounding_box": rectangle("-77.393246,38.90861 -77.304868,39.002993") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5166672, "cityName": "Reston" } }
+{ "create_at": datetime("2015-11-25T10:00:09.000Z"), "id": 669576320485986304, "text": "#Sales #Job alert: Tax Preparer -- Gain Experience in a top Seacoast Firm! | Accountemps | #Portsmouth, NH https://t.co/XMyitHGnkb #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.7625532,43.0717552"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "Job", "Portsmouth", "Jobs" }}, "user": { "id": 785251808, "name": "Accountemps Jobs", "screen_name": "AccountempsJobs", "lang": "en", "location": "Menlo Park, CA", "create_at": date("2012-08-27"), "description": "Accountemps is the largest temporary staffing firm specializing in accounting and finance. See all their Jobs and openings right here.", "followers_count": 164, "friends_count": 7, "statues_count": 32793 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portsmouth, NH", "id": "fb3aafdc1555efcf", "name": "Portsmouth", "place_type": "city", "bounding_box": rectangle("-70.820242,43.013673 -70.737324,43.098556") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33015, "countyName": "Rockingham", "cityID": 3362900, "cityName": "Portsmouth" } }
+{ "create_at": datetime("2015-11-25T10:00:09.000Z"), "id": 669576320909643777, "text": "@MattFossen I'm mad with power!", "in_reply_to_status": 669576260125765633, "in_reply_to_user": 537347290, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 537347290 }}, "user": { "id": 104387738, "name": "Thom", "screen_name": "thomicks", "lang": "en", "location": "Modesto, CA", "create_at": date("2010-01-12"), "description": "I run a guild. I'm gonna teach maths. I do #BlizzardKeets and made that #FriendshipMoose thing. Banner by @Whammo_", "followers_count": 1450, "friends_count": 707, "statues_count": 20766 }, "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-25T10:00:09.000Z"), "id": 669576321073311745, "text": "Making r way back yay @ Jack Daniels Historic Site https://t.co/iAlCfuRMZe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.88294846,35.2530945"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 706840649, "name": "Amanda Hand", "screen_name": "amandaleehand", "lang": "en", "location": "clarksville TN", "create_at": date("2013-10-09"), "description": "http://instagram.com/luvlivelaughalways/ https://www.facebook.com/AmandaleeHensley", "followers_count": 1114, "friends_count": 1599, "statues_count": 2819 }, "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": 47055, "countyName": "Giles" } }
+{ "create_at": datetime("2015-11-25T10:00:09.000Z"), "id": 669576321186459648, "text": "@jimcramer #condosOfCoins", "in_reply_to_status": 669575547765317633, "in_reply_to_user": 14216123, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "condosOfCoins" }}, "user_mentions": {{ 14216123 }}, "user": { "id": 3247034023, "name": "~jamie ⚥", "screen_name": "transgenresJami", "lang": "en", "location": "null", "create_at": date("2015-06-16"), "description": "I was never taught to nurture; therefore, I cannot love women, nor man. I'm just trans... ~jamie", "followers_count": 75, "friends_count": 82, "statues_count": 5658 }, "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("2015-11-25T10:00:09.000Z"), "id": 669576321236729858, "text": "So what girl wants some extra thanks giving food and wants to come eat with my family to make me 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": 419804428, "name": "Steelers Nation", "screen_name": "_WesVG", "lang": "en", "location": "Oklahoma City, OK", "create_at": date("2011-11-23"), "description": "#God #SteelersNation #OU #OregonDucks #UNCTarHeels #GymLife #UCOstudent #21 I'm just trying to live the life God has given me and make the best of it", "followers_count": 2027, "friends_count": 1228, "statues_count": 70455 }, "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-25T10:00:09.000Z"), "id": 669576321283018752, "text": "Even the old people are rude. Are we still in the south or?? #cookeville", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "cookeville" }}, "user": { "id": 998674592, "name": "Kaitlyn Salyer", "screen_name": "ksalyer15", "lang": "en", "location": "null", "create_at": date("2012-12-08"), "description": "||She's a good girl, loves her momma. Loves Jesus & America, too.|| VA.", "followers_count": 401, "friends_count": 269, "statues_count": 9359 }, "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("2015-11-25T10:00:09.000Z"), "id": 669576321308069889, "text": "Thanksgiving goals https://t.co/avHP88CHBB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2457076460, "name": "〽️", "screen_name": "iramlozano5511", "lang": "en", "location": "null", "create_at": date("2014-04-21"), "description": "sometimes hate can be the best love", "followers_count": 149, "friends_count": 188, "statues_count": 4621 }, "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("2015-11-25T10:00:09.000Z"), "id": 669576321484238848, "text": "The Target Christmas section 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": 870805712, "name": "Ellen Graham", "screen_name": "EllenGraham17", "lang": "en", "location": "MTHS", "create_at": date("2012-10-09"), "description": "Stand tall, and above it all, fix my eyes on You #GoDeacs snapchat: ellsbellslovesy Exodus 14:14", "followers_count": 625, "friends_count": 740, "statues_count": 8273 }, "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("2015-11-25T10:00:09.000Z"), "id": 669576321547132928, "text": "#blackfamiliesonthanksgiving #HolidaysAreComing Listen to #SoulfulSeasons https://t.co/fw18U8kcQ7 @gotradio https://t.co/1oE8F4salL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "blackfamiliesonthanksgiving", "HolidaysAreComing", "SoulfulSeasons" }}, "user_mentions": {{ 75102545 }}, "user": { "id": 229918431, "name": "MARK KEENE", "screen_name": "KEENE2B", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2010-12-23"), "description": "Radio/Music/Video/Promotion/Marketing/Branding\n Programmer @ http://WWW.GOTRADIO.COM. I can't hear the critics talkin' over the applause.", "followers_count": 423, "friends_count": 1268, "statues_count": 490 }, "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("2015-11-25T10:00:09.000Z"), "id": 669576321735897088, "text": "@tommygirl_xxx @VxExx @VenusHotWife @VxErr @Drjohnson100 @DRM00RE @DrRXE @DrRXM Mira mi amor para tu bella panochita https://t.co/vNPLYLEbr4", "in_reply_to_status": 669199254230065152, "in_reply_to_user": 2898685710, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 2898685710, 3229432811, 3226149873, 3761437407, 1651150771, 3065660945, 3611260162, 2984893510 }}, "user": { "id": 3195665928, "name": "gabino rey", "screen_name": "ReyGabino", "lang": "en", "location": "null", "create_at": date("2015-05-14"), "description": "null", "followers_count": 388, "friends_count": 1359, "statues_count": 22974 }, "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("2015-11-25T10:00:09.000Z"), "id": 669576321832517632, "text": "@FinestMarie_ nope I'm in Alabama ����������������������������", "in_reply_to_status": 669576191083343872, "in_reply_to_user": 618779899, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 618779899 }}, "user": { "id": 2236677128, "name": "taarynn", "screen_name": "_itstarynn_", "lang": "en", "location": "null", "create_at": date("2013-12-08"), "description": "null", "followers_count": 206, "friends_count": 188, "statues_count": 622 }, "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("2015-11-25T10:00:09.000Z"), "id": 669576321920405504, "text": "Ok..... I could seriously go for a hot dog rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 346170808, "name": "Madi Morgan", "screen_name": "mvdimorgvn", "lang": "en", "location": "tempe", "create_at": date("2011-07-31"), "description": "ASU • Design • AΦ • ENTP ⠀⠀⠀ ⠀⠀⠀ ⠀⠀⠀ inspiration stems from love and stress compounding.", "followers_count": 1375, "friends_count": 936, "statues_count": 12161 }, "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("2015-11-25T10:00:09.000Z"), "id": 669576322168053760, "text": "I'm at escuela modelo https://t.co/4QT91TUbYB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.497639,25.876353"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2843072095, "name": "LMT", "screen_name": "ceub07", "lang": "en", "location": "null", "create_at": date("2014-10-06"), "description": "null", "followers_count": 11, "friends_count": 39, "statues_count": 753 }, "place": { "country": "México", "country_code": "México", "full_name": "Matamoros, Tamaulipas", "id": "3309acacf870f6f5", "name": "Matamoros", "place_type": "city", "bounding_box": rectangle("-97.946585,25.047842 -97.148199,26.070144") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4810768, "cityName": "Brownsville" } }
+{ "create_at": datetime("2015-11-25T10:00:10.000Z"), "id": 669576322461503488, "text": "Join the Kelly Services team! See our latest #Education #job opening here: https://t.co/t9xA0NY9X3 #KellyJobs #KellyServices #NewAlbany, IN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.8241312,38.2856247"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Education", "job", "KellyJobs", "KellyServices", "NewAlbany" }}, "user": { "id": 21838240, "name": "TMJ-SDF Educ. Jobs", "screen_name": "tmj_sdf_edu", "lang": "en", "location": "Louisville, KY", "create_at": date("2009-02-24"), "description": "Follow this account for geo-targeted Education/Teaching job tweets in Louisville, KY from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 278, "friends_count": 234, "statues_count": 30 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Albany, IN", "id": "ec5febdaca1534b0", "name": "New Albany", "place_type": "city", "bounding_box": rectangle("-85.884579,38.25708 -85.761144,38.39041") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18043, "countyName": "Floyd", "cityID": 1852326, "cityName": "New Albany" } }
+{ "create_at": datetime("2015-11-25T10:00:10.000Z"), "id": 669576322474049536, "text": "Bout to be https://t.co/xFmEupVkHQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1106281314, "name": "James", "screen_name": "jamesv_austin", "lang": "en", "location": "Corpus Christi, TX", "create_at": date("2013-01-20"), "description": "Aww Man", "followers_count": 736, "friends_count": 652, "statues_count": 17312 }, "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("2015-11-25T10:00:10.000Z"), "id": 669576322482499584, "text": "@rkenneth Absolutely one of my favorite shows ever", "in_reply_to_status": 669576119839010816, "in_reply_to_user": 15591974, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15591974 }}, "user": { "id": 28051036, "name": "Capt. Spike", "screen_name": "spikepoint", "lang": "en", "location": "SF Bay Area", "create_at": date("2009-03-31"), "description": "Things I Like: Gay stuff. Comic books. Gaming. Sports. Music. Art. Slasher films. Podcasting. see: @GAMEBUOYdotORG @ANIMEBUOYdotORG fka-suedeheadspike", "followers_count": 909, "friends_count": 913, "statues_count": 23100 }, "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-25T10:00:10.000Z"), "id": 669576322579066886, "text": "found this card on my car!!! #justbrookdalethings https://t.co/GJDTwBZFxb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "justbrookdalethings" }}, "user": { "id": 496534565, "name": "taylor dreuer", "screen_name": "ttaylor_dreuer", "lang": "en", "location": "wherever ", "create_at": date("2012-02-18"), "description": "don't hug me I'm scared", "followers_count": 596, "friends_count": 398, "statues_count": 13093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincroft, NJ", "id": "2e21a2ad07335fd5", "name": "Lincroft", "place_type": "city", "bounding_box": rectangle("-74.15049,40.317744 -74.098831,40.374445") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3440320, "cityName": "Lincroft" } }
+{ "create_at": datetime("2015-11-25T10:00:10.000Z"), "id": 669576322637656064, "text": "@JavierSotoTV ... thanks again! https://t.co/AcN4X6mKal", "in_reply_to_status": -1, "in_reply_to_user": 23713318, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23713318 }}, "user": { "id": 23871990, "name": "Phx Salvation Army", "screen_name": "PhxSalArmy", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2009-03-11"), "description": "Doing The Most Good in the Valley of the Sun.", "followers_count": 1122, "friends_count": 763, "statues_count": 1618 }, "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-25T10:00:10.000Z"), "id": 669576322662821889, "text": "@TheRealKamie ��������", "in_reply_to_status": 669557595175313408, "in_reply_to_user": 280334933, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 280334933 }}, "user": { "id": 66184910, "name": "Joshua Acosta", "screen_name": "AcostaTraining", "lang": "en", "location": "null", "create_at": date("2009-08-16"), "description": "~FItness is Life~Enjoy the Little Things~Never Give up~Dream Big~", "followers_count": 276, "friends_count": 193, "statues_count": 5421 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sammamish, WA", "id": "21e20d6f2dd2e43a", "name": "Sammamish", "place_type": "city", "bounding_box": rectangle("-122.098847,47.558804 -121.971291,47.657437") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5361115, "cityName": "Sammamish" } }
+{ "create_at": datetime("2015-11-25T10:00:10.000Z"), "id": 669576322704740352, "text": "Nm just hanging out with a former dunk contest champion u?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17417412, "name": "Matticus Finch", "screen_name": "mattravis", "lang": "en", "location": "hinterland (okc/norman, ok)", "create_at": date("2008-11-15"), "description": "betting the under on the inevitable heat death of the universe", "followers_count": 411, "friends_count": 450, "statues_count": 21813 }, "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-25T10:00:10.000Z"), "id": 669576322759262208, "text": "I'm in such a good mood ☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 842685553, "name": "Kat", "screen_name": "alyashley_", "lang": "en", "location": "null", "create_at": date("2012-09-23"), "description": "Carpé dïem☼", "followers_count": 427, "friends_count": 198, "statues_count": 5574 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-25T10:00:10.000Z"), "id": 669576322818031616, "text": "glad I didn't go to school bc if I did I would probably still be a wreck", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3226375258, "name": ".", "screen_name": "taleenaasf", "lang": "en", "location": "Hopewell, VA", "create_at": date("2015-05-01"), "description": "HHS'17. model. pequot nation bih. cape verdean. 860 804. aquarius af. turtles & goats are cute. i also eat my rice & cabbage :-)", "followers_count": 387, "friends_count": 229, "statues_count": 16365 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopewell, VA", "id": "137fd4a4fbf74e6f", "name": "Hopewell", "place_type": "city", "bounding_box": rectangle("-77.337993,37.257957 -77.269756,37.317666") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51149, "countyName": "Prince George", "cityID": 5138424, "cityName": "Hopewell" } }
+{ "create_at": datetime("2015-11-25T10:00:10.000Z"), "id": 669576322834919424, "text": "@sarahszoke @ahalstead122 miss you more ��", "in_reply_to_status": 669523598181343232, "in_reply_to_user": 3064964902, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3064964902, 2210621429 }}, "user": { "id": 470696549, "name": "Marissa Speroni", "screen_name": "SperoniMarissa", "lang": "en", "location": "null", "create_at": date("2012-01-21"), "description": "Rowan University '18 • Cheerleader", "followers_count": 262, "friends_count": 123, "statues_count": 1395 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin, NJ", "id": "40bbc662bdb14422", "name": "Franklin", "place_type": "city", "bounding_box": rectangle("-74.610114,41.084879 -74.569938,41.138186") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34037, "countyName": "Sussex", "cityID": 3424930, "cityName": "Franklin" } }
+{ "create_at": datetime("2015-11-25T10:00:10.000Z"), "id": 669576322948186112, "text": "@_LitBish I'm tired of y'all hating on turkey.", "in_reply_to_status": 669371096987262976, "in_reply_to_user": 2400953917, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2400953917 }}, "user": { "id": 1884871526, "name": "Joshua Newton.", "screen_name": "JoshuaThotties", "lang": "en", "location": "The Wu", "create_at": date("2013-09-19"), "description": "Buffalo Bill. #HennBoys", "followers_count": 1451, "friends_count": 1013, "statues_count": 85737 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Springdale, OH", "id": "2eb3732437b18323", "name": "Springdale", "place_type": "city", "bounding_box": rectangle("-84.513022,39.269985 -84.447348,39.305817") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3974104, "cityName": "Springdale" } }
+{ "create_at": datetime("2015-11-25T10:00:10.000Z"), "id": 669576323002580992, "text": "We're #hiring! Read about our latest #job opening here: General Warehouse - https://t.co/KqDuTXea5i #KAPOLEI, HI #Labor #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-158.056892,21.3354034"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "KAPOLEI", "Labor", "CareerArc" }}, "user": { "id": 118676555, "name": "TMJ-HI Labor Jobs", "screen_name": "tmj_HI_LABOR", "lang": "en", "location": "Hawaii", "create_at": date("2010-03-01"), "description": "Follow this account for geo-targeted General Labor job tweets in Hawaii Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 284, "friends_count": 279, "statues_count": 8 }, "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("2015-11-25T10:00:10.000Z"), "id": 669576323132694529, "text": "@toriefitzgerald you act like I won't sing that to you next time we ride in the car???? but ok tonight", "in_reply_to_status": 669565566248738817, "in_reply_to_user": 314064321, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 314064321 }}, "user": { "id": 370341165, "name": "Macy Youngblood", "screen_name": "MJoYoungblood11", "lang": "en", "location": "The South", "create_at": date("2011-09-08"), "description": "•Lover of The Lord and all His people•", "followers_count": 1050, "friends_count": 641, "statues_count": 14269 }, "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("2015-11-25T10:00:10.000Z"), "id": 669576323250155520, "text": "@YourFriendBAP Similar pattern in other recent videos. I'd guess they're trained to assume there's still a threat until some point. When?", "in_reply_to_status": 669576013043531776, "in_reply_to_user": 136348259, "favorite_count": 0, "coordinate": point("-86.15201986,39.79516299"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 136348259 }}, "user": { "id": 5152561, "name": "Andrew (Andy) M.", "screen_name": "morinap", "lang": "en", "location": "Herron-Morton, Indianapolis", "create_at": date("2007-04-18"), "description": "Software developer and sports nut - Indianapolis, via Cincinnati. I love baseball, motorsports, computers & the Oxford comma. Firmly believe in Hanlon's razor.", "followers_count": 841, "friends_count": 852, "statues_count": 63246 }, "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("2015-11-25T10:00:10.000Z"), "id": 669576323426328576, "text": "@iridianmorax33 IM WEAK", "in_reply_to_status": 669576081201086464, "in_reply_to_user": 1177616011, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1177616011 }}, "user": { "id": 2413442203, "name": "carl 10-0", "screen_name": "carlDYNASTY", "lang": "en", "location": "Carldynasty on instagram", "create_at": date("2014-03-26"), "description": "@iridianmorax33 ❤️", "followers_count": 335, "friends_count": 319, "statues_count": 5929 }, "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("2015-11-25T10:00:10.000Z"), "id": 669576323606515712, "text": "@HamsLlyod @34jaIIen isn't it obvious", "in_reply_to_status": 669576205075668992, "in_reply_to_user": 1393871011, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1393871011, 2790664419 }}, "user": { "id": 1952394228, "name": "vladimir tarablakeo", "screen_name": "ShattInMyKirk", "lang": "en", "location": "can into texas", "create_at": date("2013-10-10"), "description": "@STLouisRams & @Panthers | @Cardinals & @BlueJays | @StLouisBlues & @NHLCanes | @dallasmavs | @FCDallas | @MizzouAthletics |#txhsfb | milk | dank memes", "followers_count": 437, "friends_count": 1272, "statues_count": 91918 }, "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.533978 -97.818437,33.590943") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48337, "countyName": "Montague", "cityID": 4809640, "cityName": "Bowie" } }
+{ "create_at": datetime("2015-11-25T10:00:10.000Z"), "id": 669576323728207872, "text": "We're #hiring! Read about our latest #job opening here: RN - Forensic Nurse Examiner - https://t.co/OYmVevt5gF #MedField #Kenai, AK #Nursing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-151.2583333,60.5544444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "MedField", "Kenai", "Nursing" }}, "user": { "id": 3278999924, "name": "CPH Jobs", "screen_name": "CPHJobs", "lang": "en", "location": "Soldotna, AK", "create_at": date("2015-07-13"), "description": "At CPH every employee is considered a caregiver, an integral part of our health care team. Follow us to learn more!", "followers_count": 3, "friends_count": 0, "statues_count": 50 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kenai, AK", "id": "01e900ea6bad2940", "name": "Kenai", "place_type": "city", "bounding_box": rectangle("-151.320994,60.541958 -151.174897,60.588949") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2122, "countyName": "Kenai Peninsula", "cityID": 238420, "cityName": "Kenai" } }
+{ "create_at": datetime("2015-11-25T10:00:10.000Z"), "id": 669576324248285184, "text": "Can you recommend anyone for this #job? Registered Nurse (RN)-Intermediate Critical Care Unit - https://t.co/QMw9niRKou #nurse #Lubbock, TX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-101.8919288,33.5881366"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "nurse", "Lubbock" }}, "user": { "id": 2940007148, "name": "UMC Nursing Jobs", "screen_name": "UMCNursingJobs", "lang": "en", "location": "null", "create_at": date("2014-12-22"), "description": "UMC Health System is the leader in comprehensive healthcare delivery in West Texas and Eastern New Mexico. Follow us to learn about our Nursing Jobs!", "followers_count": 81, "friends_count": 279, "statues_count": 61 }, "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("2015-11-25T10:00:10.000Z"), "id": 669576324483149824, "text": "See our latest #SanAntonio, TX #job and click to apply: Cantonese Interpreters - https://t.co/kkGeowVowk #interpreter #bilingual #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.4936282,29.4241219"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SanAntonio", "job", "interpreter", "bilingual", "Hiring" }}, "user": { "id": 71945350, "name": "TMJ-SAT CstSrv Jobs", "screen_name": "tmj_sat_cstsrv", "lang": "en", "location": "San Antonio, TX", "create_at": date("2009-09-05"), "description": "Follow this account for geo-targeted Customer Service job tweets in San Antonio, TX from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 309, "friends_count": 281, "statues_count": 129 }, "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-25T10:00:10.000Z"), "id": 669576324634152961, "text": "Cloudy this afternoon, high 69 (21 C). Low 62 (17 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712452, "name": "FortWorthWX", "screen_name": "FortWorthWX", "lang": "en", "location": "Fort Worth, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Fort Worth, Texas", "followers_count": 208, "friends_count": 93, "statues_count": 8060 }, "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("2015-11-25T10:00:10.000Z"), "id": 669576324663504896, "text": "Lmao ok RT @FreebandzAjani: Drake didn't put Migos on \nDrake didn't put Migos on\nDrake didn't put Migos on\nDrake didn't put Migos on", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.189178,29.719899"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3040638716 }}, "user": { "id": 41233958, "name": "Man of honor", "screen_name": "cubaescobar", "lang": "en", "location": "Clutch city", "create_at": date("2009-05-19"), "description": "27, construction civil certified chemical plant ,(tech genius) in school .. Sometimes I ain't shit but you get the picture Grown man tho", "followers_count": 2591, "friends_count": 981, "statues_count": 114014 }, "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("2015-11-25T10:00:10.000Z"), "id": 669576324667858945, "text": "Me: \"Chances you get a to go box?\"\n@patricktomasic: \"No\"\nWaitress: \"You want a to go box?\"\nPatrick: \"Ugh. Yeah, I'll take one\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 577606803 }}, "user": { "id": 2255308818, "name": "Thomas Harrell", "screen_name": "ThomasHarrell18", "lang": "en", "location": "Durham, NC", "create_at": date("2013-12-20"), "description": "#NCSU19 Snapchat/IG: tharrell1414", "followers_count": 310, "friends_count": 153, "statues_count": 1108 }, "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("2015-11-25T10:00:10.000Z"), "id": 669576324692840448, "text": "Interested in a #Sales #job near #Towson, MD? This could be a great fit: https://t.co/Udoq3VA5T2 #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.6012561,39.4037717"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "job", "Towson", "Hiring", "CareerArc" }}, "user": { "id": 21221464, "name": "TMJ - BAL Sales Jobs", "screen_name": "tmj_bal_sales", "lang": "en", "location": "Baltimore, MD", "create_at": date("2009-02-18"), "description": "Follow this account for geo-targeted Sales job tweets in Baltimore, MD. Need help? Tweet us at @CareerArc!", "followers_count": 484, "friends_count": 310, "statues_count": 268 }, "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("2015-11-25T10:00:10.000Z"), "id": 669576324722196481, "text": "Interested in a #Hospitality #job near ? This could be a great fit: https://t.co/6qLfUfLIqO #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.7525113,34.4403725"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 99812776, "name": "TMJ-CAA HRTA Jobs", "screen_name": "tmj_CAA_HRTA", "lang": "en", "location": "Santa Barbara, CA", "create_at": date("2009-12-27"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Santa Barbara, CA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 278, "friends_count": 268, "statues_count": 45 }, "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("2015-11-25T10:00:10.000Z"), "id": 669576324969721856, "text": "#CareerArc #Transportation #Job alert: Driver Helper | UPS | #CRITTENDEN, KY https://t.co/OshvMvs3oh #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.6052206,38.7828437"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CareerArc", "Transportation", "Job", "CRITTENDEN", "Jobs", "Hiring" }}, "user": { "id": 149001192, "name": "TMJ-KY Transport.", "screen_name": "tmj_KY_transp", "lang": "en", "location": "Kentucky", "create_at": date("2010-05-27"), "description": "Follow this account for geo-targeted Transportation job tweets in Kentucky Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 310, "friends_count": 280, "statues_count": 351 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crittenden, KY", "id": "b811e697ae8e7f99", "name": "Crittenden", "place_type": "city", "bounding_box": rectangle("-84.627238,38.751956 -84.585816,38.803449") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21081, "countyName": "Grant", "cityID": 2118568, "cityName": "Crittenden" } }
+{ "create_at": datetime("2015-11-25T10:00:10.000Z"), "id": 669576325129109504, "text": "For the first time in like a year and a half the whole family's together! ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 377673155, "name": "AnnaClaire", "screen_name": "AnnaClaireLaush", "lang": "en", "location": "Seattle", "create_at": date("2011-09-21"), "description": "And with God's grace enjoy the view and look out to endless seas\n IG:@annaclairelaush", "followers_count": 233, "friends_count": 225, "statues_count": 6533 }, "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("2015-11-25T10:00:10.000Z"), "id": 669576325179412480, "text": "I'm a argumentive 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": 391789178, "name": "lilTip Toe", "screen_name": "LitTf_Up", "lang": "en", "location": "HTX", "create_at": date("2011-10-15"), "description": "Do better.", "followers_count": 747, "friends_count": 564, "statues_count": 27703 }, "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("2015-11-25T10:00:10.000Z"), "id": 669576325292625920, "text": "Lemme keep that emotion shit off Twitter tho ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 299462605, "name": "☄©®➰➰K☄", "screen_name": "ClaytonCrook", "lang": "en", "location": "☄MO CITY TX☄", "create_at": date("2011-05-15"), "description": "✨The Last Mansa ✨ HTX✈️CoMo", "followers_count": 1592, "friends_count": 940, "statues_count": 8671 }, "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("2015-11-25T10:00:10.000Z"), "id": 669576325376544768, "text": "@uce_treyy trip me out then this nigga will turn up missing ¯\\_(ツ)_/¯ I don't play that shit", "in_reply_to_status": 669575757643935745, "in_reply_to_user": 3526895412, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3526895412 }}, "user": { "id": 1544910768, "name": "Eseneiaso", "screen_name": "anumeaa", "lang": "en", "location": "76 $outh", "create_at": date("2013-06-24"), "description": "5'2 with an attitude. Kae ✌️| @TsuKneeAhh is hott", "followers_count": 637, "friends_count": 491, "statues_count": 9519 }, "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("2015-11-25T10:00:10.000Z"), "id": 669576325397508096, "text": "Omg this isn't real I'm still 5 https://t.co/gwl2FjntMK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2909849012, "name": "Santa Kaley", "screen_name": "Kaley_Caveman", "lang": "en", "location": "SMA Texas", "create_at": date("2014-11-25"), "description": "Dont listen to Janelle my name is not Katnip", "followers_count": 183, "friends_count": 138, "statues_count": 1900 }, "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("2015-11-25T10:00:10.000Z"), "id": 669576325598875648, "text": "Can you recommend anyone for this #Nursing #job? https://t.co/SyWotwDEMX #WestSaintPaul, MN #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.1629063,44.9330076"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "WestSaintPaul", "Hiring", "CareerArc" }}, "user": { "id": 22023253, "name": "MN Nursing Jobs", "screen_name": "tmj_mn_nursing", "lang": "en", "location": "Minnesota", "create_at": date("2009-02-26"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Minnesota Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 555, "friends_count": 311, "statues_count": 121 }, "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.89076 -93.003514,44.992279") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27123, "countyName": "Ramsey", "cityID": 2758000, "cityName": "St. Paul" } }
+{ "create_at": datetime("2015-11-25T10:00:10.000Z"), "id": 669576325653368832, "text": "@FrostoWasTaken stoooop ��", "in_reply_to_status": 669576013160931328, "in_reply_to_user": 2981405306, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2981405306 }}, "user": { "id": 326567281, "name": "Savannah✨", "screen_name": "saaaaaavv", "lang": "en", "location": "Raccoon City", "create_at": date("2011-06-29"), "description": "MKM.", "followers_count": 402, "friends_count": 265, "statues_count": 20218 }, "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("2015-11-25T10:00:10.000Z"), "id": 669576325758255104, "text": "I'm down to get loss in that sauce. https://t.co/HNy36Q2SkH", "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": "$croogeMcBuck$", "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": 1318, "friends_count": 806, "statues_count": 44179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:10.000Z"), "id": 669576325951307776, "text": "art museums 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": 1519813435, "name": "lana del kay", "screen_name": "kaylene808", "lang": "en", "location": "han", "create_at": date("2013-06-15"), "description": "I'm listening to tøp , gtg . |-/", "followers_count": 444, "friends_count": 377, "statues_count": 6279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crooksville, OH", "id": "ab084c8a4e546b9c", "name": "Crooksville", "place_type": "city", "bounding_box": rectangle("-82.109943,39.739259 -82.079046,39.784193") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39127, "countyName": "Perry", "cityID": 3919456, "cityName": "Crooksville" } }
+{ "create_at": datetime("2015-11-25T10:00:10.000Z"), "id": 669576325980532736, "text": "Hbd to a girl who I couldn't live without❤️ I'm soo thankful to have the friendship we have! Cheers to many more�� https://t.co/eZvI1lDU7j", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 886052011, "name": "Torrrr", "screen_name": "VacaTori", "lang": "en", "location": "null", "create_at": date("2012-10-16"), "description": "now a days I walk over things I used to trip on❣", "followers_count": 408, "friends_count": 214, "statues_count": 1193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakeside, CA", "id": "03c9c75ede037d93", "name": "Lakeside", "place_type": "city", "bounding_box": rectangle("-116.948624,32.826697 -116.872927,32.872074") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 639766, "cityName": "Lakeside" } }
+{ "create_at": datetime("2015-11-25T10:00:10.000Z"), "id": 669576326114865152, "text": "Plant Mentality 2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 148718447, "name": "Fuck Gerwig", "screen_name": "Gerwig_", "lang": "en", "location": "Trenton, MI", "create_at": date("2010-05-27"), "description": "Lungs full of tar & mah tongue out mah mouth.", "followers_count": 654, "friends_count": 379, "statues_count": 32673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Trenton, MI", "id": "3c4cb62907491ed8", "name": "Trenton", "place_type": "city", "bounding_box": rectangle("-83.218644,42.111921 -83.165338,42.170748") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2680420, "cityName": "Trenton" } }
+{ "create_at": datetime("2015-11-25T10:00:10.000Z"), "id": 669576326227955712, "text": "#Indianapolis, IN #Retail #Job: Retail Store Positions at CVS Health https://t.co/zqCLgJp8cV #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.1816304,39.6635277"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Indianapolis", "Retail", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 28542508, "name": "TMJ-IND Retail Jobs", "screen_name": "tmj_ind_retail", "lang": "en", "location": "Indianapolis, IN", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Indianapolis, IN. Need help? Tweet us at @CareerArc!", "followers_count": 383, "friends_count": 310, "statues_count": 877 }, "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("2015-11-25T10:00:10.000Z"), "id": 669576326295060480, "text": "You need to stop.. https://t.co/USATbHber5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 547565810, "name": "♠️Finesse God♦️", "screen_name": "byron_weaver", "lang": "en", "location": "null", "create_at": date("2012-04-07"), "description": "ASU '18", "followers_count": 2421, "friends_count": 2204, "statues_count": 8994 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paloma Creek South, TX", "id": "003ef58a3a36eab7", "name": "Paloma Creek South", "place_type": "city", "bounding_box": rectangle("-96.954509,33.202712 -96.925277,33.221018") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4854872, "cityName": "Paloma Creek South" } }
+{ "create_at": datetime("2015-11-25T10:00:10.000Z"), "id": 669576326324490240, "text": "What's your point cuz https://t.co/LH6midPwmU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3251189882, "name": "Strive.Prosper.", "screen_name": "OgAssNiggaMatt", "lang": "en", "location": "Houston, TX", "create_at": date("2015-06-20"), "description": "RMBG the Squad.\nXanGang Ent.\nBeen Fly Crippin.\nOriginally Abnormal.\n\n007$hit.", "followers_count": 96, "friends_count": 137, "statues_count": 1260 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:11.000Z"), "id": 669576326504820736, "text": "What a beautiful day for suffering!.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 23295113, "name": "Scoopy Dü", "screen_name": "codyhowell", "lang": "en", "location": "Bozeman, Montana", "create_at": date("2009-03-08"), "description": "I haven't caught boogie fever yet.", "followers_count": 199, "friends_count": 236, "statues_count": 4698 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Helena, MT", "id": "9c7ce7eed4cb3dee", "name": "Helena", "place_type": "city", "bounding_box": rectangle("-112.106268,46.568875 -111.935822,46.626111") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30049, "countyName": "Lewis and Clark", "cityID": 3035600, "cityName": "Helena" } }
+{ "create_at": datetime("2015-11-25T10:00:11.000Z"), "id": 669576326517555204, "text": "@YungHeeHaw does it begin with punching the monitor? That's what I was thinking", "in_reply_to_status": 669575386133581824, "in_reply_to_user": 4265385233, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4265385233 }}, "user": { "id": 25217855, "name": "CRUNK FRIDAY SALE", "screen_name": "packman_jon", "lang": "en", "location": "rob ford's house of crunk", "create_at": date("2009-03-18"), "description": "Not Rob Ford. Proud member of #GARBAGETWITTER, president of #CRUNKTWITTER. Not to be taken too seriously. #IsJoeFlaccoElite?\n1-800-GET-CRUNK", "followers_count": 470, "friends_count": 386, "statues_count": 20592 }, "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("2015-11-25T10:00:11.000Z"), "id": 669576326529970177, "text": "All those aunt jokes are so relevant to my aunt to be honest lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2653973968, "name": "Vanny", "screen_name": "VaneeVanny", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2014-06-29"), "description": "c/o '17", "followers_count": 171, "friends_count": 240, "statues_count": 3569 }, "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("2015-11-25T10:00:11.000Z"), "id": 669576326618021889, "text": "But I'm pretty good at drinking beer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1308573090, "name": "Teagan", "screen_name": "TeaganAzbill", "lang": "en", "location": "Midvale, ID", "create_at": date("2013-03-27"), "description": "As the great Colonel Sanders said, I'm to drunk to taste this chicken #CowboysNation", "followers_count": 508, "friends_count": 334, "statues_count": 2731 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weiser, ID", "id": "02713e0aac36ce6b", "name": "Weiser", "place_type": "city", "bounding_box": rectangle("-116.993673,44.238728 -116.942731,44.267874") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16087, "countyName": "Washington", "cityID": 1686140, "cityName": "Weiser" } }
+{ "create_at": datetime("2015-11-25T10:00:11.000Z"), "id": 669576326710456320, "text": "I named him Paul and he loves croutons", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2183521093, "name": "Cass", "screen_name": "cassidy_vinson", "lang": "en", "location": "null", "create_at": date("2013-11-08"), "description": "http://crinkled-thoughts.tumblr.com", "followers_count": 124, "friends_count": 120, "statues_count": 1904 }, "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("2015-11-25T10:00:11.000Z"), "id": 669576327226195968, "text": "Cloudy this afternoon, high 70 (21 C). Low 62 (17 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712424, "name": "DallasWX", "screen_name": "DallasWX", "lang": "en", "location": "Dallas, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Dallas, Texas", "followers_count": 191, "friends_count": 93, "statues_count": 8037 }, "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-25T10:00:11.000Z"), "id": 669576327268311044, "text": "@ldemarcofox5 @lauraevansfox5 @SimmonsFox5 @suepalkafox5dc @AlexandraFox5DC Thanks Lauren! Hope you have a wonderful day tomorrow :-)", "in_reply_to_status": 669574264018903040, "in_reply_to_user": 303394187, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 303394187, 17059095, 19145536, 1426265922, 613491892 }}, "user": { "id": 31444949, "name": "Jason L", "screen_name": "NFLFan1991", "lang": "en", "location": "Lewes, Delaware", "create_at": date("2009-04-15"), "description": "big sports fan!! like the @redskins @FSU_Football Followed by @MackPhillips 9 -15-12", "followers_count": 422, "friends_count": 1124, "statues_count": 46000 }, "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("2015-11-25T10:00:11.000Z"), "id": 669576327373058048, "text": "man I should be skateboarding more often", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1596751022, "name": "Luke Delhomme", "screen_name": "delhomie", "lang": "en", "location": "null", "create_at": date("2013-07-15"), "description": "null", "followers_count": 80, "friends_count": 74, "statues_count": 175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Taylor Lake Village, TX", "id": "70651817971f9cb8", "name": "Taylor Lake Village", "place_type": "city", "bounding_box": rectangle("-95.071853,29.562882 -95.043298,29.591965") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4871960, "cityName": "Taylor Lake Village" } }
+{ "create_at": datetime("2015-11-25T10:00:11.000Z"), "id": 669576327381389312, "text": "�� https://t.co/tcOEpPG0hE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "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": 448, "friends_count": 401, "statues_count": 2702 }, "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-25T10:00:11.000Z"), "id": 669576327708700673, "text": "Would you prefer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3007743536, "name": "〰", "screen_name": "akmh_", "lang": "en", "location": "null", "create_at": date("2015-02-02"), "description": "الحمدالله على كل حال", "followers_count": 184, "friends_count": 121, "statues_count": 2311 }, "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("2015-11-25T10:00:11.000Z"), "id": 669576327721152512, "text": "@JprinceTHEGREAT ��", "in_reply_to_status": 669576270389231616, "in_reply_to_user": 312882032, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 312882032 }}, "user": { "id": 3148502268, "name": "ttttttttia.", "screen_name": "ResiDimples", "lang": "en", "location": "South Africa Dallas Houston.", "create_at": date("2015-04-10"), "description": "man-eater.", "followers_count": 879, "friends_count": 551, "statues_count": 12804 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:11.000Z"), "id": 669576327964565504, "text": "Really wished we had our own wrestling room ........", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1411089782, "name": "Triple A", "screen_name": "tripleabdullahi", "lang": "en", "location": "null", "create_at": date("2013-05-07"), "description": "614 Westerville South Wrestling.", "followers_count": 316, "friends_count": 285, "statues_count": 1755 }, "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("2015-11-25T10:00:11.000Z"), "id": 669576328098770946, "text": "@sportslogosnet I'm a Tribe fan, but man do their uniforms need a redesign. I'd like to see them change back to the Spiders, actually", "in_reply_to_status": 669573700652572672, "in_reply_to_user": 135947004, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 135947004 }}, "user": { "id": 172621113, "name": "Mike Cline Jr.", "screen_name": "MikeClineJr", "lang": "en", "location": "Hill Valley, California", "create_at": date("2010-07-29"), "description": "I am the chill that's in the air.", "followers_count": 441, "friends_count": 267, "statues_count": 11904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orchard Park, NY", "id": "00f0112125d013dc", "name": "Orchard Park", "place_type": "city", "bounding_box": rectangle("-78.797966,42.724114 -78.696772,42.811289") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3655266, "cityName": "Orchard Park" } }
+{ "create_at": datetime("2015-11-25T10:00:11.000Z"), "id": 669576328178327552, "text": "the allergy place is really upsetting me all I want is to get my shots so that I can not feel like my face is falling off. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 731944266, "name": "Taylor Evans", "screen_name": "taylorevansss97", "lang": "en", "location": "Lubbock, TX", "create_at": date("2012-08-01"), "description": "die with memories, not dreams", "followers_count": 473, "friends_count": 462, "statues_count": 2323 }, "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("2015-11-25T10:00:11.000Z"), "id": 669576328182693889, "text": "@MattBasically bless your trip", "in_reply_to_status": 669575271607988224, "in_reply_to_user": 123644513, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 123644513 }}, "user": { "id": 296106539, "name": "Timothy Huneycutt", "screen_name": "Timotheous128", "lang": "en", "location": "Dallas", "create_at": date("2011-05-09"), "description": "Believer in the Oxford Comma + SEO Coordinator over at @DealerOn + Writer + Tenor + Feminist + Nerd", "followers_count": 521, "friends_count": 423, "statues_count": 23490 }, "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("2015-11-25T10:00:11.000Z"), "id": 669576328249606144, "text": "Robert Parry's latest book:The Disastrous Presidency of George W. Bush\n\nWho drank like fish when young and turned godly when old.Evangelist", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2719172970, "name": "esmaiil fotoohi", "screen_name": "1318mef", "lang": "en", "location": "null", "create_at": date("2014-08-09"), "description": "null", "followers_count": 541, "friends_count": 537, "statues_count": 21465 }, "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("2015-11-25T10:00:11.000Z"), "id": 669576328396451840, "text": "That moment when your check with all the overtime shows up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30825983, "name": "Ratchet Fox", "screen_name": "Ratchet_the_fox", "lang": "en", "location": "Euless, TX", "create_at": date("2009-04-13"), "description": "Hey, I'm Ratchet, A furry techie living in Euless, TX My life is pretty much AV 24-7 I work for an AV Company in DFW, I also manage @furryfiesta's AV department", "followers_count": 1185, "friends_count": 816, "statues_count": 14371 }, "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-25T10:00:11.000Z"), "id": 669576328472055809, "text": "\"I wish I could major in parallel parking\" -@kaylacalvillo91", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2973343118 }}, "user": { "id": 1963394808, "name": "Alex Winkle", "screen_name": "bowtiesbandages", "lang": "en", "location": "Columbus Ohio", "create_at": date("2013-10-15"), "description": "Superman is doing good; you may or may not be doing well.", "followers_count": 85, "friends_count": 82, "statues_count": 320 }, "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("2015-11-25T10:00:11.000Z"), "id": 669576328585326592, "text": "Words can't describe how much I dislike fluffing the Christmas tree! #openlacerationsmyarms", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "openlacerationsmyarms" }}, "user": { "id": 469568705, "name": "Nicki Drinane", "screen_name": "Nicki_Drinane", "lang": "en", "location": "Detroit, Michigan", "create_at": date("2012-01-20"), "description": "Don't be afraid to fail. Be afraid not to try.", "followers_count": 116, "friends_count": 125, "statues_count": 733 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverview, MI", "id": "4dba0f986b3262ee", "name": "Riverview", "place_type": "city", "bounding_box": rectangle("-83.21789,42.155281 -83.154223,42.185302") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2668880, "cityName": "Riverview" } }
+{ "create_at": datetime("2015-11-25T10:00:11.000Z"), "id": 669576328606187520, "text": "bitch I'm diving in 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": 3376463052, "name": "blasé ☁️", "screen_name": "AshlanFields", "lang": "en", "location": "null", "create_at": date("2015-08-28"), "description": "null", "followers_count": 463, "friends_count": 407, "statues_count": 4759 }, "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("2015-11-25T10:00:11.000Z"), "id": 669576329084444673, "text": "hbd katie love you lots ❤️❤️ https://t.co/J8kECoGseA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1380209648, "name": "taylor bair", "screen_name": "little_bair", "lang": "en", "location": "null", "create_at": date("2013-04-25"), "description": "brhs 2017", "followers_count": 398, "friends_count": 298, "statues_count": 1846 }, "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-25T10:00:11.000Z"), "id": 669576329105383425, "text": "On some real shit✔️ https://t.co/OnzaFdUn6j", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3203898375, "name": "Reemm", "screen_name": "_reemiee23", "lang": "en", "location": "null", "create_at": date("2015-04-24"), "description": "better days to come", "followers_count": 262, "friends_count": 197, "statues_count": 1385 }, "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("2015-11-25T10:00:11.000Z"), "id": 669576329168207873, "text": "OfficeMax: Sales Consultant Print Services (#Kahului, HI) https://t.co/XdxKI6gj8Q #Veterans #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-156.454708,20.881571"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "Kahului", "Veterans", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 38082277, "name": "TMJ-MAU Jobs", "screen_name": "tmj_mau_jobs", "lang": "en", "location": "Maui, HI", "create_at": date("2009-05-05"), "description": "Follow this account for geo-targeted Other job tweets in Maui, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 545, "friends_count": 535, "statues_count": 15 }, "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("2015-11-25T10:00:11.000Z"), "id": 669576329373683712, "text": "See our latest #ElCajon, CA #job and click to apply: Looking for Receptionist to Grow with Company! - https://t.co/juXI4NYtHB #Clerical", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-116.9675814,32.7963222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ElCajon", "job", "Clerical" }}, "user": { "id": 23018727, "name": "TMJ-SAN Cleric. Jobs", "screen_name": "tmj_san_cler", "lang": "en", "location": "San Diego, CA", "create_at": date("2009-03-05"), "description": "Follow this account for geo-targeted Office/Clerical job tweets in San Diego, CA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 297, "friends_count": 200, "statues_count": 131 }, "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("2015-11-25T10:00:11.000Z"), "id": 669576329373724673, "text": "i wish https://t.co/oNT0fU1nHt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 734886686, "name": "lisa", "screen_name": "x_lisanicole", "lang": "en", "location": "null", "create_at": date("2012-08-03"), "description": "battle's", "followers_count": 1939, "friends_count": 980, "statues_count": 6101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:11.000Z"), "id": 669576329411457024, "text": "Can you recommend anyone for this #job? Accounting Manager/Supervisor - https://t.co/E0g0GKDf48 #Houston, TX #Accounting #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.2632201,29.7597385"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Houston", "Accounting", "Hiring", "CareerArc" }}, "user": { "id": 23240185, "name": "TMJ-HOU Acct. Jobs", "screen_name": "tmj_hou_acct", "lang": "en", "location": "Houston, TX", "create_at": date("2009-03-07"), "description": "Follow this account for geo-targeted Accounting job tweets in Houston, TX. Need help? Tweet us at @CareerArc!", "followers_count": 381, "friends_count": 301, "statues_count": 255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:11.000Z"), "id": 669576329524674560, "text": "This #Healthcare #job might be a great fit for you: PHARMACY TECHNICIAN - https://t.co/9uRv4BFI3y #Honolulu, HI #Veterans #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "Honolulu", "Veterans", "Hiring" }}, "user": { "id": 23526678, "name": "TMJ- HON Health Jobs", "screen_name": "tmj_hon_health", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Healthcare job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 401, "friends_count": 304, "statues_count": 171 }, "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("2015-11-25T10:00:11.000Z"), "id": 669576330145619968, "text": "I'm so fucking jealous but it's lowkey so stfu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 963296743, "name": "Lexi", "screen_name": "lexiwat", "lang": "en", "location": "null", "create_at": date("2012-11-21"), "description": "snapchat: lexiwat", "followers_count": 421, "friends_count": 237, "statues_count": 15657 }, "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("2015-11-25T10:00:11.000Z"), "id": 669576330338566144, "text": "It's been less than 24hrs and imy already lol @ Dutchess County https://t.co/oEcBRY8m4m", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.8997116,41.6942711"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388470964, "name": "meg", "screen_name": "MeghanSlinskey", "lang": "en", "location": "ŪSÄ", "create_at": date("2011-10-10"), "description": "ass AND titties #relatabletweet LOURDES '15 ASSUMPTION'19 XCTF", "followers_count": 213, "friends_count": 270, "statues_count": 1993 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, NY", "id": "10c62355e8155e86", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-73.90861,41.666874 -73.839299,41.73657") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36027, "countyName": "Dutchess", "cityID": 3602616, "cityName": "Arlington" } }
+{ "create_at": datetime("2015-11-25T10:00:12.000Z"), "id": 669576330623717377, "text": "There's so many pretty girls here it's insane. I've fallen in love at least 200 times already", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 456499387, "name": "IG: d.x.t.r", "screen_name": "lybdexter", "lang": "en", "location": "Kauai, Hawaii", "create_at": date("2012-01-06"), "description": "23. Kauai.", "followers_count": 70, "friends_count": 342, "statues_count": 1618 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576330695020544, "text": "@Luca_Bazooka34 ye", "in_reply_to_status": 669573989925261312, "in_reply_to_user": 581020279, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 581020279 }}, "user": { "id": 537236643, "name": "Ben Conti", "screen_name": "conti2123", "lang": "en", "location": "S. Portland, ME", "create_at": date("2012-03-26"), "description": "#RE2PECT || SPHS '16|| 2015 Western Maine Class A Champs ⚾️⚪️", "followers_count": 375, "friends_count": 223, "statues_count": 8546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Portland, ME", "id": "2438e517cb050ced", "name": "South Portland", "place_type": "city", "bounding_box": rectangle("-70.349558,43.595844 -70.220983,43.655565") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23005, "countyName": "Cumberland", "cityID": 2371990, "cityName": "South Portland" } }
+{ "create_at": datetime("2015-11-25T10:00:12.000Z"), "id": 669576330808131584, "text": "Cloudy this afternoon, high 74 (23 C). Low 64 (18 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712472, "name": "AustinWX", "screen_name": "AustinWX", "lang": "en", "location": "Austin, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Austin, Texas", "followers_count": 1009, "friends_count": 1368, "statues_count": 7898 }, "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-25T10:00:12.000Z"), "id": 669576330854408192, "text": "@aye__ron ty mi amor (':", "in_reply_to_status": 669573889572343808, "in_reply_to_user": 3024712384, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3024712384 }}, "user": { "id": 353733945, "name": "danny phantom ♌", "screen_name": "twerk4_jesus", "lang": "en", "location": "ur local trader joes", "create_at": date("2011-08-12"), "description": "milf // \\\\ $$ gucci mane gives me life $$ finesse ෆ‧⁺̣˚̣̣*̣̩✧₊̣.̩", "followers_count": 1366, "friends_count": 666, "statues_count": 29391 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westlake, OH", "id": "abcede5a546a90a3", "name": "Westlake", "place_type": "city", "bounding_box": rectangle("-81.970255,41.41104 -81.872904,41.478131") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3983622, "cityName": "Westlake" } }
+{ "create_at": datetime("2015-11-25T10:00:12.000Z"), "id": 669576330858500096, "text": "@Eddy_q8 and the ones that have baby on board", "in_reply_to_status": 669569700268015616, "in_reply_to_user": 71166554, "favorite_count": 0, "coordinate": point("-117.34050526,33.97489186"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 71166554 }}, "user": { "id": 1333094407, "name": "Y", "screen_name": "yfalmusallam", "lang": "en", "location": "California, USA", "create_at": date("2013-04-06"), "description": "Scorpio ¿?", "followers_count": 236, "friends_count": 137, "statues_count": 32157 }, "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-25T10:00:12.000Z"), "id": 669576330908831744, "text": "The Vitamin Shoppe: Key Holder (#Kailua, HI) https://t.co/CDWw4eVD7K #Retail #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-156.0042638,19.6494197"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Kailua", "Retail", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 28625039, "name": "TMJ-HON Retail Jobs", "screen_name": "tmj_hon_retail", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 374, "friends_count": 311, "statues_count": 455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kailua, HI", "id": "0118e7934cac5a2d", "name": "Kailua", "place_type": "city", "bounding_box": rectangle("-156.008921,19.615529 -155.961578,19.71363") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1523150, "cityName": "Kailua" } }
+{ "create_at": datetime("2015-11-25T10:00:12.000Z"), "id": 669576330959278080, "text": "Mddc é verdade!!!! \nSENHOR FAZEI COM QUE EU CONSIGA!!!! https://t.co/iW4NFvdQKd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 79474540, "name": "Senhora, Senhora?", "screen_name": "savicax", "lang": "pt", "location": "Teresina", "create_at": date("2009-10-03"), "description": "null", "followers_count": 175, "friends_count": 174, "statues_count": 7237 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576331030609920, "text": "Love teaching ppl how to cook", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1636228849, "name": "B☠", "screen_name": "Seitannotsatan", "lang": "en", "location": "null", "create_at": date("2013-07-31"), "description": "My body is not a graveyard", "followers_count": 826, "friends_count": 717, "statues_count": 8781 }, "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-25T10:00:12.000Z"), "id": 669576331034828800, "text": "@CrossClark I MISS YOU SO MUCH ��", "in_reply_to_status": 669575319037284352, "in_reply_to_user": 1260618506, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1260618506 }}, "user": { "id": 476203225, "name": "Taylor", "screen_name": "Taylor_Beerman", "lang": "en", "location": "Greenbrier, TN", "create_at": date("2012-01-27"), "description": "IG: Taylor_Beerman | snapchat: tbeerman | Vol State", "followers_count": 1097, "friends_count": 676, "statues_count": 11779 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenbrier, TN", "id": "012d6ac605978797", "name": "Greenbrier", "place_type": "city", "bounding_box": rectangle("-86.826405,36.396338 -86.771003,36.455366") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47147, "countyName": "Robertson", "cityID": 4730960, "cityName": "Greenbrier" } }
+{ "create_at": datetime("2015-11-25T10:00:12.000Z"), "id": 669576331101769728, "text": "We fucking out here homie https://t.co/p0JQ1MN64T", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 433003340, "name": "wujustyle.", "screen_name": "KyleAdamz", "lang": "en", "location": "null", "create_at": date("2011-12-09"), "description": "RSD has my heart.", "followers_count": 221, "friends_count": 270, "statues_count": 6277 }, "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-25T10:00:12.000Z"), "id": 669576331219353602, "text": "@TheEllenShow YOU ARE TRULY AN AMAZING HUMAN BEING I JUST APPRECIATE THE LOVE YOU SHOW EVERYONE LIKE WE ALL THE SAME!!!! ����������������", "in_reply_to_status": -1, "in_reply_to_user": 15846407, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15846407 }}, "user": { "id": 876765642, "name": "Kn1ckKn4ckM1n4j", "screen_name": "Kn1ckKn4ckM1n4j", "lang": "en", "location": "United States", "create_at": date("2012-10-12"), "description": "null", "followers_count": 445, "friends_count": 949, "statues_count": 2243 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576331277959168, "text": "@prettyme1699 \nI'd kill you", "in_reply_to_status": 669572781189627904, "in_reply_to_user": 2187910674, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2187910674 }}, "user": { "id": 2346160851, "name": "lexy", "screen_name": "lexayyy_morales", "lang": "en", "location": "California", "create_at": date("2014-02-15"), "description": "O.o", "followers_count": 463, "friends_count": 540, "statues_count": 6311 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576331370168321, "text": "We're #hiring! Click to apply: Club Supervisor Full Time - https://t.co/sOuOCuMBdL #Marketing #Honolulu, HI #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Marketing", "Honolulu", "Job", "Jobs", "CareerArc" }}, "user": { "id": 23493115, "name": "TMJ-HON Advert Jobs", "screen_name": "tmj_hon_adv", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Marketing/Ad/PR job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 288, "friends_count": 193, "statues_count": 9 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576331378561024, "text": "& im finally home ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 208792853, "name": "brit the goat", "screen_name": "_sacredHeart", "lang": "en", "location": "playing volleyball ", "create_at": date("2010-10-27"), "description": "addicted to breaking hearts & moving on..#DoitforJuan |#UL19| @OBJ_3 ❤️", "followers_count": 3502, "friends_count": 2584, "statues_count": 97081 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wallace, LA", "id": "0058c8ba4e9ee92a", "name": "Wallace", "place_type": "city", "bounding_box": rectangle("-90.687578,30.029095 -90.639209,30.046796") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22095, "countyName": "St. John the Baptist", "cityID": 2279275, "cityName": "Wallace" } }
+{ "create_at": datetime("2015-11-25T10:00:12.000Z"), "id": 669576331470897152, "text": "when you go to McDonald's before class and everyone asks why you didn't get them anything https://t.co/PIr4KGnDoQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1546888448, "name": "ashley", "screen_name": "asheeseidel", "lang": "en", "location": "Vegas", "create_at": date("2013-06-25"), "description": "hungry", "followers_count": 482, "friends_count": 323, "statues_count": 6791 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576331894505473, "text": "Want to work in #Anaheim, CA? View our latest opening: https://t.co/ikzZ2vroI4 #Hospitality #Veterans #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.9118558,33.8036983"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Anaheim", "Hospitality", "Veterans", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 27314974, "name": "TMJ-CAO HRTA Jobs", "screen_name": "tmj_cao_hrta", "lang": "en", "location": "Orange County, CA", "create_at": date("2009-03-28"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Orange County, CA. Need help? Tweet us at @CareerArc!", "followers_count": 424, "friends_count": 294, "statues_count": 1130 }, "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-25T10:00:12.000Z"), "id": 669576331978387456, "text": "If you're a #Hospitality professional in #Aiea, HI, check out this #job: https://t.co/02o0dV12JH #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.9427887,21.3850696"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "Aiea", "job", "Hiring", "CareerArc" }}, "user": { "id": 23897767, "name": "TMJ-HON Hosp. Jobs", "screen_name": "tmj_hon_hrta", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-11"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 406, "friends_count": 292, "statues_count": 258 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576332162916353, "text": "or green eyes, boys w green eyes hurt the soul ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 588669541, "name": "Selma Demirović", "screen_name": "OhHeySelma", "lang": "en", "location": "university of minnesota tc", "create_at": date("2012-05-23"), "description": "communist daughter", "followers_count": 615, "friends_count": 198, "statues_count": 22735 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576332188106753, "text": "@TrevorOLN got the shirt ❤️", "in_reply_to_status": 669534444424970240, "in_reply_to_user": 19746786, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19746786 }}, "user": { "id": 478312915, "name": "Brittany❤", "screen_name": "brittybritt19", "lang": "en", "location": "null", "create_at": date("2012-01-29"), "description": "23-sc-cats are life-#foreveralone", "followers_count": 1219, "friends_count": 1215, "statues_count": 24713 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576332456689665, "text": "#ObstructOnHwy at Orange Blossom Trail & W Landstreet Rd. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4046148,28.4359333"), "retweet_count": 0, "lang": "da", "is_retweet": false, "hashtags": {{ "ObstructOnHwy", "orlpol", "ocso" }}, "user": { "id": 39132832, "name": "Police Calls 32809", "screen_name": "orlpol32809", "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": 76, "friends_count": 1, "statues_count": 7331 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576332473475072, "text": "@Holly_Hamrick nice, heading to Shelby?", "in_reply_to_status": 669567619893039104, "in_reply_to_user": 137773496, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 137773496 }}, "user": { "id": 334907981, "name": "Mike Thomas", "screen_name": "tvphotog17", "lang": "en", "location": "Mooresville, NC", "create_at": date("2011-07-13"), "description": "WCCB News Rising @wilsonsworld Videographer @WCCBCharlotte. A NY'er, Husband, dad to 2 kids and a beagle. Love #NFL, #NASCAR & #bowling.", "followers_count": 769, "friends_count": 1021, "statues_count": 10637 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576332553134082, "text": "Finally get to hangout with my boy ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3174739846, "name": "Emm", "screen_name": "emmilyyt614", "lang": "en", "location": "null", "create_at": date("2015-04-16"), "description": "Ajh❤️", "followers_count": 273, "friends_count": 308, "statues_count": 3315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gahanna, OH", "id": "c97807ac2cd60207", "name": "Gahanna", "place_type": "city", "bounding_box": rectangle("-82.905845,39.987076 -82.802554,40.05651") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3929106, "cityName": "Gahanna" } }
+{ "create_at": datetime("2015-11-25T10:00:12.000Z"), "id": 669576332599144448, "text": "im 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": 628637917, "name": "Karina / / BDAY GIRL", "screen_name": "ardensboobs", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-07-06"), "description": "cool beans", "followers_count": 8345, "friends_count": 7970, "statues_count": 167861 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576332599296000, "text": "Friendship is so magical\n@AbbyMitchell7 https://t.co/hoGlyfvi2B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 332388704 }}, "user": { "id": 707932531, "name": "Sam Goodreau", "screen_name": "samuelgoodreau", "lang": "en", "location": "Denver", "create_at": date("2012-07-20"), "description": "shallower than a kiddie pool", "followers_count": 147, "friends_count": 132, "statues_count": 1049 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576332817268736, "text": "something a little different this morning �� https://t.co/qXsQsaox4W", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 117213725, "name": "felicia•marie", "screen_name": "the___fo_loso", "lang": "en", "location": "San Francisco, CA", "create_at": date("2010-02-24"), "description": "when you're a good woman, being a bad bitch is irrelevant ❤️ ig: the___fo_loso", "followers_count": 297, "friends_count": 589, "statues_count": 11753 }, "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-25T10:00:12.000Z"), "id": 669576332955615233, "text": "Just posted a photo @ LA Auto Show https://t.co/B6CJCHDD7r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.26923865,34.04127917"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14944593, "name": "yubertenator", "screen_name": "yubertenator", "lang": "en", "location": "Sunny SoCal", "create_at": date("2008-05-29"), "description": "I can do anything", "followers_count": 265, "friends_count": 798, "statues_count": 4836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-25T10:00:12.000Z"), "id": 669576333211619328, "text": "@CapehartJ: \"Five-deferment Don\" per @ColMorrisDavis is perfect fit for a lying/bigot like Trump.", "in_reply_to_status": 669161329786122240, "in_reply_to_user": 97474887, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 97474887, 489475164 }}, "user": { "id": 22336731, "name": "Suzanne Trimel", "screen_name": "STrimel", "lang": "en", "location": "Greater NYC ", "create_at": date("2009-02-28"), "description": "Comms Dir @OutRightIntl. #HumanRights. #ActOnClimate Back in the day: Higher ed VP, journalist. Hiker, aspiring farmer, irrepressible tweaker", "followers_count": 4060, "friends_count": 3663, "statues_count": 36054 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576333387825152, "text": "#SuspiciousVehicle at 5500-5899 Shasta Dr. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4101846,28.6140661"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SuspiciousVehicle", "orlpol", "ocso" }}, "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": 72, "friends_count": 1, "statues_count": 8495 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairview Shores, FL", "id": "d5c4f757d5df5fa0", "name": "Fairview Shores", "place_type": "city", "bounding_box": rectangle("-81.423995,28.582063 -81.366389,28.617568") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1221750, "cityName": "Fairview Shores" } }
+{ "create_at": datetime("2015-11-25T10:00:12.000Z"), "id": 669576333425438720, "text": "@TrillSammyy zaddy af ��", "in_reply_to_status": -1, "in_reply_to_user": 3242554800, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3242554800 }}, "user": { "id": 516947960, "name": "ebony", "screen_name": "elovewithme", "lang": "en", "location": "my lane", "create_at": date("2012-03-06"), "description": "♌ | the prize.", "followers_count": 1681, "friends_count": 234, "statues_count": 51192 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576333538627584, "text": "Morning clouds/afternoon sun this afternoon, high 74 (23 C). Low 65 (18 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712388, "name": "HoustonWX", "screen_name": "HoustonWX", "lang": "en", "location": "Houston, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Houston, Texas", "followers_count": 1737, "friends_count": 77, "statues_count": 7907 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:12.000Z"), "id": 669576333610131456, "text": "y'all dudes be acting like we got wit a nigga and knew he was gonna \"cheat on me\" we don't just be like yeap I want that fuck boy.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2528609294, "name": "kiora", "screen_name": "kioranisha_", "lang": "en", "location": "null", "create_at": date("2014-05-27"), "description": "null", "followers_count": 540, "friends_count": 301, "statues_count": 1890 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576333794525184, "text": "it's really annoying how papadosio has like no mercy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 346990051, "name": "Therian", "screen_name": "mynameisdomm", "lang": "en", "location": "Denver, CO", "create_at": date("2011-08-01"), "description": "you probably shouldn't follow me if we went to high school together.", "followers_count": 494, "friends_count": 369, "statues_count": 22044 }, "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-25T10:00:12.000Z"), "id": 669576333844844544, "text": "[Wed Nov 25th, 12:00pm] Temperature: 69.26 F, 20.7 C; Humidity: 43.9%; Light: 10.74%. https://t.co/UI5qWnQ21d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 269430133, "name": "Shoemaker Sandbox", "screen_name": "shoe_sandbox", "lang": "en", "location": "Earth (for now). ", "create_at": date("2011-03-20"), "description": "I am a bot. Developer account for @shoemaker.", "followers_count": 7, "friends_count": 1, "statues_count": 23522 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576333870161920, "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": 477919763, "name": "quita.", "screen_name": "stiasha", "lang": "en", "location": "IG: s.tiasha", "create_at": date("2012-01-29"), "description": "null", "followers_count": 1192, "friends_count": 743, "statues_count": 43453 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576333895319552, "text": "*eyerolls harder than i ever have 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": 290823494, "name": "♡ tina ♡", "screen_name": "CMellino", "lang": "en", "location": "free the nips", "create_at": date("2011-04-30"), "description": "g☯☯d woman \\\\ bad girl #buymepizza it's all about dat intersectional feminism ♡", "followers_count": 452, "friends_count": 298, "statues_count": 24827 }, "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.794651,35.767843") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3701520, "cityName": "Apex" } }
+{ "create_at": datetime("2015-11-25T10:00:12.000Z"), "id": 669576334188789760, "text": "This happens every time https://t.co/hadq66RWqH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1529583727, "name": "Brooke", "screen_name": "broookehubbard", "lang": "en", "location": "Frisco, TX", "create_at": date("2013-06-18"), "description": "null", "followers_count": 1195, "friends_count": 524, "statues_count": 15027 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576334213951489, "text": "Bulls in the Bronx // PTV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2911311564, "name": "ash ✨", "screen_name": "ashleeyfs", "lang": "en", "location": "Arizona, USA", "create_at": date("2014-11-26"), "description": "cool", "followers_count": 438, "friends_count": 270, "statues_count": 2806 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Parker, AZ", "id": "799b7ccd99238159", "name": "Parker", "place_type": "city", "bounding_box": rectangle("-114.321214,34.128976 -114.277674,34.163978") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4012, "countyName": "La Paz", "cityID": 453070, "cityName": "Parker" } }
+{ "create_at": datetime("2015-11-25T10:00:12.000Z"), "id": 669576334352490496, "text": "#AnimalCalls at 8075-9098 Turkey Lake Rd. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4750809,28.4451252"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AnimalCalls", "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": 265, "friends_count": 1, "statues_count": 64963 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576334537056260, "text": "瞧瞧那流洩而出的cheese......❤️❤️❤️惹得吃貨心花朵朵開啊!#newyork #shakeshack #shakestack @ Shake Shack Burger, NY https://t.co/1Q0RXS6HnO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.01470502,40.71535459"), "retweet_count": 0, "lang": "ja", "is_retweet": false, "hashtags": {{ "newyork", "shakeshack", "shakestack" }}, "user": { "id": 528226993, "name": "Jocelyn Hung Chien", "screen_name": "JocelynJinumuTW", "lang": "en", "location": "Taiwan", "create_at": date("2012-03-17"), "description": "null", "followers_count": 154, "friends_count": 77, "statues_count": 3443 }, "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("2015-11-25T10:00:12.000Z"), "id": 669576334562205696, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 536573652, "name": "KC❤️", "screen_name": "fuccklovex4", "lang": "en", "location": "null", "create_at": date("2012-03-25"), "description": "null", "followers_count": 862, "friends_count": 503, "statues_count": 65623 }, "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-25T10:00:13.000Z"), "id": 669576335375773696, "text": "Nature has a way to make everyone smile. Embrace it by exploring and witnessing the beauty and power… https://t.co/CwE2ugXuKQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-116.35416667,52.38722222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1489927855, "name": "KeepItContagious", "screen_name": "KICwithCOS", "lang": "en", "location": "Salt Lake City, Utah", "create_at": date("2013-06-07"), "description": "Human-powered Outdoor Sporting | Featuring 'The Most' Epic Adventurers! | Adventure . . . It's Contagious! | Join Our Community #KeepItContagious® |", "followers_count": 300, "friends_count": 387, "statues_count": 1132 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Clearwater County, Alberta", "id": "03e658ecfe149988", "name": "Clearwater County", "place_type": "city", "bounding_box": rectangle("-117.092338,51.6195 -114.429391,53.069152") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2015-11-25T10:00:13.000Z"), "id": 669576335480745984, "text": "my makeup cost $20 i love e.l.f. https://t.co/FawP9xPPvU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 776482902, "name": "peri", "screen_name": "qpdwblf", "lang": "en", "location": "Boston, MA", "create_at": date("2012-08-23"), "description": "18 • queer (she/they/he) • neu '20 • poli sci/international affairs/african studies/women's studies • photographer for @tastemakersmag", "followers_count": 288, "friends_count": 780, "statues_count": 3801 }, "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("2015-11-25T10:00:13.000Z"), "id": 669576335623237632, "text": "@connerschooler nah, I mean I didn't do much last night �� and no one saw me��", "in_reply_to_status": 669576189497896960, "in_reply_to_user": 733605806, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 733605806 }}, "user": { "id": 850182486, "name": "Steph moneeeeey", "screen_name": "steephaaanieee", "lang": "en", "location": "Taft, CA", "create_at": date("2012-09-27"), "description": "Taft College. 18. Virgo.", "followers_count": 1086, "friends_count": 787, "statues_count": 29336 }, "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("2015-11-25T10:00:13.000Z"), "id": 669576335849721856, "text": "Omg I was such a geek in middle school �� https://t.co/1WszB95ZoG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 427059297, "name": "yas", "screen_name": "jazkhalil98", "lang": "en", "location": "DMV", "create_at": date("2011-12-02"), "description": "Robo 2016. ♓️♈️cusp. Muslim.", "followers_count": 1214, "friends_count": 667, "statues_count": 21132 }, "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("2015-11-25T10:00:13.000Z"), "id": 669576335900192770, "text": "#VerbalDisturbance at 16567-16698 Tudor Grove Dr. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.1381588,28.5364295"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VerbalDisturbance", "orlpol", "ocso" }}, "user": { "id": 39344591, "name": "Police Calls 32828", "screen_name": "orlpol32828", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-11"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 124, "friends_count": 1, "statues_count": 3012 }, "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("2015-11-25T10:00:13.000Z"), "id": 669576336013299712, "text": "The students from The School of Communication presents: Eye of the Tigers \nCome out Wednesday… https://t.co/bqxgCIbPo7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.36114151,29.72203362"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 52456415, "name": "Jade♉️", "screen_name": "JayyThaTruthh", "lang": "en", "location": "Houston, TX", "create_at": date("2009-06-30"), "description": "College Senior. #TxSU16 #FemaleDJ", "followers_count": 1081, "friends_count": 1684, "statues_count": 10706 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:13.000Z"), "id": 669576336072155137, "text": "No one is going to mention Romeo's OL this year, but they'll next year when 3 are possibly signing somewhere. Trenches are going to be nasty", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27316439, "name": "The D Zone", "screen_name": "TheD_Zone", "lang": "en", "location": "mishigama (Michigan)", "create_at": date("2009-03-28"), "description": "The D Zone covers Michigan High School Sports (MHSAA / #MHSAAFB). For all inquiries contact: MichiganFootball@TheDZone.net #TheDZone", "followers_count": 39843, "friends_count": 7672, "statues_count": 127034 }, "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-25T10:00:13.000Z"), "id": 669576336151744513, "text": "Partly cloudy/wind this afternoon, high 73 (23 C). Low 64 (18 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575081, "name": "San Angelo Weather", "screen_name": "SanAngeloWX", "lang": "en", "location": "San Angelo, TX", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for San Angelo, TX", "followers_count": 1916, "friends_count": 92, "statues_count": 7797 }, "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("2015-11-25T10:00:13.000Z"), "id": 669576336256712704, "text": "Best wishes for a safe and fun #Thanksgiving from the TransOptions team! #BeStreetSmartNJ https://t.co/s1bABn2T2c", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Thanksgiving", "BeStreetSmartNJ" }}, "user": { "id": 57378085, "name": "TransOptions", "screen_name": "TransOptions", "lang": "en", "location": "Cedar Knolls, NJ", "create_at": date("2009-07-16"), "description": "We help you get around safely and efficiently by improving mobility, the environment and quality of life in northwestern NJ.", "followers_count": 432, "friends_count": 298, "statues_count": 1290 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morris Plains, NJ", "id": "01b1ca90360bd9f4", "name": "Morris Plains", "place_type": "city", "bounding_box": rectangle("-74.533584,40.816841 -74.460175,40.871364") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34027, "countyName": "Morris", "cityID": 3448210, "cityName": "Morris Plains" } }
+{ "create_at": datetime("2015-11-25T10:00:13.000Z"), "id": 669576336344682496, "text": "Photos: #Liverpool players train at Melwood ahead of Bordeaux clash https://t.co/JDdPzBJpAt https://t.co/tOoQ7qJpXv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.1017172,42.36405958"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Liverpool" }}, "user": { "id": 85661916, "name": "LFC Boston", "screen_name": "LFCBoston", "lang": "en", "location": "Boston, MA", "create_at": date("2009-10-27"), "description": "Official @LFC supporters club Boston, MA. Meet us for an #LFC game at the @PhoenixLanding2. Facebook & Instagram: /LFCBoston #YNWA #Liverpool Chair: @TimTreacy", "followers_count": 24861, "friends_count": 17681, "statues_count": 39431 }, "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("2015-11-25T10:00:13.000Z"), "id": 669576336525144064, "text": "@_CaRtOoNz Plz play the new Rainbow 6 beta!", "in_reply_to_status": -1, "in_reply_to_user": 2231422037, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2231422037 }}, "user": { "id": 711013675, "name": "BrendanPearce", "screen_name": "BrendanOrPearce", "lang": "en", "location": "Land O' Lakes", "create_at": date("2012-07-22"), "description": "null", "followers_count": 6, "friends_count": 17, "statues_count": 220 }, "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("2015-11-25T10:00:13.000Z"), "id": 669576336705327104, "text": "This #Transportation #job might be a great fit for you: Driver Opportunity - Big... - https://t.co/R10VDQN29y https://t.co/l1Elyt1Uu8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.3339324,42.4795019"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Transportation", "job" }}, "user": { "id": 2414879204, "name": "JOBS at AIM", "screen_name": "JobsAtAim", "lang": "en", "location": "www.aimntls.com", "create_at": date("2014-03-27"), "description": "STABILITY. ADVANCEMENT. IMPACT. We're looking for professionals like you to join our team!\nApply to our open #jobs below, and follow us here @aimntls", "followers_count": 26, "friends_count": 4, "statues_count": 73 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dunkirk, NY", "id": "7561ff4fba7e382d", "name": "Dunkirk", "place_type": "city", "bounding_box": rectangle("-79.400179,42.45681 -79.295345,42.506593") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36013, "countyName": "Chautauqua", "cityID": 3621105, "cityName": "Dunkirk" } }
+{ "create_at": datetime("2015-11-25T10:00:13.000Z"), "id": 669576336772501504, "text": "Wind 4.0 mph S. Barometer 29.939 in, Rising. Temperature 36.9 °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": 191, "friends_count": 58, "statues_count": 228803 }, "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("2015-11-25T10:00:13.000Z"), "id": 669576336944603136, "text": "my fish fat", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1409014771, "name": "laurie", "screen_name": "shutuplaurie", "lang": "en", "location": "null", "create_at": date("2013-05-06"), "description": "null", "followers_count": 977, "friends_count": 739, "statues_count": 25475 }, "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("2015-11-25T10:00:13.000Z"), "id": 669576337074593792, "text": "#VerbalDisturbance at 1700-1819 Flintshire St. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4031192,28.502251"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VerbalDisturbance", "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": 144, "friends_count": 1, "statues_count": 33304 }, "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("2015-11-25T10:00:13.000Z"), "id": 669576337443717122, "text": "Aye ladies @ilovedbarr is a fuckboy ask gabby ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 502524510 }}, "user": { "id": 2937758001, "name": "Hurtboyz", "screen_name": "KTshiona37", "lang": "en", "location": "Mint Hill, NC", "create_at": date("2014-12-22"), "description": "null", "followers_count": 141, "friends_count": 197, "statues_count": 1250 }, "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("2015-11-25T10:00:13.000Z"), "id": 669576337724583936, "text": "@iLLWritersGuild Hope it's not too serious! Maybe our #HealtheXpress clinicians can help you! https://t.co/NuThq8kGt1 ^LR", "in_reply_to_status": 669557841078845440, "in_reply_to_user": 1969420842, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HealtheXpress" }}, "user_mentions": {{ 1969420842 }}, "user": { "id": 518240271, "name": "Providence Health", "screen_name": "Prov_Health", "lang": "en", "location": "Renton, Washington", "create_at": date("2012-03-07"), "description": "null", "followers_count": 994, "friends_count": 57, "statues_count": 4313 }, "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-25T10:00:13.000Z"), "id": 669576337774931969, "text": "We're #hiring! Read about our latest #job opening here: Deli-Bakery Lead - https://t.co/qAu82ou2qy #CHARLOTTE, NC #Retail", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.779662,35.248586"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "CHARLOTTE", "Retail" }}, "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": 12111 }, "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("2015-11-25T10:00:13.000Z"), "id": 669576337959600131, "text": "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": 374193311, "name": "haaayley", "screen_name": "HAY_is_4_horses", "lang": "en", "location": "null", "create_at": date("2011-09-15"), "description": "my presence is a present, junior at ichs", "followers_count": 693, "friends_count": 1255, "statues_count": 25315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mingo Junction, OH", "id": "1bfb8cb47b6369af", "name": "Mingo Junction", "place_type": "city", "bounding_box": rectangle("-80.638406,40.288383 -80.599896,40.348841") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39081, "countyName": "Jefferson", "cityID": 3950904, "cityName": "Mingo Junction" } }
+{ "create_at": datetime("2015-11-25T10:00:13.000Z"), "id": 669576338139799557, "text": "@SoldierboyXB1 youtube vid coming out checking his card out!!!", "in_reply_to_status": 669566774413672449, "in_reply_to_user": 181620423, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 181620423 }}, "user": { "id": 479150546, "name": "Average Joe", "screen_name": "Ropie30", "lang": "en", "location": "Parts Unknown", "create_at": date("2012-01-30"), "description": "Ea Sports Game Changer and Youtuber. \nFor Madden ultimate team Tips tricks and Videos Visit me at http://Youtube.com/user/BigRopie", "followers_count": 5504, "friends_count": 203, "statues_count": 29181 }, "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("2015-11-25T10:00:13.000Z"), "id": 669576338186117120, "text": "Clean up volunteers should meet at lunch in Mr. Bertram's room��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2615334575, "name": "OWest Sock N' Buskin", "screen_name": "OWestDrama", "lang": "en", "location": "null", "create_at": date("2014-06-16"), "description": "The official Twitter acount for Oshkosh West's Sock N' Buskin Drama club!", "followers_count": 61, "friends_count": 93, "statues_count": 210 }, "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("2015-11-25T10:00:13.000Z"), "id": 669576338261434368, "text": "Snapchats update 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": 431293222, "name": "David Franklin", "screen_name": "DavidF105", "lang": "en", "location": "AL", "create_at": date("2011-12-07"), "description": "18. HB Grad. JSU. Instagram/Snapchat: DavidF105", "followers_count": 521, "friends_count": 306, "statues_count": 8772 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rainbow City, AL", "id": "793a678d3cac5024", "name": "Rainbow City", "place_type": "city", "bounding_box": rectangle("-86.091603,33.923633 -86.004203,33.991393") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1055, "countyName": "Etowah", "cityID": 163288, "cityName": "Rainbow City" } }
+{ "create_at": datetime("2015-11-25T10:00:13.000Z"), "id": 669576338353864704, "text": "#Standby at Maguire Rd & Reserve at Belmere Dr, Windermere, FL 34786. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.5416344,28.520872"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Standby", "orlpol", "ocso" }}, "user": { "id": 77447069, "name": "Police Calls 34786", "screen_name": "orlpol34786", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-09-26"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 53, "friends_count": 1, "statues_count": 1138 }, "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("2015-11-25T10:00:13.000Z"), "id": 669576338567618560, "text": "@Carleykuehn only 2 see you", "in_reply_to_status": 669572612297637888, "in_reply_to_user": 173763443, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 173763443 }}, "user": { "id": 1553749218, "name": "EDDY", "screen_name": "Eddysams20", "lang": "en", "location": "null", "create_at": date("2013-06-28"), "description": "| Txst Sigma Chi Pledge • live simply |", "followers_count": 602, "friends_count": 385, "statues_count": 9544 }, "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("2015-11-25T10:00:13.000Z"), "id": 669576338588770305, "text": "Gonna be here a while. Time to re-up the rainbow! (@ Headlights Hair Studio in San Francisco, CA) https://t.co/LzUYnlCiQq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.42434502,37.77668498"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43144049, "name": "Amber Gregory", "screen_name": "cardassianv0le", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-05-28"), "description": "Concert & event photographer, music lover, world traveler, Gemini, INFP.", "followers_count": 1225, "friends_count": 620, "statues_count": 77852 }, "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-25T10:00:13.000Z"), "id": 669576338802671617, "text": "I need to wash and cook.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 61546390, "name": "️", "screen_name": "httpscottie", "lang": "en", "location": "Ypsilanti, MI", "create_at": date("2009-07-30"), "description": "I was runnin' thru the 6 by myself!", "followers_count": 7013, "friends_count": 391, "statues_count": 230883 }, "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.32554") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26161, "countyName": "Washtenaw", "cityID": 2689140, "cityName": "Ypsilanti" } }
+{ "create_at": datetime("2015-11-25T10:00:13.000Z"), "id": 669576338831863808, "text": "I feel like this crack is getting bigger and bigger", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1953228307, "name": "christa", "screen_name": "_christaconner", "lang": "en", "location": "null", "create_at": date("2013-10-10"), "description": "sc: christaconner • Patterson, La.", "followers_count": 517, "friends_count": 286, "statues_count": 7681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Patterson, LA", "id": "27bab96869e9bfc1", "name": "Patterson", "place_type": "city", "bounding_box": rectangle("-91.350534,29.67399 -91.291746,29.725939") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22101, "countyName": "St. Mary", "cityID": 2259340, "cityName": "Patterson" } }
+{ "create_at": datetime("2015-11-25T10:00:13.000Z"), "id": 669576338869764098, "text": "No turkey today. (@ I Gemelli Ristorante in South Hackensack, NJ) https://t.co/bCNFhCW1PA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.048632,40.870613"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34144225, "name": "Andrew", "screen_name": "andrewlondon16", "lang": "en", "location": "Meadowlands & Exit 8a", "create_at": date("2009-04-21"), "description": "The Best Buildings in the Best locations.", "followers_count": 131, "friends_count": 182, "statues_count": 1120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Hackensack, NJ", "id": "7adc1fbe4ee2acd1", "name": "South Hackensack", "place_type": "city", "bounding_box": rectangle("-74.058107,40.853784 -74.038184,40.874207") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3428680, "cityName": "Hackensack" } }
+{ "create_at": datetime("2015-11-25T10:00:13.000Z"), "id": 669576338873802752, "text": "Partly cloudy this afternoon, high 74 (23 C). Low 61 (16 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575147, "name": "Baton Rouge Weather", "screen_name": "BatonRougeWX", "lang": "en", "location": "Baton Rouge, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Baton Rouge, LA", "followers_count": 2858, "friends_count": 92, "statues_count": 7750 }, "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("2015-11-25T10:00:13.000Z"), "id": 669576338974466048, "text": "I certainly figured this was the case! I bet your pecan pies are #BestInTheWorld ������ Happy Thanksgiving Eve Alana! https://t.co/KFYwkU7jrW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BestInTheWorld" }}, "user": { "id": 120108315, "name": "Tarek Jeremy Rogers", "screen_name": "tareksocal", "lang": "en", "location": "Hell Desert LA CA 661 NOW", "create_at": date("2010-03-05"), "description": "Just me...Loyalty & Integrity ChihuaDad! Crystal Hunt, Vanessa Marcil, qod, empire, T-Mobile, StewHams, Sobriety, these R my Fave Nouns", "followers_count": 746, "friends_count": 1011, "statues_count": 13508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Los Angeles, CA", "id": "ea9bfbd43c93400f", "name": "Lake Los Angeles", "place_type": "city", "bounding_box": rectangle("-117.862886,34.579878 -117.809256,34.642145") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639612, "cityName": "Lake Los Angeles" } }
+{ "create_at": datetime("2015-11-25T10:00:14.000Z"), "id": 669576339016544256, "text": "@anitamrijaj I fell asleep & forgot about it when I woke up", "in_reply_to_status": 669576259412865024, "in_reply_to_user": 300917530, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 300917530 }}, "user": { "id": 3196787913, "name": "cocash", "screen_name": "KocajSkocaj", "lang": "en", "location": "null", "create_at": date("2015-04-22"), "description": "ya look snazzy", "followers_count": 114, "friends_count": 62, "statues_count": 1273 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hartsdale, NY", "id": "2ceac98ca57995e2", "name": "Hartsdale", "place_type": "city", "bounding_box": rectangle("-73.826857,41.001752 -73.783178,41.039606") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3632523, "cityName": "Hartsdale" } }
+{ "create_at": datetime("2015-11-25T10:00:14.000Z"), "id": 669576339028971520, "text": "@Kill_Squadd yaaaaaaa", "in_reply_to_status": 669573089081040896, "in_reply_to_user": 2432490684, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2432490684 }}, "user": { "id": 1682025524, "name": "RickyBobbyPapi", "screen_name": "RickyBobbyPapi", "lang": "en", "location": "null", "create_at": date("2013-08-18"), "description": "just passing by", "followers_count": 233, "friends_count": 330, "statues_count": 2261 }, "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("2015-11-25T10:00:14.000Z"), "id": 669576339406491648, "text": "I'd fucking kill them https://t.co/yjbDS5DO6N", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 990116622, "name": "yung cranberry sauce", "screen_name": "sufferances", "lang": "en", "location": "So Cal", "create_at": date("2012-12-04"), "description": "Alex // 18 // gay as hell // pretends to be a bear // shitposting is all I know // INFJ // neighborhood hoe //\nicon by @horrorbuns", "followers_count": 263, "friends_count": 1102, "statues_count": 13380 }, "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-25T10:00:14.000Z"), "id": 669576339461140480, "text": "I am the god", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3031133055, "name": "SAD & HUNGRY", "screen_name": "jakewhoot", "lang": "en", "location": "Romulus, MI", "create_at": date("2015-02-11"), "description": "@_hannahd_ Starkweather", "followers_count": 184, "friends_count": 136, "statues_count": 9492 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Romulus, MI", "id": "781e991b9f95b37f", "name": "Romulus", "place_type": "city", "bounding_box": rectangle("-83.446302,42.179271 -83.306006,42.268212") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2669420, "cityName": "Romulus" } }
+{ "create_at": datetime("2015-11-25T10:00:14.000Z"), "id": 669576339830079488, "text": "#CareerArc #Clerical #Job alert: COLLECTOR (Job Fair June 17th) | Northside Hospital | #Atlanta, GA https://t.co/xooxrekLum #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.352759,33.909187"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CareerArc", "Clerical", "Job", "Atlanta", "Jobs", "Hiring" }}, "user": { "id": 22985155, "name": "Atlanta Clerical Job", "screen_name": "tmj_atl_cler", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-03-05"), "description": "Follow this account for geo-targeted Office/Clerical job tweets in Atlanta, GA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 490, "friends_count": 236, "statues_count": 113 }, "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("2015-11-25T10:00:14.000Z"), "id": 669576339855384576, "text": "Cool @Julio_beagle ! https://t.co/dYSdOVeP1m", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.88572095,40.12364512"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3045312730 }}, "user": { "id": 2569856559, "name": "Obe Wan Kenobe", "screen_name": "ajstenaka", "lang": "en", "location": "U.S. Pennsylvania", "create_at": date("2014-05-28"), "description": "My name is Obe I am a Captain in #TheAviators MARRIED to @wellytopping my kids are Little Nikki and @TTBTeddybear ! My Bff's are Binky & Didi! #keepfits, #BBOT", "followers_count": 3653, "friends_count": 3604, "statues_count": 192505 }, "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("2015-11-25T10:00:14.000Z"), "id": 669576339960131584, "text": "@GrandTheftAutm always trying to make people hungry on her story", "in_reply_to_status": -1, "in_reply_to_user": 1600456501, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1600456501 }}, "user": { "id": 1066522620, "name": "Legs", "screen_name": "whitegirlhaley", "lang": "en", "location": "California, USA", "create_at": date("2013-01-06"), "description": ".... . . ig:haleyisaacson. you prolly met me doing some handstands in some crazy place", "followers_count": 596, "friends_count": 485, "statues_count": 15817 }, "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("2015-11-25T10:00:14.000Z"), "id": 669576339989643269, "text": "I'm ready to get my hair done ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "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": 3363, "friends_count": 2732, "statues_count": 126740 }, "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("2015-11-25T10:00:14.000Z"), "id": 669576340035649536, "text": "@luisvvazquezz happy birthday luis. ��☺. I hope you have an amazing day. I lovee yaaa ������", "in_reply_to_status": -1, "in_reply_to_user": 1641683586, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1641683586 }}, "user": { "id": 2266489555, "name": "moniii", "screen_name": "sweett_moniii", "lang": "en", "location": "manda & god got me.", "create_at": date("2013-12-28"), "description": "anything she could do, i could do better.", "followers_count": 722, "friends_count": 633, "statues_count": 24243 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montclair, CA", "id": "3153d677b795e293", "name": "Montclair", "place_type": "city", "bounding_box": rectangle("-117.721042,34.046668 -117.680734,34.096817") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 648788, "cityName": "Montclair" } }
+{ "create_at": datetime("2015-11-25T10:00:14.000Z"), "id": 669576340107063296, "text": "Crystal Beach Loganberry 2 liter������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 459647299, "name": "Grim Sleeper", "screen_name": "BlackZeus_315", "lang": "en", "location": "TresUnoCinco ", "create_at": date("2012-01-09"), "description": "SC:roddie23 | Major Marqui is my pride and joy | baby let's have broke sex", "followers_count": 1081, "friends_count": 636, "statues_count": 66212 }, "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("2015-11-25T10:00:14.000Z"), "id": 669576340169957376, "text": "Work is forcing it very hard 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": 2167491276, "name": "Taylor", "screen_name": "taylorhaase21", "lang": "en", "location": "null", "create_at": date("2013-10-31"), "description": "pursue your happiness", "followers_count": 523, "friends_count": 309, "statues_count": 2797 }, "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("2015-11-25T10:00:14.000Z"), "id": 669576340211920896, "text": "Someone teach her how to cook please.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 40092446, "name": "Ashley Semple", "screen_name": "AshNigga", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2009-05-14"), "description": "New Yorker, Chef, Art connoisseur, Mother. My personality is like bacon - It's salty, it's bad for you but it's delicious.", "followers_count": 929, "friends_count": 199, "statues_count": 45080 }, "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-25T10:00:14.000Z"), "id": 669576340840955905, "text": "Can you recommend anyone for this #job? Activities Director - FT - Nursing and Rehab Aurora - https://t.co/FVks4OplWg #Aurora, CO #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.8319195,39.7294319"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Aurora", "Hiring" }}, "user": { "id": 141302910, "name": "Kindred Jobs ", "screen_name": "Kindred_Jobs", "lang": "en", "location": "National", "create_at": date("2010-05-07"), "description": "Kindred Healthcare, named a Fortune Magazine Most Admired Company, is the largest provider of post-acute care services in the United States.", "followers_count": 698, "friends_count": 0, "statues_count": 12904 }, "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": 8005, "countyName": "Arapahoe", "cityID": 804000, "cityName": "Aurora" } }
+{ "create_at": datetime("2015-11-25T10:00:14.000Z"), "id": 669576341008678912, "text": "Want to work at Schnitzer Steel? We're #hiring in #Modesto, CA! Click for details: https://t.co/2oJ8cwLkh9 #Transportation #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-120.9968782,37.6390972"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Modesto", "Transportation", "Job", "Jobs" }}, "user": { "id": 148606848, "name": "TMJ-CA Transport.", "screen_name": "tmj_CA_transp", "lang": "en", "location": "California", "create_at": date("2010-05-26"), "description": "Follow this account for geo-targeted Transportation job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 423, "friends_count": 296, "statues_count": 707 }, "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("2015-11-25T10:00:14.000Z"), "id": 669576341327626240, "text": "@MeganLuther @CDCgov I just did, how does one die from turkey???", "in_reply_to_status": 669574672569102336, "in_reply_to_user": 85613384, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 85613384, 146569971 }}, "user": { "id": 265559945, "name": "Jessica De Leon", "screen_name": "JDeleon1012", "lang": "en", "location": "Bradenton, FL", "create_at": date("2011-03-13"), "description": "Law Enforcement Reporter @ The Bradenton Herald, covering crime, courts and breaking news. FIU graduate and Miami girl at heart always!", "followers_count": 998, "friends_count": 515, "statues_count": 8396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bradenton, FL", "id": "008338532a26abed", "name": "Bradenton", "place_type": "city", "bounding_box": rectangle("-82.688331,27.424889 -82.459019,27.52306") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12081, "countyName": "Manatee", "cityID": 1207950, "cityName": "Bradenton" } }
+{ "create_at": datetime("2015-11-25T10:00:14.000Z"), "id": 669576341520433154, "text": "oops��☕️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1297635570, "name": "º° ᗰKᗩY °º", "screen_name": "makaylaa74", "lang": "en", "location": "Dry Phone, TX", "create_at": date("2013-03-24"), "description": "nothing is coming to me", "followers_count": 280, "friends_count": 248, "statues_count": 2423 }, "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("2015-11-25T10:00:14.000Z"), "id": 669576341533163520, "text": "Of course me and Alex get to the airport 5 hours early", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 47307231, "name": "Hannah Rose", "screen_name": "Hannahrosebx", "lang": "en", "location": "London", "create_at": date("2009-06-15"), "description": "an obsessive shopper", "followers_count": 125, "friends_count": 74, "statues_count": 2308 }, "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("2015-11-25T10:00:14.000Z"), "id": 669576341692420096, "text": "Mostly cloudy this afternoon, high 70 (21 C). Low 63 (17 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575204, "name": "New Orleans Weather", "screen_name": "NewOrleansWX", "lang": "en", "location": "New Orleans, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for New Orleans, LA", "followers_count": 2896, "friends_count": 92, "statues_count": 7768 }, "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("2015-11-25T10:00:14.000Z"), "id": 669576341704957952, "text": "This #SocialSciences #job might be a great fit for you: Lmsw - https://t.co/EStAo8KWKH #NewYork, NY #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9800645,40.74727"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SocialSciences", "job", "NewYork", "Hiring" }}, "user": { "id": 92694193, "name": "TMJ-USA SocSci Jobs", "screen_name": "tmj_usa_socsci", "lang": "en", "location": "USA", "create_at": date("2009-11-25"), "description": "Follow this account for geo-targeted Social Sciences job tweets in USA. Need help? Tweet us at @CareerArc!", "followers_count": 182, "friends_count": 101, "statues_count": 586 }, "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("2015-11-25T10:00:14.000Z"), "id": 669576341960921089, "text": "@Eslee__ thanks beauty��", "in_reply_to_status": 669574934268649472, "in_reply_to_user": 2997385145, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2997385145 }}, "user": { "id": 294706655, "name": "A", "screen_name": "yourstruly_ap", "lang": "en", "location": "978", "create_at": date("2011-05-07"), "description": "D.Newton❤️ † 10/21", "followers_count": 1135, "friends_count": 904, "statues_count": 44218 }, "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("2015-11-25T10:00:14.000Z"), "id": 669576342069866496, "text": "Amazing #lamborghini #aventador #sv #superveloce #super #veloce #supercar #car #racecar #luxury #luxe… https://t.co/KBTNrvi4fg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.15123093,34.14570683"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "lamborghini", "aventador", "sv", "superveloce", "super", "veloce", "supercar", "car", "racecar", "luxury", "luxe" }}, "user": { "id": 708008804, "name": "WELL", "screen_name": "maxwellengLmann", "lang": "en", "location": "Pasadena, CA", "create_at": date("2012-07-20"), "description": "null", "followers_count": 195, "friends_count": 418, "statues_count": 3085 }, "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("2015-11-25T10:00:14.000Z"), "id": 669576342082465793, "text": "Bright, airy, and what youâ\u0080\u0099ve been looking for! Taking a stroll through this house brings you into a charming li https://t.co/MPI38mTZqc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.108914,33.593333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 370243534, "name": "Desert View Realty", "screen_name": "DesertViewRealt", "lang": "en", "location": "Queen Creek, Arizona", "create_at": date("2011-09-08"), "description": "Full service Real Estate brokerage in Arizona. We serve the Phoenix East Valley, and pride ourselves on our incredible customer service!", "followers_count": 693, "friends_count": 533, "statues_count": 123533 }, "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-25T10:00:14.000Z"), "id": 669576342212468736, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 238073554, "name": "Hope McCabe", "screen_name": "Hope_McCabe", "lang": "en", "location": "null", "create_at": date("2011-01-14"), "description": "yikes", "followers_count": 155, "friends_count": 159, "statues_count": 3997 }, "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("2015-11-25T10:00:14.000Z"), "id": 669576342216810497, "text": "But she apparently isn't showing up for it so that'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": 300459907, "name": "Kyle Acly", "screen_name": "livinghypocrisy", "lang": "en", "location": "Elmira", "create_at": date("2011-05-17"), "description": "21 - suny Oswego '17", "followers_count": 225, "friends_count": 195, "statues_count": 13741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Horseheads, NY", "id": "b38ed6e85a8d3a22", "name": "Horseheads", "place_type": "city", "bounding_box": rectangle("-76.861348,42.135019 -76.748148,42.203132") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36015, "countyName": "Chemung", "cityID": 3635694, "cityName": "Horseheads" } }
+{ "create_at": datetime("2015-11-25T10:00:14.000Z"), "id": 669576342346698752, "text": "Swimming underwater flying over fire", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 496502261, "name": "KRI", "screen_name": "krislikenoodles", "lang": "en", "location": "San Pedro, Los Angeles", "create_at": date("2012-02-18"), "description": "toasted like bread on plasma", "followers_count": 207, "friends_count": 156, "statues_count": 3368 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-25T10:00:14.000Z"), "id": 669576342485233666, "text": "But we need bigger nets. https://t.co/50zkDRE9m5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 504956017, "name": "nick", "screen_name": "LiberateLarsson", "lang": "en", "location": "in a tough building", "create_at": date("2012-02-26"), "description": "|| NJDevils || Oilers || Good Hockey || Sarcasm ||Bobby Farnham has as many goals as Crosby. 'Tough Building' -Peter Deboer, 2011-14 PruCenter: 5-4-1", "followers_count": 449, "friends_count": 727, "statues_count": 22735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Frenchtown, NJ", "id": "008d11fa5093a960", "name": "Frenchtown", "place_type": "city", "bounding_box": rectangle("-75.086282,40.517631 -75.048744,40.557824") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34019, "countyName": "Hunterdon", "cityID": 3425350, "cityName": "Frenchtown" } }
+{ "create_at": datetime("2015-11-25T10:00:14.000Z"), "id": 669576342594129920, "text": "They won't ever know", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2187759955, "name": "MachoMechacho", "screen_name": "MecoMehmeti", "lang": "en", "location": "Dallas, TX", "create_at": date("2013-11-10"), "description": "〽️Fresh prince of Bel-Air 〽️", "followers_count": 104, "friends_count": 102, "statues_count": 287 }, "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("2015-11-25T10:00:14.000Z"), "id": 669576342610931712, "text": "The forecast for tonight is overcast and a low of 47°F. #fullMoon #InternationalHatDay #NationalParfaitDay #TieOneOnDay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.5333,41.5167"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "fullMoon", "InternationalHatDay", "NationalParfaitDay", "TieOneOnDay" }}, "user": { "id": 42666285, "name": "Quad Cities Weather", "screen_name": "qcweather", "lang": "en", "location": "Quad Cities", "create_at": date("2009-05-26"), "description": "Is it going to be warm, wet or cold in the Quad Cities today? You can also find us on Facebook.", "followers_count": 734, "friends_count": 0, "statues_count": 10426 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rock Island Arsenal, IL", "id": "8f966c3da836a196", "name": "Rock Island Arsenal", "place_type": "city", "bounding_box": rectangle("-90.570223,41.509992 -90.511303,41.523635") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17161, "countyName": "Rock Island", "cityID": 1765104, "cityName": "Rock Island Arsenal" } }
+{ "create_at": datetime("2015-11-25T10:00:14.000Z"), "id": 669576342636138496, "text": "Also boyfriend jeans because some days I just wanna look frumpy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1586301762, "name": "Syd", "screen_name": "sydneyxrenee", "lang": "en", "location": "null", "create_at": date("2013-07-11"), "description": "brb im probably dancing or somewhere between day dreaming and trying to plan out every detail of my future", "followers_count": 441, "friends_count": 280, "statues_count": 4390 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mineral Wells, TX", "id": "72c989ec59b8359b", "name": "Mineral Wells", "place_type": "city", "bounding_box": rectangle("-98.145924,32.781326 -98.040323,32.842211") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48363, "countyName": "Palo Pinto", "cityID": 4848684, "cityName": "Mineral Wells" } }
+{ "create_at": datetime("2015-11-25T10:00:14.000Z"), "id": 669576342887878657, "text": "#AccidentMinor at Spessard L. Holland East-West Expy & I-4. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3789165,28.5359279"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AccidentMinor", "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": 515, "friends_count": 1, "statues_count": 96023 }, "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("2015-11-25T10:00:14.000Z"), "id": 669576342933999616, "text": "yall know I like my guys Drake & lighter....��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 133408613, "name": "Queen Noel", "screen_name": "IAmNoel_Lamour", "lang": "en", "location": "DTX", "create_at": date("2010-04-15"), "description": "IG/Snapchat: iamnoel_lamour", "followers_count": 2725, "friends_count": 572, "statues_count": 90714 }, "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-25T10:00:14.000Z"), "id": 669576343055638530, "text": "@nj1015 @DennisandJudi I would love to go to Chester!! Can I have what you are giving away?", "in_reply_to_status": -1, "in_reply_to_user": 29407288, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 29407288, 2157803052 }}, "user": { "id": 113966711, "name": "Dennis Tester", "screen_name": "DTESTER24", "lang": "en", "location": "Union NJ", "create_at": date("2010-02-13"), "description": "Union NJ", "followers_count": 408, "friends_count": 793, "statues_count": 1417 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roselle Park, NJ", "id": "bcab414422facfa5", "name": "Roselle Park", "place_type": "city", "bounding_box": rectangle("-74.285711,40.657943 -74.24661,40.67251") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3464650, "cityName": "Roselle Park" } }
+{ "create_at": datetime("2015-11-25T10:00:14.000Z"), "id": 669576343156297729, "text": "@abbie_holba #friendshipgoals #pursegoals", "in_reply_to_status": 669576170334121984, "in_reply_to_user": 135702830, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "friendshipgoals", "pursegoals" }}, "user_mentions": {{ 135702830 }}, "user": { "id": 465996542, "name": "samson rickerson ❤", "screen_name": "SammieRicker", "lang": "en", "location": "Normal, IL", "create_at": date("2012-01-16"), "description": "illinois state '19 ❤️", "followers_count": 830, "friends_count": 589, "statues_count": 20850 }, "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("2015-11-25T10:00:15.000Z"), "id": 669576343236026368, "text": "Oh Wow! Time to stock up! Plus add code Jimm10 for additional 10% and free shipping # https://t.co/Os6QHG75fP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28753438, "name": "jimm parsons", "screen_name": "southsidejimm", "lang": "en", "location": "North Olmsted", "create_at": date("2009-04-03"), "description": "ProSupps Demo Rep save 10% with free shipping use code Jimm10 at http://www.prosupps.com", "followers_count": 1074, "friends_count": 1650, "statues_count": 987 }, "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("2015-11-25T10:00:15.000Z"), "id": 669576343558955008, "text": "On my way to Coshocton, OH. Thanksgiving is the best holiday. 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": 130533766, "name": "babs", "screen_name": "babs83xo", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2010-04-07"), "description": "null", "followers_count": 181, "friends_count": 811, "statues_count": 2173 }, "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("2015-11-25T10:00:15.000Z"), "id": 669576343911288832, "text": "Summer���� @cassidyfaithmcd https://t.co/aMWf9LuaqD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2904774026 }}, "user": { "id": 1975570645, "name": "karleyyyyyyyy", "screen_name": "karley_vance", "lang": "en", "location": "Wilmer, AL", "create_at": date("2013-10-20"), "description": "saraland// varsity soccer// sc:karley_vance", "followers_count": 752, "friends_count": 1119, "statues_count": 8746 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Saraland, AL", "id": "971d706ed1253595", "name": "Saraland", "place_type": "city", "bounding_box": rectangle("-88.157277,30.778082 -88.050084,30.859349") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile", "cityID": 168160, "cityName": "Saraland" } }
+{ "create_at": datetime("2015-11-25T10:00:15.000Z"), "id": 669576343919661057, "text": "DEADPOOl! USE THE CODE BF2015 FOR 20% OFF!\nhttps://t.co/g7IN1CHM2P\n#BLACKFRIDAY #HOLIDAYSHOPPING #SHOPNOW #ONSALE https://t.co/WMOEC7m8G1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BLACKFRIDAY", "HOLIDAYSHOPPING", "SHOPNOW", "ONSALE" }}, "user": { "id": 108335262, "name": "Yellow Rat Bastard", "screen_name": "yrbnyc", "lang": "en", "location": "New York City", "create_at": date("2010-01-25"), "description": "YRB is the official destination on the net for street style fashion at prices that won't make your pocket bleed. If it's hot, it's here: http://www.yrbnyc.com/", "followers_count": 998, "friends_count": 791, "statues_count": 943 }, "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("2015-11-25T10:00:15.000Z"), "id": 669576344167059457, "text": "You win. https://t.co/JgrPgUf5P7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 314149369, "name": "alexis™", "screen_name": "LuvLexx_", "lang": "en", "location": "null", "create_at": date("2011-06-09"), "description": "i aint shit & cant do shit for you.", "followers_count": 2738, "friends_count": 1402, "statues_count": 73781 }, "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("2015-11-25T10:00:15.000Z"), "id": 669576344196349952, "text": "If you're a #Engineering professional in #Folsom, California, check out this #job: https://t.co/H67AkU5zYk #Intel #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.1760583,38.6779591"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Engineering", "Folsom", "job", "Intel", "Hiring" }}, "user": { "id": 23372352, "name": "TMJ-SAC Engin. Jobs", "screen_name": "tmj_sac_eng", "lang": "en", "location": "Sacramento, CA", "create_at": date("2009-03-08"), "description": "Follow this account for geo-targeted Engineering job tweets in Sacramento, CA. Need help? Tweet us at @CareerArc!", "followers_count": 280, "friends_count": 228, "statues_count": 247 }, "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.714333") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 624638, "cityName": "Folsom" } }
+{ "create_at": datetime("2015-11-25T10:00:15.000Z"), "id": 669576344460591104, "text": "Mostly sunny this afternoon, high 71 (22 C). Low 61 (16 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575255, "name": "Shreveport Weather", "screen_name": "ShreveportWX", "lang": "en", "location": "Shreveport, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Shreveport, LA", "followers_count": 691, "friends_count": 92, "statues_count": 7826 }, "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("2015-11-25T10:00:15.000Z"), "id": 669576344485793792, "text": "If you still say yeet you need to reevaluate 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": 888080534, "name": "Freebandz Jr", "screen_name": "IamJnastyy", "lang": "en", "location": "null", "create_at": date("2012-10-17"), "description": "They said I wouldnt do it then we did it twice and had em lookin stupid sc: jnastyy7 l DevilGang Rest in peace micky ~~ you want some clout huh?", "followers_count": 1161, "friends_count": 729, "statues_count": 22243 }, "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("2015-11-25T10:00:15.000Z"), "id": 669576344741679104, "text": "Not hatin fam https://t.co/CzseIyVyxN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 321752529, "name": "Jess Koprowicz", "screen_name": "JessKoprowicz", "lang": "en", "location": "instagram: jess_koprowicz", "create_at": date("2011-06-21"), "description": "SJV'16", "followers_count": 950, "friends_count": 1040, "statues_count": 25161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Freehold, NJ", "id": "831d179c11927773", "name": "East Freehold", "place_type": "city", "bounding_box": rectangle("-74.265453,40.244158 -74.223417,40.293457") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3419150, "cityName": "East Freehold" } }
+{ "create_at": datetime("2015-11-25T10:00:15.000Z"), "id": 669576345173794816, "text": "Why does this look like a little Landon @Just_justinkidz �� https://t.co/3EbaljFqi9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 334800011 }}, "user": { "id": 232084978, "name": "Kay", "screen_name": "kailacridge", "lang": "en", "location": "Indianapolis", "create_at": date("2010-12-29"), "description": "RIP Levi & Tay. Indy.", "followers_count": 767, "friends_count": 459, "statues_count": 15876 }, "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("2015-11-25T10:00:15.000Z"), "id": 669576345291243520, "text": "our kid ❤️ https://t.co/y8xT8DBj8m", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 483347399, "name": "☹mergates☹", "screen_name": "savagegirley", "lang": "en", "location": "in the backfield ", "create_at": date("2012-02-04"), "description": "may your cup runneth over forever my niggas. jillian winn❤️", "followers_count": 676, "friends_count": 354, "statues_count": 14152 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mayfield, KY", "id": "f72d0fe211e1ad64", "name": "Mayfield", "place_type": "city", "bounding_box": rectangle("-88.680985,36.693546 -88.575406,36.767315") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21083, "countyName": "Graves", "cityID": 2150898, "cityName": "Mayfield" } }
+{ "create_at": datetime("2015-11-25T10:00:15.000Z"), "id": 669576345328816131, "text": "Robert Half Technology: C# Net Developer (Associate Level) (FULL-TIME) (#DESMOINES, IA) https://t.co/VmC8Z8PJnW #IT #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.6243133,41.5816456"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DESMOINES", "IT", "Job", "Jobs", "Hiring" }}, "user": { "id": 113370459, "name": "TMJ-IAD IT Jobs", "screen_name": "tmj_IAD_it", "lang": "en", "location": "Des Moines, IA", "create_at": date("2010-02-11"), "description": "Follow this account for geo-targeted Software Dev. - General/IT job tweets in Des Moines, IA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 333, "friends_count": 309, "statues_count": 114 }, "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("2015-11-25T10:00:15.000Z"), "id": 669576345333157888, "text": "@ashleyy_matt I am blessedt", "in_reply_to_status": 669548229650980864, "in_reply_to_user": 2342699857, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2342699857 }}, "user": { "id": 1312530799, "name": "liz.", "screen_name": "lizldunlap", "lang": "en", "location": "alabama / georgia / ohio", "create_at": date("2013-03-28"), "description": "will tweet but won't text back | ohs", "followers_count": 249, "friends_count": 109, "statues_count": 565 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheshire, OH", "id": "00fc40803ca6f9cf", "name": "Cheshire", "place_type": "city", "bounding_box": rectangle("-82.962712,40.212001 -82.929931,40.238807") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39053, "countyName": "Gallia", "cityID": 3913932, "cityName": "Cheshire" } }
+{ "create_at": datetime("2015-11-25T10:00:15.000Z"), "id": 669576345358233600, "text": "#Columbus, OH #Nursing #Job: Crit Care Outreach Inst II at OhioHealth https://t.co/YiryAa3rbO #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.0092803,39.9602601"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Columbus", "Nursing", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22149548, "name": "TMJ-CHH Nursing Jobs", "screen_name": "tmj_chh_nursing", "lang": "en", "location": "Columbus, OH", "create_at": date("2009-02-27"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Columbus, OH. Need help? Tweet us at @CareerArc!", "followers_count": 389, "friends_count": 303, "statues_count": 370 }, "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("2015-11-25T10:00:15.000Z"), "id": 669576345366749184, "text": "\"Shove it up your fucking ass\"����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.70466001,44.60163167"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 91605241, "name": "teri-anna libby", "screen_name": "libbs_00", "lang": "en", "location": "613", "create_at": date("2009-11-21"), "description": "Sunshine mixed with a double shot of whiskey", "followers_count": 1017, "friends_count": 1192, "statues_count": 7874 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Brockville, Ontario", "id": "02bfa8238069803a", "name": "Brockville", "place_type": "city", "bounding_box": rectangle("-75.736629,44.566171 -75.662382,44.635167") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36089, "countyName": "St. Lawrence" } }
+{ "create_at": datetime("2015-11-25T10:00:15.000Z"), "id": 669576345525981184, "text": "Trying to get this blue out of my hair is gonna be 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": 1397864138, "name": "The Weeknight", "screen_name": "MerandaTho", "lang": "en", "location": "null", "create_at": date("2013-05-02"), "description": "the hood loves me", "followers_count": 177, "friends_count": 87, "statues_count": 3951 }, "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("2015-11-25T10:00:15.000Z"), "id": 669576345769365504, "text": "@spars_22 \n\nنضحك يعني؟", "in_reply_to_status": 669575767718821889, "in_reply_to_user": 2265893950, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user_mentions": {{ 2265893950 }}, "user": { "id": 2697205908, "name": "️", "screen_name": "me9_AF", "lang": "ar", "location": "null", "create_at": date("2014-07-31"), "description": "null", "followers_count": 311, "friends_count": 356, "statues_count": 1854 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:15.000Z"), "id": 669576345966534657, "text": "@anna_99banana has snow in chicago and it looks so christmassy����❄️ so jealous", "in_reply_to_status": -1, "in_reply_to_user": 479581095, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 479581095 }}, "user": { "id": 321064028, "name": "VBP♠️", "screen_name": "VictoriaBee6", "lang": "en", "location": "johnson city // waynesboro", "create_at": date("2011-06-20"), "description": ";classy & vry sassy", "followers_count": 894, "friends_count": 354, "statues_count": 11838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waynesboro, VA", "id": "ae182036ae63c038", "name": "Waynesboro", "place_type": "city", "bounding_box": rectangle("-78.951336,38.042755 -78.855374,38.099106") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51820, "countyName": "Waynesboro", "cityID": 5183680, "cityName": "Waynesboro" } }
+{ "create_at": datetime("2015-11-25T10:00:15.000Z"), "id": 669576346125791232, "text": "I'm just trying hard to believe that there are good genuine ppl 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": 77437327, "name": "the Kings daughter", "screen_name": "KryshnaDeAmour", "lang": "en", "location": "null", "create_at": date("2009-09-26"), "description": "One day he will wake up & realize how amazing she really was, & when that day comes, she will be waking up next to the man who already knew✨", "followers_count": 750, "friends_count": 661, "statues_count": 6469 }, "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("2015-11-25T10:00:15.000Z"), "id": 669576346151034880, "text": "everyone in my life comes over and either talks to my mom for hours or falls asleep in my bed am I really that uninteresting ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 340453720, "name": "Cheyenne Hunt", "screen_name": "cheyennekaylee_", "lang": "en", "location": "Mansfield|AshlandU|Columbus", "create_at": date("2011-07-22"), "description": "I watch a lot of football. English & Philosophy are my things. Ohio Dominican Cheerleading.", "followers_count": 1117, "friends_count": 352, "statues_count": 26954 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mansfield, OH", "id": "f136163002bd51f6", "name": "Mansfield", "place_type": "city", "bounding_box": rectangle("-82.572489,40.681009 -82.378653,40.815657") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39139, "countyName": "Richland", "cityID": 3947138, "cityName": "Mansfield" } }
+{ "create_at": datetime("2015-11-25T10:00:15.000Z"), "id": 669576346369044480, "text": "@karriebearr https://t.co/wAXAiLivd5", "in_reply_to_status": -1, "in_reply_to_user": 18807649, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 18807649 }}, "user": { "id": 15286430, "name": "レン♚天下御免!", "screen_name": "eneru", "lang": "en", "location": "#リア充王国", "create_at": date("2008-07-01"), "description": "San Francisco, CA\n\nCurrently playing Splatoon (Wii U).\n\n未来で待ってる。EN/日本語。 Instagram: http://instagram.com/ener", "followers_count": 340, "friends_count": 183, "statues_count": 48887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downtown, San Francisco", "id": "d7ffb21e3a0ee55c", "name": "Downtown", "place_type": "neighborhood", "bounding_box": rectangle("-122.418962,37.781511 -122.395587,37.79383") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2015-11-25T10:00:15.000Z"), "id": 669576346385805312, "text": "@ThugDove we can https://t.co/aOYoSx5mZM", "in_reply_to_status": 669576169897881601, "in_reply_to_user": 1950604086, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1950604086 }}, "user": { "id": 61951609, "name": "Y E A H B A B Y", "screen_name": "DeondrexD", "lang": "en", "location": "Hidden Leaf Village", "create_at": date("2009-07-31"), "description": "I never go back on my word, cause thats my ninja way", "followers_count": 344, "friends_count": 289, "statues_count": 28491 }, "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("2015-11-25T10:00:15.000Z"), "id": 669576346440327169, "text": "does someone want to do something today?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1535150563, "name": "k", "screen_name": "kellimakala", "lang": "en", "location": "huntington beach", "create_at": date("2013-06-20"), "description": "pizza enthusiast", "followers_count": 416, "friends_count": 381, "statues_count": 1661 }, "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("2015-11-25T10:00:15.000Z"), "id": 669576346553708544, "text": "can someone please take me to Jubala", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2580593965, "name": "Claire Garcia", "screen_name": "clairitaa", "lang": "en", "location": "Walt Disney World", "create_at": date("2014-06-21"), "description": "living is easy with eyes closed", "followers_count": 196, "friends_count": 133, "statues_count": 2006 }, "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("2015-11-25T10:00:15.000Z"), "id": 669576346587254784, "text": "Df I look like", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2148988103, "name": "Obi‑Zay Kenobi", "screen_name": "e_xjavier", "lang": "en", "location": "making it out the 757", "create_at": date("2013-10-22"), "description": "$$ is the motive", "followers_count": 444, "friends_count": 131, "statues_count": 10483 }, "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-25T10:00:15.000Z"), "id": 669576346645983237, "text": "Thanks bro! https://t.co/Kjv5YIhRgO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1599923414, "name": "itsMYbirthday", "screen_name": "Kayylaaaa__", "lang": "en", "location": "New Orleans, LA", "create_at": date("2013-07-16"), "description": "Jayni❤️. Nigerian-American theWinningTeam", "followers_count": 1462, "friends_count": 939, "statues_count": 23383 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Terrytown, LA", "id": "9c97662afdd448aa", "name": "Terrytown", "place_type": "city", "bounding_box": rectangle("-90.043765,29.878501 -90.008313,29.92917") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2275180, "cityName": "Terrytown" } }
+{ "create_at": datetime("2015-11-25T10:00:15.000Z"), "id": 669576346801188864, "text": "@blackgent_1 thanks man! Hit me up Sunday", "in_reply_to_status": 669572806955372545, "in_reply_to_user": 310707113, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 310707113 }}, "user": { "id": 225632222, "name": "Chris Weber", "screen_name": "chrisweber479", "lang": "en", "location": "Detroit ✈️Murfreesboro", "create_at": date("2010-12-11"), "description": "MTSU. Advertising. Political Science. Preds fan and @Mtsuhockey player", "followers_count": 767, "friends_count": 885, "statues_count": 13596 }, "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("2015-11-25T10:00:15.000Z"), "id": 669576346851483649, "text": "Genesis Healthc is hiring! Cook Assistant #jobs in SCRANTON Apply today https://t.co/yDTxzD5kGl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.6626,41.4088"), "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": 166, "friends_count": 42, "statues_count": 68698 }, "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("2015-11-25T10:00:15.000Z"), "id": 669576346889269249, "text": "Don't get upset when you see me on Saturday but I'm telling you rn plz forgive me ���� @PeaceToMateoo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2338774847 }}, "user": { "id": 2882940489, "name": "Dayana Juca", "screen_name": "dnjax_", "lang": "en", "location": "null", "create_at": date("2014-11-18"), "description": "19.Queens,NY.", "followers_count": 91, "friends_count": 107, "statues_count": 542 }, "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("2015-11-25T10:00:15.000Z"), "id": 669576346993954816, "text": "Mostly cloudy this afternoon, high 74 (23 C). Low 62 (17 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575312, "name": "Lake Charles Weather", "screen_name": "LakeCharlesWX", "lang": "en", "location": "Lake Charles, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Lake Charles, LA", "followers_count": 536, "friends_count": 92, "statues_count": 7810 }, "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-25T10:00:15.000Z"), "id": 669576347203870720, "text": "Left hand is steering the other is grippen my thigh ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1538092771, "name": "sea sea", "screen_name": "ayee_seannaxo", "lang": "en", "location": "Lansing, MI", "create_at": date("2013-06-21"), "description": "Marlo❤️", "followers_count": 5577, "friends_count": 4758, "statues_count": 23921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Holt, MI", "id": "acdfa7edd1c52f4a", "name": "Holt", "place_type": "city", "bounding_box": rectangle("-84.582462,42.606392 -84.483579,42.674703") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2638780, "cityName": "Holt" } }
+{ "create_at": datetime("2015-11-25T10:00:15.000Z"), "id": 669576347354832897, "text": "@j_schwarz123 FINALLY", "in_reply_to_status": -1, "in_reply_to_user": 4275997768, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4275997768 }}, "user": { "id": 280743018, "name": "dzzaddy", "screen_name": "chloedeee", "lang": "en", "location": "null", "create_at": date("2011-04-11"), "description": "my eyelashes are longer than your dick.\n \ni can't taste my lips, could you do it for me?", "followers_count": 826, "friends_count": 1058, "statues_count": 28852 }, "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("2015-11-25T10:00:16.000Z"), "id": 669576347430330368, "text": "I'm back and I'm better", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 737640306, "name": "Taniya Breanda", "screen_name": "NiyaaaLove", "lang": "en", "location": "Buffalo, NY", "create_at": date("2012-08-04"), "description": "null", "followers_count": 3913, "friends_count": 3757, "statues_count": 121984 }, "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("2015-11-25T10:00:16.000Z"), "id": 669576347820367872, "text": "Girls that say man after everything is a man tbh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 621875785, "name": "тιиα❀", "screen_name": "TinaZientara", "lang": "en", "location": "Illinois", "create_at": date("2012-06-29"), "description": "♡ 3-27-14♡", "followers_count": 359, "friends_count": 135, "statues_count": 20509 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake in the Hills, IL", "id": "4864b48aaa6f2438", "name": "Lake in the Hills", "place_type": "city", "bounding_box": rectangle("-88.399452,42.168411 -88.286495,42.216104") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17111, "countyName": "McHenry", "cityID": 1741183, "cityName": "Lake in the Hills" } }
+{ "create_at": datetime("2015-11-25T10:00:16.000Z"), "id": 669576347870732289, "text": "No matter how perfect my my parallel parking attempt is, I always end up on the curb.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 201317415, "name": "Lauren Vaughn", "screen_name": "lovoVaughn", "lang": "en", "location": "Kansas City, MO", "create_at": date("2010-10-11"), "description": "I am Lauren Vaughn, and you are not. Tweet, tweet.", "followers_count": 237, "friends_count": 101, "statues_count": 1908 }, "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("2015-11-25T10:00:16.000Z"), "id": 669576347895754752, "text": "Can you recommend anyone for this #Retail #job? https://t.co/QAxInnRFV8 #Jacksonville, FL #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.5533442,30.3200526"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "Jacksonville", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 28440291, "name": "TMJ-JAX Retail Jobs", "screen_name": "tmj_jax_retail", "lang": "en", "location": "Jacksonville, FL", "create_at": date("2009-04-02"), "description": "Follow this account for geo-targeted Retail job tweets in Jacksonville, FL. Need help? Tweet us at @CareerArc!", "followers_count": 392, "friends_count": 312, "statues_count": 656 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, Jacksonville", "id": "2f16bd9318332d7c", "name": "Arlington", "place_type": "neighborhood", "bounding_box": rectangle("-81.630148,30.28664 -81.421131,30.412524") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12031, "countyName": "Duval", "cityID": 1235000, "cityName": "Jacksonville" } }
+{ "create_at": datetime("2015-11-25T10:00:16.000Z"), "id": 669576347908374528, "text": "اينكه انقدر بعضى از مشترى هاى ما خوبن و خوشتيپ، خودش #خشونت_علیه_زنان حساب مى شه! مثلاً ديروز آقاى پيتر! يا خدا!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fa", "is_retweet": false, "hashtags": {{ "خشونت_علیه_زنان" }}, "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": 249, "friends_count": 171, "statues_count": 5916 }, "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-25T10:00:16.000Z"), "id": 669576347983966208, "text": "Is Taylor Smiley in middle school why does she keep sending these FWD messages", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 469933196, "name": "Dalton Boccanfuso", "screen_name": "dc_boccanfuso", "lang": "en", "location": "null", "create_at": date("2012-01-20"), "description": "take a chill", "followers_count": 534, "friends_count": 437, "statues_count": 16248 }, "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("2015-11-25T10:00:16.000Z"), "id": 669576348470542337, "text": "IN ESCROW\n3 bed 3 bath Quartz Hill CA 93536 @ Quartz Hill, California https://t.co/rdfznpKXfu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.221,34.6522"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 53311154, "name": "Johnny James", "screen_name": "johnnyloans", "lang": "en", "location": "Southern California", "create_at": date("2009-07-02"), "description": "Your California Real estate Broker", "followers_count": 639, "friends_count": 1364, "statues_count": 805 }, "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("2015-11-25T10:00:16.000Z"), "id": 669576348634112000, "text": "Genesis Healthc is hiring! Licensed Practi #jobs in FREDON Apply today https://t.co/jgctQlc0MU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.803428,41.0644"), "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": 166, "friends_count": 42, "statues_count": 68699 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paulins Kill, NJ", "id": "008e98b27e357d78", "name": "Paulins Kill", "place_type": "city", "bounding_box": rectangle("-74.84126,41.04161 -74.791752,41.073959") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34041, "countyName": "Warren" } }
+{ "create_at": datetime("2015-11-25T10:00:16.000Z"), "id": 669576348990468096, "text": "Giving thanks for the Royals, Sam Brownback and Sly James - plus some 'no thanks' https://t.co/xEq4laBBfS @KCStar", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14955722 }}, "user": { "id": 110538332, "name": "Yael T. Abouhalkah", "screen_name": "YaelTAbouhalkah", "lang": "en", "location": "Kansas City, MO", "create_at": date("2010-02-01"), "description": "Kansas City Star Editorial Writer, dad, husband, distance runner.", "followers_count": 6219, "friends_count": 360, "statues_count": 27845 }, "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("2015-11-25T10:00:16.000Z"), "id": 669576349011410944, "text": "they might have gay marriage now but its always been and always will be an institution targeted towards straight people", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3783731534, "name": "the borst", "screen_name": "brady374411331", "lang": "en", "location": "null", "create_at": date("2015-10-04"), "description": "best/worst", "followers_count": 129, "friends_count": 182, "statues_count": 4854 }, "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("2015-11-25T10:00:16.000Z"), "id": 669576349271494657, "text": "Join the Citizens Financial Group team! See our latest #Banking #job opening here: https://t.co/BPvWlioVrv #cfgjobs #Pittsburgh, PA #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.9958864,40.4406248"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Banking", "job", "cfgjobs", "Pittsburgh", "Hiring" }}, "user": { "id": 22787034, "name": "TMJ-PIT Bank Jobs", "screen_name": "tmj_pit_banking", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2009-03-04"), "description": "Follow this account for geo-targeted Banking job tweets in Pittsburgh, PA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 371, "friends_count": 316, "statues_count": 83 }, "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("2015-11-25T10:00:16.000Z"), "id": 669576349758042112, "text": "S/o to ravyn for waking me up to eat 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": 3279111900, "name": "Maddydaddy", "screen_name": "MaddySalazar2", "lang": "en", "location": "Lubbock, TX", "create_at": date("2015-07-13"), "description": "null", "followers_count": 346, "friends_count": 274, "statues_count": 1146 }, "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("2015-11-25T10:00:16.000Z"), "id": 669576349879668736, "text": "I need a fiya New Years resolution", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 103252185, "name": "KissesToMyMistress", "screen_name": "TheeReallHer", "lang": "en", "location": "null", "create_at": date("2010-01-09"), "description": "A beautiful mind with a hell of a grind", "followers_count": 1441, "friends_count": 636, "statues_count": 26843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:16.000Z"), "id": 669576350030823424, "text": "@DunkinDonuts you really need a location in Rocky River, OH by the highway that's easy to get to..", "in_reply_to_status": -1, "in_reply_to_user": 8771022, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 8771022 }}, "user": { "id": 1035834324, "name": "#george™", "screen_name": "WreckItGeorge", "lang": "en", "location": "CLE", "create_at": date("2012-12-25"), "description": "Never settle for average. Stay young but not childish. Stand up for what you believe in.", "followers_count": 339, "friends_count": 488, "statues_count": 2517 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rocky River, OH", "id": "ad1fd8a17d633e77", "name": "Rocky River", "place_type": "city", "bounding_box": rectangle("-81.874568,41.449742 -81.823152,41.490668") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3968056, "cityName": "Rocky River" } }
+{ "create_at": datetime("2015-11-25T10:00:16.000Z"), "id": 669576350085328896, "text": "Genesis Healthc is hiring! Licensed Practi #jobs in FREDON Apply today https://t.co/jyJE6tGVAp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.803428,41.0644"), "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": 166, "friends_count": 42, "statues_count": 68700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paulins Kill, NJ", "id": "008e98b27e357d78", "name": "Paulins Kill", "place_type": "city", "bounding_box": rectangle("-74.84126,41.04161 -74.791752,41.073959") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34041, "countyName": "Warren" } }
+{ "create_at": datetime("2015-11-25T10:00:16.000Z"), "id": 669576350190014464, "text": "#ProjectMgmt #Job in #LosAngeles, CA: Programmer/Analyst II-Desktop Services at MotivAction Recruiting https://t.co/Mue6DKgFAT #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.2436849,34.0522342"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ProjectMgmt", "Job", "LosAngeles", "Jobs" }}, "user": { "id": 23903871, "name": "TMJ-LAX IT PM Jobs", "screen_name": "tmj_lax_itpm1", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-03-11"), "description": "Follow this account for geo-targeted IT-PM/BA job tweets in Los Angeles, CA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 311, "friends_count": 201, "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("2015-11-25T10:00:16.000Z"), "id": 669576350282334208, "text": "@agonzalezro @apolovald @jobandtalentEng cc @Konamito @cmilanf", "in_reply_to_status": 669573088581959680, "in_reply_to_user": 14273689, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 14273689, 233485015, 2805452341, 16945923, 120302975 }}, "user": { "id": 14732972, "name": "David Iglesias", "screen_name": "ditman", "lang": "en", "location": "Seattle, WA", "create_at": date("2008-05-11"), "description": "Father. Husband. Not-so-rookie Web Development Engineer @ Amazon · Ramblings here are mine. Not my employer's.", "followers_count": 959, "friends_count": 317, "statues_count": 12631 }, "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-25T10:00:16.000Z"), "id": 669576350538272768, "text": "Go there now https://t.co/W6ylehcHXr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14788764, "name": "Grant Cardone", "screen_name": "GrantCardone", "lang": "en", "location": "Miami Beach", "create_at": date("2008-05-15"), "description": "CEO 5 Companies, #Speaker #RealEstate Investor #Husband, #Father, #Author #SocialMediaLeader Founder of http://GrantCardoneTV.com", "followers_count": 340333, "friends_count": 1158, "statues_count": 73694 }, "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-25T10:00:16.000Z"), "id": 669576350844461056, "text": "Can we fuck 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": 620563065, "name": "GNRL_Randy", "screen_name": "Lilrandy_sc", "lang": "en", "location": "Allapatah Miami, FL", "create_at": date("2012-06-27"), "description": "Miami High Alumni (Class 2k15) • Miami Dade '17 • Future Journalist and Educator of brilliant minds • No matter what you do in life be the best at it!", "followers_count": 414, "friends_count": 402, "statues_count": 20821 }, "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("2015-11-25T10:00:16.000Z"), "id": 669576351163138048, "text": "See our latest #Madison, WI #job and click to apply: Diesel Mechanic/Technician III - https://t.co/tOIPs7EhDo #gopenskecareers #Automotive", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.328843,43.123533"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Madison", "job", "gopenskecareers", "Automotive" }}, "user": { "id": 180525542, "name": "Wisconsin Automotive", "screen_name": "tmj_WI_auto", "lang": "en", "location": "Wisconsin", "create_at": date("2010-08-19"), "description": "Follow this account for geo-targeted Automotive job tweets in Wisconsin Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 152, "friends_count": 127, "statues_count": 97 }, "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.491889 -86.24955,47.309715") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55025, "countyName": "Dane", "cityID": 5548000, "cityName": "Madison" } }
+{ "create_at": datetime("2015-11-25T10:00:16.000Z"), "id": 669576351272337411, "text": "@JeffRakestraw @voxdotcom this list was garbage, but do was the @TIME list. Everybody hates A Bug's Life. Idiots.", "in_reply_to_status": 669575121917616128, "in_reply_to_user": 73233459, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 73233459, 2347049341, 14293310 }}, "user": { "id": 35283591, "name": "Curt Rakestraw", "screen_name": "curtrakestraw", "lang": "en", "location": "Main Street, Memphis", "create_at": date("2009-04-25"), "description": "null", "followers_count": 167, "friends_count": 536, "statues_count": 2506 }, "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("2015-11-25T10:00:16.000Z"), "id": 669576351339454464, "text": "Helllll nawww���������� https://t.co/czxe65q85V", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 3394737827, "name": "Ken", "screen_name": "SosaStuntin", "lang": "en", "location": "West Virginia, USA", "create_at": date("2015-07-29"), "description": "wvu18", "followers_count": 220, "friends_count": 199, "statues_count": 1468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North East, MD", "id": "01394a901ecfda99", "name": "North East", "place_type": "city", "bounding_box": rectangle("-76.01977,39.556031 -75.886302,39.663376") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24015, "countyName": "Cecil", "cityID": 2456450, "cityName": "North East" } }
+{ "create_at": datetime("2015-11-25T10:00:16.000Z"), "id": 669576351364612097, "text": "GENESIS REHABIL is hiring! Speech-Language #jobs in EAST KANE Apply today https://t.co/CZmAPdUv2B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.687587,41.679886"), "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": 166, "friends_count": 42, "statues_count": 68701 }, "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": 42083, "countyName": "McKean" } }
+{ "create_at": datetime("2015-11-25T10:00:16.000Z"), "id": 669576351544922112, "text": "Gators love horse meat", "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 Bitch I'm From Flawda ☀️ 2⃣1⃣ Snapchat: LRF14 IG:_KDawgg", "followers_count": 1517, "friends_count": 1994, "statues_count": 26209 }, "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("2015-11-25T10:00:17.000Z"), "id": 669576351653826560, "text": "Interested in a #Clerical #job near #WEBSTER, NY? This could be a great fit: https://t.co/dLRJF0uWZ9 #OfficeTeam #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.4299939,43.2122851"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Clerical", "job", "WEBSTER", "OfficeTeam", "Hiring" }}, "user": { "id": 785315365, "name": "OfficeTeam Jobs", "screen_name": "OfficeTeamJobs", "lang": "en", "location": "Menlo Park, CA", "create_at": date("2012-08-27"), "description": "OfficeTeam is the world's leader in professional staffing for office support jobs. A Robert Half Company.", "followers_count": 169, "friends_count": 8, "statues_count": 33633 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Webster, NY", "id": "01e5198d558fc4d5", "name": "Webster", "place_type": "city", "bounding_box": rectangle("-77.534189,43.1963 -77.375276,43.267749") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3678960, "cityName": "Webster" } }
+{ "create_at": datetime("2015-11-25T10:00:17.000Z"), "id": 669576351729369088, "text": "At 7:00 AM, 2 WNW Bend [Deschutes Co, OR] COCORAHS reports HEAVY SNOW of 13.00 INCH #PDT https://t.co/wLO4Fp5nBk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.35,44.08"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PDT" }}, "user": { "id": 34930138, "name": "IEMBot PDT", "screen_name": "iembot_pdt", "lang": "en", "location": "null", "create_at": date("2009-04-24"), "description": "null", "followers_count": 13, "friends_count": 0, "statues_count": 35986 }, "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("2015-11-25T10:00:17.000Z"), "id": 669576351888773120, "text": "Temp 56.2°F Rising, Pressure 30.246in Falling slowly, Dew point 42.1°, Wind N 0mph, Rain today 0.00in https://t.co/gPtnr2dDtV #STLwx #MOwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.35555556,38.47694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "STLwx", "MOwx" }}, "user": { "id": 1191304230, "name": "Oakville Weather", "screen_name": "OakvilleWx", "lang": "en", "location": "Oakville, Missouri, USA", "create_at": date("2013-02-17"), "description": "Current conditions on the hour brought to you by @DaveStLou", "followers_count": 86, "friends_count": 80, "statues_count": 23276 }, "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("2015-11-25T10:00:17.000Z"), "id": 669576351964262401, "text": "Don't have anywhere to call home anymore��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 81270312, "name": "Mayuana.✨♒️", "screen_name": "MayuanaRachell", "lang": "en", "location": "between i want it and i got it", "create_at": date("2009-10-09"), "description": "MO✈️TX; MIZZOU✊ 1 Corinthians 13:13❤️ snap: mayrachell", "followers_count": 1175, "friends_count": 1033, "statues_count": 59463 }, "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("2015-11-25T10:00:17.000Z"), "id": 669576352027160576, "text": "Cloudy this afternoon, high 64 (18 C). Low 56 (13 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575409, "name": "Fort Smith Weather", "screen_name": "FortSmithWX", "lang": "en", "location": "Fort Smith, AR", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Fort Smith, AR", "followers_count": 440, "friends_count": 92, "statues_count": 7898 }, "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("2015-11-25T10:00:17.000Z"), "id": 669576352211701760, "text": "Doses & Mimosas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 58986123, "name": "Josie Mo", "screen_name": "Jtown95", "lang": "en", "location": "null", "create_at": date("2009-07-21"), "description": "20.\nCO. \nMay the sun kiss you unconditionally ϕιλαυτία", "followers_count": 392, "friends_count": 432, "statues_count": 6592 }, "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("2015-11-25T10:00:17.000Z"), "id": 669576352559853569, "text": "I need my Bestfriend @John_Pena1 to wake that ass up!!! I NEED to see you!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 53478470 }}, "user": { "id": 578220643, "name": "J. C'Mone⚜", "screen_name": "jusholmes28", "lang": "en", "location": "Seguin -- The Ville", "create_at": date("2012-05-12"), "description": "Dipped in chocolate, bronzed in elegance, enameled with grace, toasted with beauty. My lord, she's a black woman ✨ #SHSU", "followers_count": 941, "friends_count": 816, "statues_count": 38971 }, "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("2015-11-25T10:00:17.000Z"), "id": 669576352627097601, "text": "When @Nykei_34 says your forehead is thicker than a Harry Potter book ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 278706196 }}, "user": { "id": 524945148, "name": "Alexis", "screen_name": "ItsAlexisAllen", "lang": "en", "location": "null", "create_at": date("2012-03-14"), "description": "you were worth the cross, always remember that. #FCHW", "followers_count": 871, "friends_count": 373, "statues_count": 23115 }, "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("2015-11-25T10:00:17.000Z"), "id": 669576352975208449, "text": "Sometimes I wish I knew how to do makeup", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 291437368, "name": "Kyesha ✨", "screen_name": "Simply_Kye_15", "lang": "en", "location": "Beloit, WI ✈️ Ohio", "create_at": date("2011-05-01"), "description": "#CentralStateUniversity #CSU19 Biology Major IG:➡️Simply_Kye_15 SC: SimplyKye15", "followers_count": 8304, "friends_count": 9086, "statues_count": 58932 }, "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("2015-11-25T10:00:17.000Z"), "id": 669576353079914496, "text": "Want to work in #Dover, NH? View our latest opening: https://t.co/14lWETo5N1 #Nursing #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.8736698,43.1978624"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Dover", "Nursing", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 67377345, "name": "TMJ-NH Nursing Jobs", "screen_name": "tmj_nh_nursing", "lang": "en", "location": "New Hampshire", "create_at": date("2009-08-20"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in New Hampshire Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 323, "friends_count": 294, "statues_count": 140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dover, NH", "id": "41fdfd56e9c5e4d3", "name": "Dover", "place_type": "city", "bounding_box": rectangle("-70.96159,43.119192 -70.820896,43.255004") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33017, "countyName": "Strafford", "cityID": 3318820, "cityName": "Dover" } }
+{ "create_at": datetime("2015-11-25T10:00:17.000Z"), "id": 669576353109295104, "text": "@dustopian Yup", "in_reply_to_status": 669576165904920576, "in_reply_to_user": 2485528477, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2485528477 }}, "user": { "id": 162301746, "name": "Evelyn Garone", "screen_name": "EvelynGarone", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2010-07-03"), "description": "Conservative and PROUD of it! We Must Take Back America! #WakeUpAmerica No PC Spoken Here! Followed by Alveda C. King, James Rosen & Wayne Dupree Please NO PORN", "followers_count": 3983, "friends_count": 3578, "statues_count": 19538 }, "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-25T10:00:17.000Z"), "id": 669576353608572928, "text": "@DeionSlanders29 not you hush", "in_reply_to_status": 669575308237021184, "in_reply_to_user": 864869070, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 864869070 }}, "user": { "id": 1296914082, "name": "adrianna♡", "screen_name": "AdriannaGiaco", "lang": "en", "location": "huron Ohio", "create_at": date("2013-03-24"), "description": "im over it. snapchat & instagram: adriannagiaco", "followers_count": 749, "friends_count": 918, "statues_count": 10900 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huron, OH", "id": "370b68289d7cf4d8", "name": "Huron", "place_type": "city", "bounding_box": rectangle("-82.616017,41.372307 -82.511789,41.414712") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39043, "countyName": "Erie", "cityID": 3937016, "cityName": "Huron" } }
+{ "create_at": datetime("2015-11-25T10:00:17.000Z"), "id": 669576353658699777, "text": "he has taken my life away... ���� https://t.co/KW16FS3SKl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2697416329, "name": "Jasmine {stD} Guba", "screen_name": "jazzyguba", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-07-31"), "description": "@stevendoung inspires me, and he makes me smile | Cali and Pinas life is the good life | co-owner of @StevensEyebrows w/ @TWSavesMe_Doung | @itssamcollins", "followers_count": 271, "friends_count": 198, "statues_count": 5769 }, "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("2015-11-25T10:00:17.000Z"), "id": 669576353914724352, "text": "Snapchat✨ callme_dej", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 261049082, "name": "MOODY .", "screen_name": "callmedej__", "lang": "en", "location": "null", "create_at": date("2011-03-04"), "description": "null", "followers_count": 368, "friends_count": 183, "statues_count": 3815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Union City, GA", "id": "d1683f3c5c646d61", "name": "Union City", "place_type": "city", "bounding_box": rectangle("-84.597615,33.542538 -84.485509,33.633988") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1378324, "cityName": "Union City" } }
+{ "create_at": datetime("2015-11-25T10:00:17.000Z"), "id": 669576354338365444, "text": "Lala looked so cute today", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 750832531, "name": "britt", "screen_name": "brittneysizer_", "lang": "en", "location": "new london, ct", "create_at": date("2012-08-11"), "description": "ig: brittneysizer_", "followers_count": 861, "friends_count": 753, "statues_count": 16886 }, "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("2015-11-25T10:00:17.000Z"), "id": 669576354489307136, "text": "�� https://t.co/1b9dTwMYI5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 244179098, "name": "Al Sah•him", "screen_name": "Revof__", "lang": "en", "location": "Wingstop™", "create_at": date("2011-01-28"), "description": "Half man, half amazing probably half God but that don't fit my calculations FMOIG: @nvr_satisfied_", "followers_count": 292, "friends_count": 197, "statues_count": 43361 }, "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-25T10:00:17.000Z"), "id": 669576354648604673, "text": "Cloudy this afternoon, high 63 (17 C). Low 55 (13 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575444, "name": "Little Rock Weather", "screen_name": "LittleRockWX", "lang": "en", "location": "Little Rock, AR", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Little Rock, AR", "followers_count": 1834, "friends_count": 92, "statues_count": 7772 }, "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("2015-11-25T10:00:17.000Z"), "id": 669576354799579136, "text": "We about to take off sit tight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2917072129, "name": "the kid", "screen_name": "isthatton", "lang": "en", "location": "null", "create_at": date("2014-12-02"), "description": "oh really.. yes really", "followers_count": 602, "friends_count": 768, "statues_count": 7756 }, "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("2015-11-25T10:00:17.000Z"), "id": 669576354958999553, "text": "happy birthday saigers���� you stunning babe I hope it's filled with love! hope to see u soon�� https://t.co/kPYDaWdZSb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3065743729, "name": "D", "screen_name": "DeannaYujin", "lang": "en", "location": "null", "create_at": date("2015-03-06"), "description": "null", "followers_count": 208, "friends_count": 262, "statues_count": 1527 }, "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("2015-11-25T10:00:17.000Z"), "id": 669576355013632001, "text": "@woolie Doesn't Austin near UT? @DanKeshet", "in_reply_to_status": 669559054335766528, "in_reply_to_user": 3699001, "favorite_count": 0, "coordinate": point("-74.00739433,40.72072072"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3699001, 1069116314 }}, "user": { "id": 36166766, "name": "Dennis Griffith", "screen_name": "ISANobody_", "lang": "en", "location": "Lower East Side, Manhattan", "create_at": date("2009-04-28"), "description": "UIUC CS PhD student", "followers_count": 165, "friends_count": 132, "statues_count": 15334 }, "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("2015-11-25T10:00:17.000Z"), "id": 669576355089133568, "text": "@samanthafiske_ literally me rn too", "in_reply_to_status": 669572357045010432, "in_reply_to_user": 874402064, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 874402064 }}, "user": { "id": 625091872, "name": "lαυrα ❃", "screen_name": "lauraatandy_", "lang": "en", "location": "null", "create_at": date("2012-07-02"), "description": "la vie est belle • psu '19", "followers_count": 301, "friends_count": 202, "statues_count": 17068 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Somersworth, NH", "id": "80d905c7f0b9cf66", "name": "Somersworth", "place_type": "city", "bounding_box": rectangle("-70.927116,43.223489 -70.838483,43.286211") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33017, "countyName": "Strafford", "cityID": 3369940, "cityName": "Somersworth" } }
+{ "create_at": datetime("2015-11-25T10:00:17.000Z"), "id": 669576355109990400, "text": "Lol @ totally just bombing my math test.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2743895802, "name": "Stephanie Fitzi", "screen_name": "_stephhh24", "lang": "en", "location": "null", "create_at": date("2014-08-18"), "description": "Just a teen trying to have fun.", "followers_count": 167, "friends_count": 279, "statues_count": 751 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ripon, WI", "id": "65b6cf84960e5370", "name": "Ripon", "place_type": "city", "bounding_box": rectangle("-88.879255,43.824402 -88.816083,43.860473") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55039, "countyName": "Fond du Lac", "cityID": 5568175, "cityName": "Ripon" } }
+{ "create_at": datetime("2015-11-25T10:00:17.000Z"), "id": 669576355407863815, "text": "Just posted a photo @ Chicago, Illinois https://t.co/8u6m20kog6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6844,41.8369"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34970203, "name": "Chris Hainsworth", "screen_name": "Hainsworthy", "lang": "en", "location": "Right Behind You", "create_at": date("2009-04-24"), "description": "Actor/Playwright", "followers_count": 343, "friends_count": 577, "statues_count": 2068 }, "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-25T10:00:17.000Z"), "id": 669576355483258881, "text": "Oversight under the temp service I worked for seemed to avoid unions offering great opportunity but seemed like a funnel for introductions.", "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": 42, "friends_count": 157, "statues_count": 1297 }, "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("2015-11-25T10:00:17.000Z"), "id": 669576355483410432, "text": "raise your hand if you want to come over, bake brownies, and watch polar express!!!!������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 591429607, "name": "Victoria Gaa", "screen_name": "brbjdb", "lang": "en", "location": "null", "create_at": date("2012-05-26"), "description": "if Justin can do it I can do it", "followers_count": 1666, "friends_count": 1015, "statues_count": 29036 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gurnee, IL", "id": "013b204f383b0d64", "name": "Gurnee", "place_type": "city", "bounding_box": rectangle("-88.002391,42.340919 -87.873318,42.414636") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1732018, "cityName": "Gurnee" } }
+{ "create_at": datetime("2015-11-25T10:00:18.000Z"), "id": 669576355793760256, "text": "Look who stopped by @luxenailbaratl to get #naiiled by #thenailvillain @iam_perez while… https://t.co/fCRllcigKf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.382221,33.877893"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "naiiled", "thenailvillain" }}, "user_mentions": {{ 2422369392, 74345732 }}, "user": { "id": 67541432, "name": "The Nail Villain", "screen_name": "TerranceTerry", "lang": "en", "location": "Atlanta Georgia", "create_at": date("2009-08-20"), "description": "Terrance Terry Celeb Nail Stylist -Boutique Salon Owner -Luxe Nail Bar -Judge On Oxygen's TV Series Nail'd It &. L'Oreal Essie Product Educator", "followers_count": 4431, "friends_count": 590, "statues_count": 6144 }, "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("2015-11-25T10:00:18.000Z"), "id": 669576355823132672, "text": "Honestly https://t.co/5Dj0ZuAT33", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3073997972, "name": "gennivvv", "screen_name": "gennivvv", "lang": "en", "location": "girlss", "create_at": date("2015-03-11"), "description": "don't I'm gay", "followers_count": 537, "friends_count": 481, "statues_count": 7783 }, "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("2015-11-25T10:00:18.000Z"), "id": 669576355957313536, "text": "I swear that is you on the right https://t.co/CbewOWsqrT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1942474885, "name": "Will Ford", "screen_name": "willford012", "lang": "en", "location": "Baltimore", "create_at": date("2013-10-06"), "description": "null", "followers_count": 85, "friends_count": 73, "statues_count": 237 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mays Chapel, MD", "id": "00d7ec5b5ad6e33e", "name": "Mays Chapel", "place_type": "city", "bounding_box": rectangle("-76.770772,39.407728 -76.633663,39.492449") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2451587, "cityName": "Mays Chapel" } }
+{ "create_at": datetime("2015-11-25T10:00:18.000Z"), "id": 669576355999297537, "text": "'Play Safe' by Scott T. Kemery\nhttps://t.co/Z1fVCCM9iT\nPaul Morris\nTim Harmon\nTreasure Island Media https://t.co/gXD1ZZz8ha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2885606546, "name": "Scott T. Kemery", "screen_name": "ScottTKemery", "lang": "en", "location": "Fort Lauderdale, FL", "create_at": date("2014-10-31"), "description": "Writer", "followers_count": 17063, "friends_count": 18121, "statues_count": 52074 }, "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("2015-11-25T10:00:18.000Z"), "id": 669576356297089024, "text": "@winged_things yes they do holy shit", "in_reply_to_status": 669576161433886720, "in_reply_to_user": 55915627, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 55915627 }}, "user": { "id": 2771556927, "name": "Meara Brady", "screen_name": "itsmearabrady", "lang": "en", "location": "null", "create_at": date("2014-09-15"), "description": "Once namechecked by the Chicago Tribune for agreeing that Adam Sandler is the worst", "followers_count": 70, "friends_count": 83, "statues_count": 3505 }, "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-25T10:00:18.000Z"), "id": 669576356817072128, "text": "Goodmorning from Sandiego��✋��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 827160482, "name": "قتيبه الـمُلا.", "screen_name": "Qutaibaalmulla", "lang": "en", "location": "Kettering, OH❄️", "create_at": date("2012-09-16"), "description": "وأجمل من الحلم جيتي تقبلين✨ .. December #عبدالمجيد_عبدالله #أليووو؟", "followers_count": 648, "friends_count": 319, "statues_count": 24009 }, "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-25T10:00:18.000Z"), "id": 669576356829593600, "text": "Wow, how awesome is this! All done with @MorphiApp. Next up, R2-D2 #ctomoment @creativeedtech @billselak https://t.co/Wxgiu3v4UM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ctomoment" }}, "user_mentions": {{ 2430995557, 145744514, 14680984 }}, "user": { "id": 108097126, "name": "Brian Briggs", "screen_name": "bribriggs", "lang": "en", "location": "Davis, CA", "create_at": date("2010-01-24"), "description": "Director of Innovation and Technology for Plumas Lake School District. Or... Tech Goof for short :) #GTAMTV", "followers_count": 2918, "friends_count": 3160, "statues_count": 6492 }, "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("2015-11-25T10:00:18.000Z"), "id": 669576356884189185, "text": "*������ https://t.co/2CaBhZxPPZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 173653130, "name": "hannah cole", "screen_name": "hannahcole15", "lang": "en", "location": "starkvegas", "create_at": date("2010-08-01"), "description": "the cross before me, the world behind me. no turning back.", "followers_count": 799, "friends_count": 734, "statues_count": 8211 }, "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("2015-11-25T10:00:18.000Z"), "id": 669576357270171648, "text": "already know how it goes/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": 182209566, "name": "שָׁלוֹם", "screen_name": "xdmaitis", "lang": "en", "location": "deeznuts", "create_at": date("2010-08-23"), "description": "19 | οἶος Λύκος | Limits don't exist.", "followers_count": 338, "friends_count": 148, "statues_count": 36782 }, "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-25T10:00:18.000Z"), "id": 669576357651726336, "text": "Cloudy/wind this afternoon, high 65 (18 C). Low 58 (14 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26195145, "name": "Wichita Weather", "screen_name": "WichitaWeather", "lang": "en", "location": "Wichita, KS", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Wichita, KS", "followers_count": 1851, "friends_count": 88, "statues_count": 7627 }, "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("2015-11-25T10:00:18.000Z"), "id": 669576358012448768, "text": "The gratuitous shot while racing the clock. #carwash @ Waretown, New Jersey https://t.co/j1QpUVxAyG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.1884,39.7901"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "carwash" }}, "user": { "id": 733361352, "name": "bullitt auto detail ", "screen_name": "bullittdetail", "lang": "en", "location": "clayton, nc", "create_at": date("2012-08-02"), "description": "North Carolina's Premier Auto Detailer and vinyl wrapper for anything automotive: Antique, Collector, Motorhomes, Motorcycles, Show Cars, and Daily Drivers.", "followers_count": 109, "friends_count": 144, "statues_count": 420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waretown, NJ", "id": "00e30f515af15107", "name": "Waretown", "place_type": "city", "bounding_box": rectangle("-74.244698,39.762203 -74.169622,39.8114") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean", "cityID": 3476820, "cityName": "Waretown" } }
+{ "create_at": datetime("2015-11-25T10:00:18.000Z"), "id": 669576358134218752, "text": "J.R.R. Tolkien books......$3 hardcover, $2… https://t.co/NhdiaWWLsK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.113353,26.191726"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2482559504, "name": "Finding Forresters", "screen_name": "FFBooks2014", "lang": "en", "location": "Ft Lauderdale FL", "create_at": date("2014-05-07"), "description": "Like new Book, DVD Cd store. buy sell trade prices aver $3 Friendly helpful staff. Relaxing environment.", "followers_count": 59, "friends_count": 163, "statues_count": 588 }, "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("2015-11-25T10:00:18.000Z"), "id": 669576358255833088, "text": "@LilACups ahhhh, I think it was crucible. During salem witch trials. She was accused of killing her husband, and started acting like a witch", "in_reply_to_status": 669575064946241536, "in_reply_to_user": 1337642544, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1337642544 }}, "user": { "id": 2804306959, "name": "HOEZER", "screen_name": "Basedintherain", "lang": "en", "location": "Tucson", "create_at": date("2014-09-11"), "description": "+BASEDWORLD IN MY FUCKIN HART+LGND+ Father To A Beautiful Boy.", "followers_count": 551, "friends_count": 611, "statues_count": 5724 }, "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-25T10:00:18.000Z"), "id": 669576358461317122, "text": "@BettyMWhite Sweetheart...you're the greatest! Thank YOU for dedicating your life of entertaining the whole world.", "in_reply_to_status": 669569055234195456, "in_reply_to_user": 544517731, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 544517731 }}, "user": { "id": 462817661, "name": "John Doran", "screen_name": "JackAffy", "lang": "en", "location": "Lowell, MA", "create_at": date("2012-01-13"), "description": "Mild mannered certifiable geek, part time optimist, full time goof ball, Desert Shield/Storm/Watch vet. Watching the sun set knowing it will rise again.", "followers_count": 67, "friends_count": 42, "statues_count": 10056 }, "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("2015-11-25T10:00:18.000Z"), "id": 669576358591270913, "text": "@SumErgoMonstro 2. since Yemen is the destination, not the principal", "in_reply_to_status": 669566824019681280, "in_reply_to_user": 36689123, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 36689123 }}, "user": { "id": 682333, "name": "Fausta", "screen_name": "Fausta", "lang": "en", "location": "Princeton, NJ", "create_at": date("2007-01-22"), "description": "Blogger", "followers_count": 3547, "friends_count": 3895, "statues_count": 31100 }, "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("2015-11-25T10:00:18.000Z"), "id": 669576358800945152, "text": "fdb tbh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 2653973968, "name": "Vanny", "screen_name": "VaneeVanny", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2014-06-29"), "description": "c/o '17", "followers_count": 171, "friends_count": 240, "statues_count": 3570 }, "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("2015-11-25T10:00:18.000Z"), "id": 669576358868090880, "text": "We're #hiring! Click to apply: APIs and Portals Intern - https://t.co/EcjTXZsfOI #IT #Denver, CO #Job #Jobs", "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", "IT", "Denver", "Job", "Jobs" }}, "user": { "id": 714404437, "name": "VantivCareers", "screen_name": "VantivCareers", "lang": "en", "location": "Cincinnati, Ohio", "create_at": date("2012-07-24"), "description": "Vantiv is always looking for the best, brightest and most driven individuals to join our team.", "followers_count": 116, "friends_count": 34, "statues_count": 689 }, "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-25T10:00:18.000Z"), "id": 669576358876479488, "text": "Greatest night 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": 3053348222, "name": "Brandyycandyy", "screen_name": "BrandyFormisano", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-03-01"), "description": "✨Don't play games with a girl that can play them better, bitch. snapchat~ Brandyyyy", "followers_count": 281, "friends_count": 218, "statues_count": 696 }, "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("2015-11-25T10:00:18.000Z"), "id": 669576359203631104, "text": "@ejayfyke How sweet! What is your favorite song to sing together in the car?", "in_reply_to_status": 669336688745947136, "in_reply_to_user": 404132803, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 404132803 }}, "user": { "id": 312832454, "name": "Superior ToyotaScion", "screen_name": "SuperiorToyota", "lang": "en", "location": "Erie, PA", "create_at": date("2011-06-07"), "description": "Superior Toyota Scion is committed to sincere customer service and quality vehicles. | 5615 Peach Street, Erie, PA 16509 | (814) 868-3656", "followers_count": 264, "friends_count": 467, "statues_count": 2794 }, "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("2015-11-25T10:00:18.000Z"), "id": 669576359333617664, "text": "YASSS IM AT JESS HOUSE WAITING DOR U https://t.co/YSACRihqUK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 327877482, "name": "yung hooly", "screen_name": "hollahoolianna", "lang": "en", "location": "Long Beach, CA", "create_at": date("2011-07-02"), "description": "Julianna. 21. Lit like a candle. SC: yunghooly", "followers_count": 891, "friends_count": 726, "statues_count": 69166 }, "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("2015-11-25T10:00:18.000Z"), "id": 669576359568543744, "text": "Mood https://t.co/dKpk2rTtMM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2251424948, "name": "Marcell King Jr.", "screen_name": "iamjawanza", "lang": "en", "location": "#NOI", "create_at": date("2013-12-17"), "description": "Rawandan American Seattle✈️Salt Lake", "followers_count": 1100, "friends_count": 1086, "statues_count": 137 }, "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("2015-11-25T10:00:18.000Z"), "id": 669576359614799875, "text": "5'4 but everything all good ❤️ https://t.co/AKzm5A7fs2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1968714446, "name": "S. ⛵️", "screen_name": "sam__toYOU", "lang": "en", "location": "Meridian, MS", "create_at": date("2013-10-18"), "description": "gone keep your memory alive @Devonta16 ✨", "followers_count": 2728, "friends_count": 892, "statues_count": 41345 }, "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("2015-11-25T10:00:18.000Z"), "id": 669576359660773376, "text": "Cenpatico National #Insurance #Job: Behavioral Referral Specialist I (#Austin, TX) https://t.co/kIIOBm1wbJ #Jobs #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": {{ "Insurance", "Job", "Austin", "Jobs", "Hiring" }}, "user": { "id": 302104655, "name": "Austin Insurance Job", "screen_name": "tmj_aus_ins", "lang": "en", "location": "Austin, TX", "create_at": date("2011-05-20"), "description": "Follow this account for geo-targeted Insurance job tweets in Austin, TX from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 148, "friends_count": 115, "statues_count": 61 }, "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-25T10:00:18.000Z"), "id": 669576359836913664, "text": "@Titois ha no like seriouslu", "in_reply_to_status": 669572655029227520, "in_reply_to_user": 39281270, "favorite_count": 0, "coordinate": point("-122.48458293,47.15903698"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 39281270 }}, "user": { "id": 197278336, "name": "jimmy", "screen_name": "jmusclee", "lang": "en", "location": "null", "create_at": date("2010-09-30"), "description": "null", "followers_count": 321, "friends_count": 283, "statues_count": 10488 }, "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("2015-11-25T10:00:19.000Z"), "id": 669576359971164162, "text": "Seriously hop off my social media, I don't care if I've known you for 7 years or 3 months. I don't want to be associated with any of you now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1468842182, "name": "thankful thot", "screen_name": "partygnarly", "lang": "en", "location": "Youngstown, OH", "create_at": date("2013-05-29"), "description": "Rocking power brows and an impending sense of doom. Andrew's lady friend.", "followers_count": 371, "friends_count": 241, "statues_count": 22965 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Poland, OH", "id": "bd46604b88949129", "name": "Poland", "place_type": "city", "bounding_box": rectangle("-80.630194,40.987704 -80.519054,41.055936") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning", "cityID": 3963954, "cityName": "Poland" } }
+{ "create_at": datetime("2015-11-25T10:00:19.000Z"), "id": 669576359992270852, "text": "They beg for mercy like they lambo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 193166878, "name": "Wizards 6-4", "screen_name": "Pato_fromPoland", "lang": "en", "location": "Virginia", "create_at": date("2010-09-20"), "description": "#HalaMadrid #WizKid @andreitalove22", "followers_count": 126, "friends_count": 134, "statues_count": 8963 }, "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("2015-11-25T10:00:19.000Z"), "id": 669576360042602496, "text": "@pepper_RONNIE_ just come home its fine", "in_reply_to_status": 669516934312951808, "in_reply_to_user": 378140934, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 378140934 }}, "user": { "id": 609359247, "name": "Nicoal", "screen_name": "nicole_wenzl", "lang": "en", "location": "null", "create_at": date("2012-06-15"), "description": "NEBRASKA ➡ IOWA ⚽", "followers_count": 602, "friends_count": 620, "statues_count": 10785 }, "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("2015-11-25T10:00:19.000Z"), "id": 669576360105496576, "text": "If theres ever a nuke dropped on us, ill be ready! #Fallout4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Fallout4" }}, "user": { "id": 396640343, "name": "T. Taylor", "screen_name": "Jgrosodonia", "lang": "en", "location": "Upstate New York", "create_at": date("2011-10-23"), "description": "Sucka", "followers_count": 150, "friends_count": 215, "statues_count": 4702 }, "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("2015-11-25T10:00:19.000Z"), "id": 669576360168308736, "text": "@buffer I get to wake up and experience another day. Lol I get to see my wife one more day, live a life free. #bufferchat #littlethings", "in_reply_to_status": 669574792677322752, "in_reply_to_user": 197962366, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "bufferchat", "littlethings" }}, "user_mentions": {{ 197962366 }}, "user": { "id": 202268973, "name": "Phinisey", "screen_name": "phinisey", "lang": "en", "location": "Tacoma, WA", "create_at": date("2010-10-13"), "description": "Recording/Mixing Engineer, Music Producer. I live to bring a musical vision to fruition. I sing and write a little bit too. 1/3 Of @enter_exit_stay", "followers_count": 816, "friends_count": 387, "statues_count": 6601 }, "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("2015-11-25T10:00:19.000Z"), "id": 669576360268988417, "text": "Can you recommend anyone for this #job? Automotive Production Specialist (Bachelor’s Degree Required) - https://t.co/f0DK1FSNJn #Automotive", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.883282,32.509329"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Automotive" }}, "user": { "id": 180442164, "name": "Georgia Automotive", "screen_name": "tmj_GA_auto", "lang": "en", "location": "Georgia", "create_at": date("2010-08-19"), "description": "Follow this account for geo-targeted Automotive job tweets in Georgia Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 162, "friends_count": 125, "statues_count": 130 }, "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-25T10:00:19.000Z"), "id": 669576360294150145, "text": "@fatimarosaleees @kennedikristen https://t.co/liw0GqLHIr", "in_reply_to_status": -1, "in_reply_to_user": 1938602977, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1938602977, 2223217231 }}, "user": { "id": 2600158338, "name": "lexiiii", "screen_name": "alexisgamez_", "lang": "en", "location": "HTX", "create_at": date("2014-07-02"), "description": "sensational", "followers_count": 302, "friends_count": 178, "statues_count": 5211 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:19.000Z"), "id": 669576360378134529, "text": "@kayvonwebster #GetitGoGetit!", "in_reply_to_status": 669500457417441281, "in_reply_to_user": 136036360, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "GetitGoGetit" }}, "user_mentions": {{ 136036360 }}, "user": { "id": 2855961490, "name": "Gabriel Guerra", "screen_name": "PlayoffsG2", "lang": "en", "location": "null", "create_at": date("2014-11-01"), "description": "null", "followers_count": 17, "friends_count": 85, "statues_count": 185 }, "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-25T10:00:19.000Z"), "id": 669576360415760384, "text": "Afternoon drizzle this afternoon, high 56 (13 C). Low 35 (2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26199170, "name": "Omaha Weather", "screen_name": "OmahaWX", "lang": "en", "location": "Omaha, Nebraska", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Omaha, Nebraska", "followers_count": 2571, "friends_count": 88, "statues_count": 7620 }, "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("2015-11-25T10:00:19.000Z"), "id": 669576360449282048, "text": "How your man working late with no job ���� https://t.co/nGtmULbhMn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 412430635, "name": "Callie Nelson✌️", "screen_name": "CalliieeeJo", "lang": "en", "location": "Conway, AR", "create_at": date("2011-11-14"), "description": "Love yourself girl or nobody will✨ UCA 2018", "followers_count": 1007, "friends_count": 677, "statues_count": 31461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Little Rock, AR", "id": "31bb014b56203c53", "name": "North Little Rock", "place_type": "city", "bounding_box": rectangle("-92.352506,34.735212 -92.161794,34.840962") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 550450, "cityName": "North Little Rock" } }
+{ "create_at": datetime("2015-11-25T10:00:19.000Z"), "id": 669576360550002688, "text": "Christmas shopped a lil, now at work til 6, then afterwards ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2896460964, "name": "Tylor Graceanne!", "screen_name": "TylorGraceanne", "lang": "en", "location": "Beaumont, TX", "create_at": date("2014-11-11"), "description": "- junior , snapchat ogtyloor", "followers_count": 670, "friends_count": 848, "statues_count": 3537 }, "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("2015-11-25T10:00:19.000Z"), "id": 669576360642220032, "text": "when you rip off the blankets and you're in perfect booty pic position", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1450385143, "name": "meena", "screen_name": "Piameena", "lang": "en", "location": "CHSE", "create_at": date("2013-05-22"), "description": "KM", "followers_count": 402, "friends_count": 182, "statues_count": 9814 }, "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("2015-11-25T10:00:19.000Z"), "id": 669576360696741888, "text": "Tomorrow finna be oh 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": 1048882142, "name": "㊗️™", "screen_name": "Joneshoops24_", "lang": "en", "location": "null", "create_at": date("2012-12-30"), "description": "R.I.P Papa | Connally high School c/o 2017|", "followers_count": 1178, "friends_count": 857, "statues_count": 21302 }, "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-25T10:00:19.000Z"), "id": 669576360851992576, "text": "This might be the first time in weeks I've slept till one. It might be the small hangover and the fact my room is 25degrees", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 65102056, "name": "Emily", "screen_name": "hey_mango", "lang": "en", "location": "null", "create_at": date("2009-08-12"), "description": "Able to know where each road goes, but not the infinite in the coincidence each inch of it holds.", "followers_count": 215, "friends_count": 196, "statues_count": 8091 }, "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("2015-11-25T10:00:19.000Z"), "id": 669576361460264960, "text": "@GrantMaiorana4 happppy birthday grant ������", "in_reply_to_status": -1, "in_reply_to_user": 415106515, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 415106515 }}, "user": { "id": 331104696, "name": "Rach", "screen_name": "rachbroooks", "lang": "en", "location": "Norfolk, VA", "create_at": date("2011-07-07"), "description": "No such thing as a life that's better than yours", "followers_count": 1091, "friends_count": 836, "statues_count": 29494 }, "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("2015-11-25T10:00:19.000Z"), "id": 669576361552584708, "text": "@devolverdigital eyyy can we get a little update on holiday star uwuuuu", "in_reply_to_status": -1, "in_reply_to_user": 41609275, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 41609275 }}, "user": { "id": 834524390, "name": "witch bitch.", "screen_name": "hayley0wns", "lang": "en", "location": "null", "create_at": date("2012-09-19"), "description": "how in god's name did you find this twitter\n\nnsfw", "followers_count": 119, "friends_count": 198, "statues_count": 10276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wilbraham, MA", "id": "b5bf9fb45eef6b51", "name": "Wilbraham", "place_type": "city", "bounding_box": rectangle("-72.48489,42.084585 -72.388723,42.184765") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25013, "countyName": "Hampden", "cityID": 2579705, "cityName": "Wilbraham" } }
+{ "create_at": datetime("2015-11-25T10:00:19.000Z"), "id": 669576361762234368, "text": "@herrdopen you're right :(", "in_reply_to_status": 669389593771290624, "in_reply_to_user": 3089649469, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3089649469 }}, "user": { "id": 565769719, "name": "kal ❃", "screen_name": "kali_lain", "lang": "en", "location": "null", "create_at": date("2012-04-28"), "description": "SHS junior :-)", "followers_count": 1053, "friends_count": 715, "statues_count": 7983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shelbyville, IN", "id": "2037586e46d4bbca", "name": "Shelbyville", "place_type": "city", "bounding_box": rectangle("-85.808882,39.485252 -85.737081,39.569021") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18145, "countyName": "Shelby", "cityID": 1869318, "cityName": "Shelbyville" } }
+{ "create_at": datetime("2015-11-25T10:00:19.000Z"), "id": 669576362399629312, "text": "Lol H-Town", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1646856690, "name": "Pat", "screen_name": "KatPeane", "lang": "en", "location": "Downtown Los Angeles, CA", "create_at": date("2013-08-04"), "description": "user of a lot of stupid slang", "followers_count": 129, "friends_count": 119, "statues_count": 10467 }, "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("2015-11-25T10:00:19.000Z"), "id": 669576362408169473, "text": "I can never love again ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2953923448, "name": "8⃣", "screen_name": "DiamezJ", "lang": "en", "location": "null", "create_at": date("2014-12-31"), "description": "Humble•Blessed•Faithful•Thankful•Patience #GAFF Rip Quincy,Faye,Blinky&Pooh Arkansas Baptist College Student-Athlete #8", "followers_count": 255, "friends_count": 391, "statues_count": 800 }, "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("2015-11-25T10:00:19.000Z"), "id": 669576362575929345, "text": "@TheFunnyVine @rana_chouaib", "in_reply_to_status": 669202795459960834, "in_reply_to_user": 498177610, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 498177610, 3021033783 }}, "user": { "id": 3247393141, "name": "nama", "screen_name": "namacharara", "lang": "en", "location": "null", "create_at": date("2015-06-16"), "description": "#3", "followers_count": 233, "friends_count": 229, "statues_count": 282 }, "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("2015-11-25T10:00:19.000Z"), "id": 669576362852679681, "text": "Interested in a #Engineering #job near #Champaign, Illinois? This could be a great fit: https://t.co/R9mXGzC87J #Intel #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.2433829,40.1164204"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Engineering", "job", "Champaign", "Intel", "Hiring" }}, "user": { "id": 846116227, "name": "IL Engineering", "screen_name": "tmj_il_eng", "lang": "en", "location": "Illinois Non-Metro", "create_at": date("2012-09-25"), "description": "Follow this account for geo-targeted Engineering job tweets in Illinois Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 260, "friends_count": 230, "statues_count": 115 }, "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("2015-11-25T10:00:19.000Z"), "id": 669576362865201152, "text": "adele������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 185109947, "name": "jade a", "screen_name": "aloserlikeme", "lang": "en", "location": "manteca, CA", "create_at": date("2010-08-31"), "description": "i'd rather be center-left barricade✨", "followers_count": 1485, "friends_count": 993, "statues_count": 33379 }, "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("2015-11-25T10:00:19.000Z"), "id": 669576363070783489, "text": "Cloudy this afternoon, high 56 (13 C). Low 51 (11 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433220, "name": "St. Louis Weather", "screen_name": "StLouisWeather", "lang": "en", "location": "St. Louis Metro Area", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for St. Louis Metro Area", "followers_count": 3675, "friends_count": 88, "statues_count": 7593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "St. Louis, MO", "id": "60e6df5778ff9dac", "name": "St. Louis", "place_type": "city", "bounding_box": rectangle("-90.320515,38.531852 -90.166409,38.774346") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29510, "countyName": "St. Louis", "cityID": 2965000, "cityName": "St. Louis" } }
+{ "create_at": datetime("2015-11-25T10:00:19.000Z"), "id": 669576363225952257, "text": "This is insane https://t.co/JG0UdlsVlw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2162093425, "name": "汤米男孩", "screen_name": "isaiahthomass", "lang": "en", "location": "GREAT OAK HIGH SCHOOL", "create_at": date("2013-10-28"), "description": "5/31/15 a date which will live in infamy", "followers_count": 860, "friends_count": 379, "statues_count": 26848 }, "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("2015-11-25T10:00:19.000Z"), "id": 669576363414790144, "text": "I feel bad for anyone who has never had lumpys", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2782441017, "name": "Nov. 26", "screen_name": "LukeBender_3", "lang": "en", "location": "Around", "create_at": date("2014-09-24"), "description": "[PhilaU '19]", "followers_count": 464, "friends_count": 341, "statues_count": 3704 }, "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("2015-11-25T10:00:19.000Z"), "id": 669576363800571905, "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": 981523405, "name": "Carolinaa.♡", "screen_name": "caaaarolinaaaa_", "lang": "en", "location": "w/ aunisti ", "create_at": date("2012-11-30"), "description": "null", "followers_count": 827, "friends_count": 542, "statues_count": 42093 }, "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("2015-11-25T10:00:19.000Z"), "id": 669576363888746496, "text": "My family is literally the most boring family ever ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1075393004, "name": "Leslie Arias", "screen_name": "lesliearias16", "lang": "en", "location": "Winston-Salem, NC", "create_at": date("2013-01-09"), "description": "null", "followers_count": 176, "friends_count": 147, "statues_count": 730 }, "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("2015-11-25T10:00:19.000Z"), "id": 669576363943202816, "text": "Just a little salt���� https://t.co/5mhxysW7Mj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2757854189, "name": "09' in your basement", "screen_name": "ogxci", "lang": "en", "location": "Everywhere", "create_at": date("2014-08-31"), "description": "null", "followers_count": 135, "friends_count": 174, "statues_count": 718 }, "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("2015-11-25T10:00:20.000Z"), "id": 669576364190732288, "text": "the answer is don't think 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": 2610442546, "name": "p trin", "screen_name": "pat_trinidad", "lang": "en", "location": "vb", "create_at": date("2014-06-14"), "description": "basically just carter quotes", "followers_count": 178, "friends_count": 299, "statues_count": 1322 }, "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-25T10:00:20.000Z"), "id": 669576364295499776, "text": "been holding this cup for too long..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 625133609, "name": "JONOCTOBℜ", "screen_name": "JonOctober", "lang": "en", "location": "insta/snapchat: jonoctober ", "create_at": date("2012-07-02"), "description": "m y m u s i c p a i n t s p i c t u r e s . . . ∴ | utah | 801 |", "followers_count": 19946, "friends_count": 670, "statues_count": 26525 }, "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("2015-11-25T10:00:20.000Z"), "id": 669576364526292992, "text": "When you're all alone but you get the intense feeling that you're being watched... #pomlife… https://t.co/vOwIW0DEay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.04577845,38.80491934"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "pomlife" }}, "user": { "id": 893951646, "name": "Mia Lewis", "screen_name": "Mia_The_Pom", "lang": "en", "location": "null", "create_at": date("2012-10-20"), "description": "Bon vivant. Pomeranian extraordinaire. Born in Iowa, live in NYC. Find me on Instagram @miathepom", "followers_count": 54, "friends_count": 121, "statues_count": 669 }, "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("2015-11-25T10:00:20.000Z"), "id": 669576364672946176, "text": "@maryizzledawg ��������", "in_reply_to_status": 669571901593784320, "in_reply_to_user": 971703462, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 971703462 }}, "user": { "id": 50111109, "name": "David Edge", "screen_name": "DavidxEdge", "lang": "en", "location": "San Jose, CA", "create_at": date("2009-06-23"), "description": "I'm that guy who thinks he's fuuny. Straight edge and pro wrestling is my life. Instagram/Vine/Snapchat: Davidxedge #mountmarkmore", "followers_count": 1414, "friends_count": 722, "statues_count": 11176 }, "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("2015-11-25T10:00:20.000Z"), "id": 669576364706541570, "text": "We're #hiring! Click to apply: Health Enthusiast Part-Time - https://t.co/liocdrvJvM #Retail #Gilroy, CA #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.5625646,37.0162826"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Retail", "Gilroy", "Job", "Jobs", "CareerArc" }}, "user": { "id": 28537899, "name": "TMJ-SJC Retail Jobs", "screen_name": "tmj_sjc_retail", "lang": "en", "location": "San Jose, CA", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in San Jose, CA. Need help? Tweet us at @CareerArc!", "followers_count": 381, "friends_count": 314, "statues_count": 786 }, "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("2015-11-25T10:00:20.000Z"), "id": 669576364844957696, "text": "@MrVincredible @NYJ_Henry https://t.co/P9MQ7yR13D", "in_reply_to_status": 669576155381547013, "in_reply_to_user": 500485708, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 500485708, 260309136 }}, "user": { "id": 248989689, "name": "Meredith", "screen_name": "TheMiamiQueet", "lang": "en", "location": "SF Bay Area ✈️ Philly", "create_at": date("2011-02-07"), "description": "Atty. Equestrian. Lifter of Heavy Things. Football is the reason I live. ΚΚΓ Alum. Philly Sports. Jon Gruden is my spirit animal. Oh, and....I like to party.", "followers_count": 1901, "friends_count": 2010, "statues_count": 138914 }, "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("2015-11-25T10:00:20.000Z"), "id": 669576365536968704, "text": "I wish I had a room where no one could bug me so I could get all of these essays done ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 628034124, "name": "chancho", "screen_name": "jazael9", "lang": "en", "location": "null", "create_at": date("2012-07-05"), "description": "| Fútbol. #9 | BHS | stay humble", "followers_count": 506, "friends_count": 420, "statues_count": 13570 }, "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("2015-11-25T10:00:20.000Z"), "id": 669576365662932992, "text": "Who are you? �� https://t.co/MTqXbzywLT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 300344763, "name": "Drew Michelle", "screen_name": "drewmichelle77", "lang": "en", "location": "Birmingham, AL", "create_at": date("2011-05-17"), "description": "NM ☀️ || AL • UAB • Remembering Always Max 6/7/14", "followers_count": 672, "friends_count": 268, "statues_count": 14501 }, "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("2015-11-25T10:00:20.000Z"), "id": 669576365742555140, "text": "@iandotiansta @ZachSkov Oh shitt. Is it good?", "in_reply_to_status": 669574347229560832, "in_reply_to_user": 62102857, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 62102857, 35300944 }}, "user": { "id": 71491719, "name": "Eric", "screen_name": "canceric", "lang": "en", "location": "Future Hive, WA", "create_at": date("2009-09-04"), "description": "Immigrant // Graphic designer // Hip-Hop Head // Living the dream while you chumps sleep", "followers_count": 2082, "friends_count": 1019, "statues_count": 10727 }, "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-25T10:00:20.000Z"), "id": 669576365780303872, "text": "@nahidroxan https://t.co/kHCtnf6mWM", "in_reply_to_status": 669564246502604800, "in_reply_to_user": 52719044, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 52719044 }}, "user": { "id": 2196023482, "name": "rahim mojaver", "screen_name": "rahimmojaver1", "lang": "en", "location": "null", "create_at": date("2013-11-25"), "description": "null", "followers_count": 135, "friends_count": 465, "statues_count": 1006 }, "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("2015-11-25T10:00:20.000Z"), "id": 669576365813796864, "text": "We're #hiring! Click to apply: Registered Nurse – Inpatient – Per Diem - https://t.co/YvXzgwwDo5 #Goodyear, AZ https://t.co/7lVfMbeCPT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.3596206,33.4528164"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Goodyear" }}, "user": { "id": 128004921, "name": "CTCA Jobs", "screen_name": "CTCAJobs", "lang": "en", "location": "#AZ #GA #IL #OK #PA #FL", "create_at": date("2010-03-30"), "description": "Here you'll find our latest job opportunities across the U.S. Follow @CTCACareers to learn more & ask us questions! #jobs #careers #healthcare", "followers_count": 834, "friends_count": 484, "statues_count": 10778 }, "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("2015-11-25T10:00:20.000Z"), "id": 669576365876768768, "text": "Cloudy this afternoon, high 60 (16 C). Low 58 (14 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433608, "name": "Kansas City Weather", "screen_name": "KansasCityWX", "lang": "en", "location": "Kansas City Metro Area", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Kansas City Metro Area", "followers_count": 1921, "friends_count": 88, "statues_count": 7713 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "ddabe75a8f44fe5a", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.765917,38.827176 -94.385522,39.356662") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2015-11-25T10:00:20.000Z"), "id": 669576365981614081, "text": "@CambiBrown @julezeverywhere shhh don't tell ��", "in_reply_to_status": 669576128646877184, "in_reply_to_user": 331794703, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 331794703, 1514683081 }}, "user": { "id": 27238125, "name": "Matt D", "screen_name": "dcap22803", "lang": "en", "location": "Elk Grove, CA", "create_at": date("2009-03-28"), "description": "Diehard @GoodDaySac @markatthemovies @buffalobilks fan with brain/eye tumors lover of life fun and football. Toy Soldier Collector", "followers_count": 1386, "friends_count": 2010, "statues_count": 72620 }, "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("2015-11-25T10:00:20.000Z"), "id": 669576366061260800, "text": "Come check out our awesome office! We're having an Open House on Saturday, Dec 5 https://t.co/Wu47HFOeDi https://t.co/Gq1jV8k3Bf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4034356333, "name": "Gateway Incubator", "screen_name": "gtwyinc", "lang": "en", "location": "Oakland", "create_at": date("2015-10-27"), "description": "California's first cannabis incubator, and the first program of its kind anywhere. #cannabis #startup #entrepreneurs #oakland #incubator #investment", "followers_count": 161, "friends_count": 216, "statues_count": 86 }, "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("2015-11-25T10:00:20.000Z"), "id": 669576366082424833, "text": "I love this women ����❤️ https://t.co/3K4gCo6Lbt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 403609480, "name": "James Melo Deen", "screen_name": "jamesdeen11", "lang": "en", "location": "Laurence Harbor", "create_at": date("2011-11-02"), "description": "Just Living A Blue Life", "followers_count": 217, "friends_count": 176, "statues_count": 8931 }, "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("2015-11-25T10:00:20.000Z"), "id": 669576366313074688, "text": "Chill @ Denver, Colorado https://t.co/fEfxz4Bipg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.984,39.7392"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 190655719, "name": "midzman", "screen_name": "midzman", "lang": "en", "location": "colorado", "create_at": date("2010-09-14"), "description": "Play your own game, be your own man, and don't ask anyone for a stamp of approval. - HST", "followers_count": 78, "friends_count": 266, "statues_count": 341 }, "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-25T10:00:20.000Z"), "id": 669576366682062848, "text": "We're #hiring! Read about our latest #job opening here: Full Time Sales - https://t.co/xYkzMVitHh #Cheshire, CT #Sales #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.9106202,41.5083668"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Cheshire", "Sales", "CareerArc" }}, "user": { "id": 22056893, "name": "TMJ-CT-US Sales Jobs", "screen_name": "tmj_ct_sales", "lang": "en", "location": "Connecticut", "create_at": date("2009-02-26"), "description": "Follow this account for geo-targeted Sales job tweets in Connecticut Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 399, "friends_count": 304, "statues_count": 237 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheshire, CT", "id": "2c440fa845f602fa", "name": "Cheshire", "place_type": "city", "bounding_box": rectangle("-72.948301,41.452735 -72.846378,41.563837") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 914200, "cityName": "Cheshire Village" } }
+{ "create_at": datetime("2015-11-25T10:00:20.000Z"), "id": 669576366942064640, "text": "If my red eyes don't see you anymore and I can't hear you through the white noise, just send your heartbeat I'll go to the blue ocean floor��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2835559386, "name": "Raymond Santiago", "screen_name": "LaserFocusRay", "lang": "en", "location": "Marina Del Rey, CA", "create_at": date("2014-09-29"), "description": "Long time WWE fan, Huge LA Lakers fan, Captivator, @jtimberlake is my idol", "followers_count": 190, "friends_count": 152, "statues_count": 2522 }, "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("2015-11-25T10:00:20.000Z"), "id": 669576366963081216, "text": "Hey guys! A friend of mine is in financial trouble! Let's spread this around and help him out! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29594054, "name": "Christina Kelly", "screen_name": "ChristinaMarieK", "lang": "en", "location": "Houston, TX", "create_at": date("2009-04-07"), "description": "BFA in Acting from the University of Houston School of Theatre and Dance. Voice Actress for Sentai Filmworks. Represented by the Pastorini Bosby Agency.", "followers_count": 804, "friends_count": 277, "statues_count": 1324 }, "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-25T10:00:20.000Z"), "id": 669576366967230465, "text": "When I just want to run away and hide for the day but you've got nowhere to do so", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 297006004, "name": "Lily", "screen_name": "liliesforall", "lang": "en", "location": "419, 575, 850", "create_at": date("2011-05-11"), "description": "dreams are only the roots to our own destiny. follow them and fight for them", "followers_count": 64, "friends_count": 64, "statues_count": 811 }, "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("2015-11-25T10:00:20.000Z"), "id": 669576367017709568, "text": "fair -> partly cloudy\ntemperature down 77°F -> 73°F\nhumidity up 55% -> 69%\nwind 20mph -> 24mph\npressure 30.27in falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.68599,28.35709"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 226790408, "name": "Weather", "screen_name": "MerrittIslandFL", "lang": "en", "location": "Merritt Island, FL", "create_at": date("2010-12-14"), "description": "Weather updates, forecast, warnings and information for Merritt Island, FL. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 79, "friends_count": 2, "statues_count": 16792 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merritt Island, FL", "id": "13fa221b2b90ae80", "name": "Merritt Island", "place_type": "city", "bounding_box": rectangle("-80.731463,28.142537 -80.60386,28.408742") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12009, "countyName": "Brevard", "cityID": 1244275, "cityName": "Merritt Island" } }
+{ "create_at": datetime("2015-11-25T10:00:20.000Z"), "id": 669576367122464768, "text": "my time gone come �� https://t.co/1qOhlxwlAi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 615869147, "name": "goldie ⚡️", "screen_name": "AlmightyBrvdy", "lang": "en", "location": "Houston, TX", "create_at": date("2012-06-22"), "description": "add me on sc: brvvvdy | #SHSU19", "followers_count": 857, "friends_count": 589, "statues_count": 9033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:20.000Z"), "id": 669576367315390464, "text": "Today was the first time I slept in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2307922502, "name": "A.", "screen_name": "Asariah2blessed", "lang": "en", "location": "Pearsall, Texas", "create_at": date("2014-01-23"), "description": "somewhere running from the cops.", "followers_count": 469, "friends_count": 508, "statues_count": 12832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pearsall, TX", "id": "8e0fcb138f08c738", "name": "Pearsall", "place_type": "city", "bounding_box": rectangle("-99.125353,28.8701 -99.072782,28.911642") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48163, "countyName": "Frio", "cityID": 4856384, "cityName": "Pearsall" } }
+{ "create_at": datetime("2015-11-25T10:00:20.000Z"), "id": 669576367332200449, "text": "My birthday is tomorrow!!!!! ✨✨✨✨", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3538833672, "name": "Yung Squanto", "screen_name": "VivaLaJane_", "lang": "en", "location": "Dallas, TX", "create_at": date("2015-09-12"), "description": "YOUS a bitch", "followers_count": 77, "friends_count": 76, "statues_count": 330 }, "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-25T10:00:20.000Z"), "id": 669576368095698944, "text": "Kiana drive so slow omg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.186397,36.8815467"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1409351400, "name": "Mya Davis", "screen_name": "myadaviss", "lang": "en", "location": "null", "create_at": date("2013-05-06"), "description": "Black Lives Matter, Always.", "followers_count": 1327, "friends_count": 1703, "statues_count": 16493 }, "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-25T10:00:20.000Z"), "id": 669576368192032769, "text": "I wanna go to the JB concert ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2273457362, "name": "katie", "screen_name": "_ksmithhh_", "lang": "en", "location": "iowa", "create_at": date("2014-01-02"), "description": "jeremiah 29:11", "followers_count": 382, "friends_count": 470, "statues_count": 2059 }, "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("2015-11-25T10:00:21.000Z"), "id": 669576368372342784, "text": "@lisamb44ny Yes It is! I posted it���� https://t.co/NUWb9ImrzZ", "in_reply_to_status": -1, "in_reply_to_user": 3302197661, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3302197661 }}, "user": { "id": 3942951853, "name": "Bobbi Reno", "screen_name": "bobbi_Reno", "lang": "en", "location": "Minnesota, USA", "create_at": date("2015-10-18"), "description": "null", "followers_count": 84, "friends_count": 80, "statues_count": 2257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lonsdale, MN", "id": "0072079038d1474a", "name": "Lonsdale", "place_type": "city", "bounding_box": rectangle("-93.448004,44.464569 -93.408047,44.493659") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27131, "countyName": "Rice", "cityID": 2738150, "cityName": "Lonsdale" } }
+{ "create_at": datetime("2015-11-25T10:00:21.000Z"), "id": 669576368376578048, "text": "Partly cloudy this afternoon, high 65 (18 C). Low 52 (11 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433972, "name": "Memphis Weather", "screen_name": "MemphisWX", "lang": "en", "location": "Memphis, TN", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Memphis, TN", "followers_count": 1687, "friends_count": 88, "statues_count": 7625 }, "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("2015-11-25T10:00:21.000Z"), "id": 669576368670183424, "text": "#TerribleStockingStuffers \nA picture of yourself twenty years ago", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TerribleStockingStuffers" }}, "user": { "id": 2575517041, "name": "Jenzy Jen Jenzinita", "screen_name": "47young1", "lang": "en", "location": "null", "create_at": date("2014-06-18"), "description": "Mom.Wife.Headbanger. Old teacher w/a wish to reform education. I was here b4 computers, thank God. @ClintSmitthIII is my inspiration. Cancer survivor.", "followers_count": 2472, "friends_count": 2667, "statues_count": 58827 }, "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("2015-11-25T10:00:21.000Z"), "id": 669576369001508864, "text": "Yelling about comics on twitter, which it may have been invented for.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 668423, "name": "Mikeal Rogers", "screen_name": "mikeal", "lang": "en", "location": "San Francisco, CA", "create_at": date("2007-01-19"), "description": "Anarcho-hipster, JS Community Organizer. Creator of NodeConf & JSFest. Community @ Linux/Node.js Foundation.", "followers_count": 12389, "friends_count": 445, "statues_count": 36412 }, "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-25T10:00:21.000Z"), "id": 669576369035083777, "text": "I just wanna look 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": 2596959673, "name": "Crystal❥", "screen_name": "chmpagnecrystal", "lang": "en", "location": "281-210 Texass", "create_at": date("2014-06-30"), "description": "-from the jungle...spurs.cowboys.chuck e cheese. 11.12.13 5.24.15", "followers_count": 194, "friends_count": 157, "statues_count": 14547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:21.000Z"), "id": 669576369110544385, "text": "I'll still keep you, dancing by the moonlight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2594873586, "name": "JJA98", "screen_name": "J_Anzaldo18", "lang": "en", "location": "Azusa, CA", "create_at": date("2014-06-29"), "description": "《http://God.Family.Vanessa.Fitness.Football》", "followers_count": 462, "friends_count": 449, "statues_count": 8268 }, "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("2015-11-25T10:00:21.000Z"), "id": 669576369660014593, "text": "NEW KING KRULE PROJECT FUCK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1302115147, "name": "isaiah", "screen_name": "IsaiahSmith311", "lang": "en", "location": "Fairport, NY", "create_at": date("2013-03-25"), "description": "jump off the roof // isaiahsmith58599@gmail.com", "followers_count": 1047, "friends_count": 1499, "statues_count": 39994 }, "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("2015-11-25T10:00:21.000Z"), "id": 669576370364784640, "text": "If I got paid a dollar by Twitter for every RT I got I could easily make a living on here. Easily", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 555517603, "name": "#BadThingsHappen", "screen_name": "VersaceSilk", "lang": "en", "location": "Jersey", "create_at": date("2012-04-16"), "description": "Don't ask so many questions.", "followers_count": 8235, "friends_count": 1170, "statues_count": 69821 }, "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("2015-11-25T10:00:21.000Z"), "id": 669576370440175617, "text": "Ich bin sicher dass Troye und Connor wissen kein Deutsch haha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user": { "id": 2852023061, "name": "Elizabeth //86", "screen_name": "simpsonsivan", "lang": "en", "location": "Minnesota, US", "create_at": date("2014-10-30"), "description": "So many people have saved my life, but they do not even know my name.", "followers_count": 740, "friends_count": 278, "statues_count": 19860 }, "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("2015-11-25T10:00:21.000Z"), "id": 669576370603868162, "text": "ESTUPIDA Y SI ME LLEVAN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 1280624300, "name": "tá", "screen_name": "thaliaanette", "lang": "es", "location": "null", "create_at": date("2013-03-19"), "description": "sc:thaliaanette", "followers_count": 679, "friends_count": 404, "statues_count": 12567 }, "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("2015-11-25T10:00:21.000Z"), "id": 669576371019059200, "text": "������ my man > your man \nNo debate https://t.co/tGEKKru9Bx", "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 and gave birth to @hsmitty3", "followers_count": 8421, "friends_count": 828, "statues_count": 296483 }, "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("2015-11-25T10:00:21.000Z"), "id": 669576371065106432, "text": "#The Cuban Cafe https://t.co/lYrIQqMIiy great https://t.co/2NOAgkffCk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.08513613,26.38118064"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "The" }}, "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": 4131, "friends_count": 876, "statues_count": 329961 }, "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("2015-11-25T10:00:21.000Z"), "id": 669576371107041280, "text": "#The Cuban Cafe / https://t.co/lYrIQqMIiy great https://t.co/KDqMKTobsq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.08513613,26.38118064"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "The" }}, "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": 4131, "friends_count": 876, "statues_count": 329961 }, "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("2015-11-25T10:00:21.000Z"), "id": 669576371283202048, "text": "@PirateRadioo soon?", "in_reply_to_status": 669576170103373824, "in_reply_to_user": 218240457, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 218240457 }}, "user": { "id": 419237432, "name": "Maira Luevano", "screen_name": "_mairaalexandra", "lang": "en", "location": "Where The Wild Things Are", "create_at": date("2011-11-22"), "description": "Mauro Rolando Gomez; My Forever & Always❤️", "followers_count": 231, "friends_count": 132, "statues_count": 13870 }, "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-25T10:00:21.000Z"), "id": 669576371509817344, "text": "Feels good to be back home wit my guys #OMB4L", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "OMB4L" }}, "user": { "id": 194809215, "name": "John Williams", "screen_name": "JWillClutch_1", "lang": "en", "location": "Alvernia University", "create_at": date("2010-09-24"), "description": "TPOD 14' | AU Basketball | 302 | 610", "followers_count": 1509, "friends_count": 991, "statues_count": 88302 }, "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("2015-11-25T10:00:21.000Z"), "id": 669576371744673792, "text": "That's 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": 1301974584, "name": "2K", "screen_name": "SOPVT", "lang": "en", "location": "null", "create_at": date("2013-03-25"), "description": "PVTUSA #freesim #freekeem #freehellrell #freezalo", "followers_count": 727, "friends_count": 520, "statues_count": 16393 }, "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("2015-11-25T10:00:21.000Z"), "id": 669576371774078980, "text": "Shout out to my ex she a fool with that neck* -@__MG9__ https://t.co/UNJajToVch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1104884484 }}, "user": { "id": 546065280, "name": "steph freebandz ", "screen_name": "__Steph22", "lang": "en", "location": "CMU 2019", "create_at": date("2012-04-05"), "description": "A wise man once said- If a man does not have the sauce, he is lost. But the same man can be lost in the sauce. RIP Bonnie Brush. #FREEAARONHERNANDEZ #VoteBernie", "followers_count": 1562, "friends_count": 1132, "statues_count": 24840 }, "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-25T10:00:21.000Z"), "id": 669576371836862464, "text": "TBH I JUST DRAW THE HAIRSTYLE IM THINKIN ABOUT AT 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": 4126321933, "name": "duxheckler69", "screen_name": "smoldux", "lang": "en", "location": " saru or ryan .♂", "create_at": date("2015-11-04"), "description": "this is like how to catch a predator, 360 edition ☆ nattie is beautiful and i love her more than anything ☆ matching with @punkpoptart ☆ ig: bokubro", "followers_count": 119, "friends_count": 103, "statues_count": 486 }, "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("2015-11-25T10:00:21.000Z"), "id": 669576372096905216, "text": "�� RT @OGBEARD: Salute to @boulderbeardco best beard oil I've received so far in my bearded career. Smells amazing !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.365163,29.741435"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 207203414, 3194480394 }}, "user": { "id": 261469386, "name": "Word Link", "screen_name": "LordWink", "lang": "en", "location": "H-Tine•#NinerEmpire•#TeamEther", "create_at": date("2011-03-05"), "description": "Stop retweeting and faving my compliments and me laughing at your pics you pilfered from your TL to seem like it's yours. I ain't your groupie", "followers_count": 9699, "friends_count": 7303, "statues_count": 259103 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:21.000Z"), "id": 669576372440821760, "text": "@_isfake regarding church ������", "in_reply_to_status": 669568969737572352, "in_reply_to_user": 2381100036, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2381100036 }}, "user": { "id": 3173007679, "name": "abigail ⚽️", "screen_name": "alwayysabbyy", "lang": "en", "location": "null", "create_at": date("2015-04-25"), "description": "shs soccer #16", "followers_count": 373, "friends_count": 485, "statues_count": 3889 }, "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("2015-11-25T10:00:22.000Z"), "id": 669576372625371136, "text": "@Aaron_Glo @MacMiller Wild Rush", "in_reply_to_status": 669576118182088705, "in_reply_to_user": 380100662, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 380100662, 23065354 }}, "user": { "id": 2241840919, "name": "Sara Leguizamon", "screen_name": "saravictoria_24", "lang": "en", "location": "null", "create_at": date("2013-12-11"), "description": "Colombian • OSU '17 stilly • 08.15.14 S ❤️", "followers_count": 459, "friends_count": 433, "statues_count": 14602 }, "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("2015-11-25T10:00:22.000Z"), "id": 669576372675850240, "text": "I ❤️ @kaylyn_noel avi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 136082287 }}, "user": { "id": 175280137, "name": "Lex", "screen_name": "AlexxaCrosby", "lang": "en", "location": "null", "create_at": date("2010-08-05"), "description": "bienvenido a mi mundo", "followers_count": 875, "friends_count": 708, "statues_count": 20434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Englewood, OH", "id": "b152589e47844e57", "name": "Englewood", "place_type": "city", "bounding_box": rectangle("-84.344742,39.832662 -84.22556,39.891704") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39113, "countyName": "Montgomery", "cityID": 3925396, "cityName": "Englewood" } }
+{ "create_at": datetime("2015-11-25T10:00:22.000Z"), "id": 669576373044948992, "text": "@Nadeenasnake @Skype you mean is shit love.\n\nI have more faith in @telegram", "in_reply_to_status": 669576154630746117, "in_reply_to_user": 2943626679, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2943626679, 2459371, 1689053928 }}, "user": { "id": 1857195152, "name": "Rocky Cage", "screen_name": "RockawayCarter", "lang": "en", "location": "Brooklyn NY", "create_at": date("2013-09-12"), "description": "Furaffinity's resident anti-FGC fighting game veteran, social furry and all around cynical black guy.\r\n\r\nPSN: Sabrerockaway", "followers_count": 454, "friends_count": 1007, "statues_count": 13053 }, "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-25T10:00:22.000Z"), "id": 669576373124624384, "text": "I honestly find it offensive when a basic bitch had the audacity to post biggie lyrics lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 104994709, "name": "Eleonora", "screen_name": "ellie_alessio", "lang": "en", "location": "704", "create_at": date("2010-01-14"), "description": "Aiutati che Dio t’aiuta.", "followers_count": 569, "friends_count": 273, "statues_count": 18260 }, "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("2015-11-25T10:00:22.000Z"), "id": 669576373627805696, "text": "Partly cloudy this afternoon, high 63 (17 C). Low 45 (7 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119697197, "name": "Huntsville Weather", "screen_name": "HuntsvilleWX", "lang": "en", "location": "null", "create_at": date("2010-03-04"), "description": "null", "followers_count": 665, "friends_count": 65, "statues_count": 5936 }, "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("2015-11-25T10:00:22.000Z"), "id": 669576373837520897, "text": "\"Let's get more mimosas\" \"but we have to take Harrison to school!\" \"Harrison can you uber?\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2686566974, "name": "Ellie Langenfeld", "screen_name": "EllieLangenfeld", "lang": "en", "location": "null", "create_at": date("2014-07-27"), "description": "luv u", "followers_count": 228, "friends_count": 253, "statues_count": 1808 }, "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("2015-11-25T10:00:22.000Z"), "id": 669576373875314688, "text": "Bae is the realist for getting me tacos from Los Gallos ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 717296622, "name": "Joe Angel", "screen_name": "Juicyy_Joe", "lang": "en", "location": "null", "create_at": date("2012-07-25"), "description": "Welcome to the good life| sc:juicyyyyjoe | insta: thejuicyjoe_", "followers_count": 986, "friends_count": 473, "statues_count": 11878 }, "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("2015-11-25T10:00:22.000Z"), "id": 669576374018027520, "text": "Swollen, red, and pissed off. Sounds about right for the DOOMICORN!!! #docholliday #unicorn #flames… https://t.co/Pbu3vPclq3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.3856201,33.9605789"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "docholliday", "unicorn", "flames" }}, "user": { "id": 25573447, "name": "Sara Machen", "screen_name": "grumppopotami", "lang": "en", "location": "null", "create_at": date("2009-03-20"), "description": "null", "followers_count": 54, "friends_count": 203, "statues_count": 807 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Athens, GA", "id": "01f9c9fd7bb1aa6b", "name": "Athens", "place_type": "city", "bounding_box": rectangle("-83.521866,33.877554 -83.277693,34.035918") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13059, "countyName": "Clarke", "cityID": 1303440, "cityName": "Athens-Clarke County unified government (balance)" } }
+{ "create_at": datetime("2015-11-25T10:00:22.000Z"), "id": 669576374093385728, "text": "I want it all, half was never the agreement.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2533545716, "name": "Jalen Darden", "screen_name": "darden_jalen", "lang": "en", "location": "null", "create_at": date("2014-05-29"), "description": "RIP JOHNNY.", "followers_count": 331, "friends_count": 391, "statues_count": 599 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Robinson, TX", "id": "fc7c53c16999dfc8", "name": "Robinson", "place_type": "city", "bounding_box": rectangle("-97.1445,31.448051 -97.095627,31.507322") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4862588, "cityName": "Robinson" } }
+{ "create_at": datetime("2015-11-25T10:00:22.000Z"), "id": 669576374273908740, "text": "@kelsedwards I gotta work Friday and Saturday", "in_reply_to_status": 669573879803813888, "in_reply_to_user": 404924424, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 404924424 }}, "user": { "id": 368721453, "name": "Lauren Baxter", "screen_name": "Lourrennn", "lang": "en", "location": "null", "create_at": date("2011-09-05"), "description": "I IX MMX", "followers_count": 606, "friends_count": 438, "statues_count": 16705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beavercreek, OH", "id": "050c276f7bcbb37e", "name": "Beavercreek", "place_type": "city", "bounding_box": rectangle("-84.106764,39.676687 -83.988342,39.779185") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39057, "countyName": "Greene", "cityID": 3904720, "cityName": "Beavercreek" } }
+{ "create_at": datetime("2015-11-25T10:00:22.000Z"), "id": 669576374282121217, "text": "Love yourself by Justin Bieber��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 315127788, "name": "TANNA", "screen_name": "Montanna_Love1", "lang": "en", "location": "null", "create_at": date("2011-06-11"), "description": "Mind your own biscuits and life will be gravy.☀️ ||Marissa is my Bff.", "followers_count": 931, "friends_count": 574, "statues_count": 6865 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sun City West, AZ", "id": "859948b6ebcbbf25", "name": "Sun City West", "place_type": "city", "bounding_box": rectangle("-112.409262,33.638577 -112.311175,33.706973") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 470355, "cityName": "Sun City West" } }
+{ "create_at": datetime("2015-11-25T10:00:22.000Z"), "id": 669576374517002240, "text": "thankful as fuck to be back in the greatest city on earth", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 237875775, "name": "meghan", "screen_name": "mburns23", "lang": "en", "location": "West Roxbury, MA", "create_at": date("2011-01-13"), "description": "dad bod enthusiast #blessed", "followers_count": 421, "friends_count": 491, "statues_count": 14838 }, "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("2015-11-25T10:00:22.000Z"), "id": 669576374605058049, "text": "#Labor #Job in #Norfolk, Virginia: Gas Station Team Member at BJs Wholesale Club https://t.co/bO0KIU6jtY #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.2858726,36.8507689"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Labor", "Job", "Norfolk", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22779528, "name": "TMJ-ORF Labor Jobs", "screen_name": "tmj_orf_labor", "lang": "en", "location": "Norfolk area, VA", "create_at": date("2009-03-04"), "description": "Follow this account for geo-targeted General Labor job tweets in Norfolk area, VA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 300, "friends_count": 290, "statues_count": 12 }, "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("2015-11-25T10:00:22.000Z"), "id": 669576374617788416, "text": "I love the Sooners", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 109378617, "name": "Cubbie Woods.", "screen_name": "C_Marchio2", "lang": "en", "location": "The 5 †", "create_at": date("2010-01-28"), "description": "The Boy.", "followers_count": 664, "friends_count": 313, "statues_count": 43112 }, "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("2015-11-25T10:00:22.000Z"), "id": 669576374634586112, "text": "when I die, I hope I'm remembered for my spotify playlists.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 233747425, "name": "Carlee Fultz", "screen_name": "carlsfultz", "lang": "en", "location": "Wilmington, NC", "create_at": date("2011-01-03"), "description": "grab a slice of 'za, bruh", "followers_count": 435, "friends_count": 290, "statues_count": 4819 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Troutman, NC", "id": "00dabb8d63f0dae3", "name": "Troutman", "place_type": "city", "bounding_box": rectangle("-80.914428,35.658036 -80.869126,35.725133") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37097, "countyName": "Iredell", "cityID": 3768500, "cityName": "Troutman" } }
+{ "create_at": datetime("2015-11-25T10:00:22.000Z"), "id": 669576374735081473, "text": "I'm thankful for delicious coffee, good conversation and awesome women like @nyikaaa CC: @thegrovecafe https://t.co/gt8TFQAt4p", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 80963031, 61088027 }}, "user": { "id": 23077455, "name": "Annemarie C. Henton", "screen_name": "ACHenton", "lang": "en", "location": "Albuquerque, New Mexico", "create_at": date("2009-03-06"), "description": "Lover of high heels, snowboarding, acoustic guitar, coffee, @UofA. Comms & Marketing Director @ABQEconDev. PM for @ABQiteam. Opinions = mine. #iteams #econdev", "followers_count": 1520, "friends_count": 1476, "statues_count": 5022 }, "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("2015-11-25T10:00:22.000Z"), "id": 669576374764503041, "text": "good afternoon mentions? RT/FAV ❤", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1959579655, "name": "♡☁ 19.7k ♡☁", "screen_name": "itscrewsworldTV", "lang": "en", "location": "Houston, TX", "create_at": date("2013-10-13"), "description": "17 | recording artist | song-writer | actress | promoter | model | booking-business inquires contact: crewloveeemusic2@gmail.com | #TheCrewTeam | #RIPRikoV ♡♕.", "followers_count": 19755, "friends_count": 8808, "statues_count": 110303 }, "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("2015-11-25T10:00:22.000Z"), "id": 669576374810574849, "text": "I hate nail shops but my nails have to stay done", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3247420262, "name": "ling ling", "screen_name": "dupreecharity01", "lang": "en", "location": "Dallas, TX", "create_at": date("2015-06-16"), "description": "null", "followers_count": 178, "friends_count": 98, "statues_count": 862 }, "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("2015-11-25T10:00:22.000Z"), "id": 669576374978375681, "text": "I need new music cause I'm SO over listening to the same shit that's on my playlist over and over again.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 473524022, "name": "Gabriel", "screen_name": "go0ley", "lang": "en", "location": "null", "create_at": date("2012-01-24"), "description": "I'm a trainer, I like carrot cake and I probably don't like you. Houston, Texas. ✉️: gabe.gooley@gmail.com", "followers_count": 4364, "friends_count": 502, "statues_count": 87714 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:22.000Z"), "id": 669576375347597312, "text": "Yeah life is hard when erbody you meet ain een got they shit together.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1199422268, "name": "幽霊ピザ (275ghostpizza)", "screen_name": "mvrrky_mvck", "lang": "en", "location": "Newport News, Virginia", "create_at": date("2013-02-19"), "description": "i wont disappoint or disappear. -The Real Ghostpizza", "followers_count": 1347, "friends_count": 388, "statues_count": 27639 }, "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("2015-11-25T10:00:22.000Z"), "id": 669576375355858944, "text": "Running from the cops and getting taized when in your head your to high and you think your getting shot �� https://t.co/dZDG7Evp1X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2305963027, "name": "Jojo", "screen_name": "Joanna_aceves12", "lang": "en", "location": "Salinas, CA ", "create_at": date("2014-01-22"), "description": "@xxx_brayan / 10'25'15 ❤️ / ehs 18", "followers_count": 2152, "friends_count": 1145, "statues_count": 15465 }, "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("2015-11-25T10:00:22.000Z"), "id": 669576375720804352, "text": "@NormsClubhouse have you ever had chocolate cake and gravy?", "in_reply_to_status": -1, "in_reply_to_user": 51240452, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 51240452 }}, "user": { "id": 334152445, "name": "Kim Martin Jones", "screen_name": "KimCMJones", "lang": "en", "location": "Grapevine, TX", "create_at": date("2011-07-12"), "description": "If you obey all the rules, you miss all the fun", "followers_count": 53, "friends_count": 48, "statues_count": 209 }, "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("2015-11-25T10:00:22.000Z"), "id": 669576375834173440, "text": "Captain americas trailor looks awesome!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 446463241, "name": "chuck tweets", "screen_name": "ChuckHunnas", "lang": "en", "location": "wherever a yung hunna be", "create_at": date("2011-12-25"), "description": "lebron time", "followers_count": 678, "friends_count": 1236, "statues_count": 17137 }, "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("2015-11-25T10:00:22.000Z"), "id": 669576376081653761, "text": "@Irate_skeleton give in", "in_reply_to_status": 669576156534804480, "in_reply_to_user": 3014526716, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3014526716 }}, "user": { "id": 2953043526, "name": "looney tune yaoi", "screen_name": "autistickanae", "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. frequently is nsfw. (she/her)", "followers_count": 300, "friends_count": 131, "statues_count": 51801 }, "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-25T10:00:22.000Z"), "id": 669576376190545920, "text": "Sunny this afternoon, high 64 (18 C). Low 47 (8 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119698223, "name": "Birmingham Weather", "screen_name": "BirminghamWX", "lang": "en", "location": "Birmingham, AL", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Birmingham, AL", "followers_count": 502, "friends_count": 68, "statues_count": 5948 }, "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("2015-11-25T10:00:22.000Z"), "id": 669576376354217984, "text": "You wanted me to text you so I do and then I don't get a reply back��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2938940428, "name": "Rylee Keeler", "screen_name": "rtkeeler3", "lang": "en", "location": "null", "create_at": date("2014-12-23"), "description": "snapchat- rkeeler_10", "followers_count": 122, "friends_count": 418, "statues_count": 546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sayre, PA", "id": "2d762631bc385f6a", "name": "Sayre", "place_type": "city", "bounding_box": rectangle("-76.549733,41.964859 -76.504695,41.999893") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42015, "countyName": "Bradford", "cityID": 4268096, "cityName": "Sayre" } }
+{ "create_at": datetime("2015-11-25T10:00:22.000Z"), "id": 669576376513638400, "text": "TY Stephanie! @KindandInspired @collrin @JillCofskyVoice @chloepink @LovableLobo @TheSmartChic @dchicdesigns So grateful for amazing friends", "in_reply_to_status": 669576129787895808, "in_reply_to_user": 700129868, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 700129868, 942624170, 2248829933, 47376382, 998018965, 339936927, 100858779 }}, "user": { "id": 1111940263, "name": "Gomee Girl™", "screen_name": "GomeeGirl", "lang": "en", "location": "New Hampshire", "create_at": date("2013-01-22"), "description": "Gomee Girl is the fun, fashionable way for girls to express themselves! Inspiring girls to love themselves just the way they are! Launch: 2015 #GoBeYOU™", "followers_count": 3695, "friends_count": 3186, "statues_count": 23425 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manchester, NH", "id": "5e6cc87c609e3a59", "name": "Manchester", "place_type": "city", "bounding_box": rectangle("-71.512722,42.890919 -71.375439,43.051545") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33011, "countyName": "Hillsborough", "cityID": 3345140, "cityName": "Manchester" } }
+{ "create_at": datetime("2015-11-25T10:00:23.000Z"), "id": 669576377193074688, "text": "Agr que comi posso morrer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 1264871366, "name": "Toni Wihelns", "screen_name": "teddwh", "lang": "pt", "location": "Boston, MA", "create_at": date("2013-03-13"), "description": "E daí, um dia eu vou morrer mesmo.", "followers_count": 2071, "friends_count": 480, "statues_count": 5830 }, "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("2015-11-25T10:00:23.000Z"), "id": 669576377352503298, "text": "#MTVStars One Direction https://t.co/Qi17csdMLD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MTVStars" }}, "user": { "id": 2890110154, "name": "Gma Jan", "screen_name": "GmaJV40", "lang": "en", "location": "Michigan US", "create_at": date("2014-11-23"), "description": "Registered Nurse, Maternity. Never regret getting older, it is a privilege denied to many.\nLoves sewing, music. One Direction and BonJovi", "followers_count": 72, "friends_count": 98, "statues_count": 12407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Traverse City, MI", "id": "25d884784990a9af", "name": "Traverse City", "place_type": "city", "bounding_box": rectangle("-85.654331,44.697537 -85.543799,44.821578") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26055, "countyName": "Grand Traverse", "cityID": 2680340, "cityName": "Traverse City" } }
+{ "create_at": datetime("2015-11-25T10:00:23.000Z"), "id": 669576377457184768, "text": "Store Manager - Dollar General Corporation: (IN) https://t.co/aZFHvuzL1F #Retail #Diversity #Veterans #Job #Jobs https://t.co/a5jqCtfylZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.735897,30.14763"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "Diversity", "Veterans", "Job", "Jobs" }}, "user": { "id": 110827861, "name": "DollarGeneralCareers", "screen_name": "DGCareers", "lang": "en", "location": "Nashville, TN", "create_at": date("2010-02-02"), "description": "Let’s Grow Together!\nAt 12,000 stores and counting, we're growing fast and so can you! Joining Dollar General is just the beginning of your best career yet.", "followers_count": 934, "friends_count": 726, "statues_count": 16426 }, "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": 12005, "countyName": "Bay", "cityID": 1241562, "cityName": "Lower Grand Lagoon" } }
+{ "create_at": datetime("2015-11-25T10:00:23.000Z"), "id": 669576377478328321, "text": "Not my nigga lil Boosie ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 72173106, "name": ".", "screen_name": "_Pheeel", "lang": "en", "location": "Winston-Salem, NC", "create_at": date("2009-09-06"), "description": "#WSSU18 , #FreeQ #FreeVick rest easy BARRY WRIGHT #RideTheWave rest in heaven @LLCoolTre Long Live ANT", "followers_count": 3470, "friends_count": 3532, "statues_count": 64608 }, "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-25T10:00:23.000Z"), "id": 669576377482506240, "text": "I observe the real you say you got in you, soon as shit hit left in a major way my whole mindset change about you. https://t.co/XLQOCBUaCp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 543506363, "name": "Key", "screen_name": "ItsLakesha", "lang": "en", "location": "Jamaica", "create_at": date("2012-04-02"), "description": "I am Key, I am the blessing God put here to make a difference ❤️.", "followers_count": 5854, "friends_count": 6042, "statues_count": 104485 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Country Club Estates, GA", "id": "c9edca327397dad5", "name": "Country Club Estates", "place_type": "city", "bounding_box": rectangle("-81.483157,31.189838 -81.435287,31.229823") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13127, "countyName": "Glynn", "cityID": 1319830, "cityName": "Country Club Estates" } }
+{ "create_at": datetime("2015-11-25T10:00:23.000Z"), "id": 669576377599836160, "text": "*dances like Janet*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27279153, "name": "Ramona Royale", "screen_name": "_WlkinParadox", "lang": "en", "location": "Behind My Camera", "create_at": date("2009-03-28"), "description": "She stays in the coolest moods. Clearly, women of the year. #AiP", "followers_count": 1427, "friends_count": 597, "statues_count": 243895 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaver Falls, PA", "id": "579173dcb66e54d5", "name": "Beaver Falls", "place_type": "city", "bounding_box": rectangle("-80.335207,40.739691 -80.309549,40.790082") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42007, "countyName": "Beaver", "cityID": 4204792, "cityName": "Beaver Falls" } }
+{ "create_at": datetime("2015-11-25T10:00:23.000Z"), "id": 669576377830662148, "text": "I be so high I be tripping 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": 1227993650, "name": "Adnawiek❤", "screen_name": "Keiwandaaaa__", "lang": "en", "location": "null", "create_at": date("2013-02-28"), "description": "null", "followers_count": 4846, "friends_count": 4721, "statues_count": 69961 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grayson Valley, AL", "id": "f433388eec4f285d", "name": "Grayson Valley", "place_type": "city", "bounding_box": rectangle("-86.662184,33.634058 -86.626743,33.667244") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1073, "countyName": "Jefferson", "cityID": 131348, "cityName": "Grayson Valley" } }
+{ "create_at": datetime("2015-11-25T10:00:23.000Z"), "id": 669576378375733250, "text": "Playing NBA jam", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2923313942, "name": "mamacita", "screen_name": "shvywhvt", "lang": "en", "location": "BIGTEXA$", "create_at": date("2014-12-08"), "description": "intellectual at its best", "followers_count": 669, "friends_count": 650, "statues_count": 12633 }, "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("2015-11-25T10:00:23.000Z"), "id": 669576378426093568, "text": "@alexaisugly17 I thought what we had was real..", "in_reply_to_status": 669569802948624385, "in_reply_to_user": 1345871720, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1345871720 }}, "user": { "id": 1376156569, "name": "Diana Millan", "screen_name": "Dmillan05", "lang": "en", "location": "csun'20", "create_at": date("2013-04-23"), "description": "Real So Cal ECNL⚽️ •Everything happens for a reason•", "followers_count": 496, "friends_count": 650, "statues_count": 4731 }, "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("2015-11-25T10:00:23.000Z"), "id": 669576378723909633, "text": "Mostly sunny this afternoon, high 70 (21 C). Low 57 (14 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119698549, "name": "Mobile Weather", "screen_name": "MobileWX", "lang": "en", "location": "Mobile, Al", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Mobile, Al", "followers_count": 387, "friends_count": 68, "statues_count": 6023 }, "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("2015-11-25T10:00:23.000Z"), "id": 669576378900070401, "text": "@andrewlainjr @CNN Not WWIII . Just Russia not complying with international protocol as usual .", "in_reply_to_status": 669551657449992193, "in_reply_to_user": 38710621, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 38710621, 759251 }}, "user": { "id": 1120641384, "name": "Chad Clement", "screen_name": "cclement873", "lang": "en", "location": "Houston, TX.", "create_at": date("2013-01-25"), "description": "Texas conservative ,southern baptist christian, #CruzCrew #Cruz2016 #2A #CCOT #TCOT #NRA #PJNET #StandwithIsrael", "followers_count": 1465, "friends_count": 2110, "statues_count": 3828 }, "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("2015-11-25T10:00:23.000Z"), "id": 669576379407568897, "text": "why you got weed in ya crack hoe ☹�������� https://t.co/1C8K7W5L4e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 354768473, "name": "ryan", "screen_name": "ryansdope", "lang": "en", "location": "limbo", "create_at": date("2011-08-14"), "description": "fuckin shit up wbu", "followers_count": 1084, "friends_count": 400, "statues_count": 117128 }, "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("2015-11-25T10:00:23.000Z"), "id": 669576379466448898, "text": "Bridge ¿?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 471434043, "name": "rav", "screen_name": "ravenhiggins4", "lang": "en", "location": "Mayfair, Philadelphia", "create_at": date("2012-01-22"), "description": "I like long walks on the beach and big penis.", "followers_count": 1295, "friends_count": 2023, "statues_count": 69345 }, "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-25T10:00:23.000Z"), "id": 669576379583758336, "text": "Real theater. https://t.co/zTOQYDrcVW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.193235,33.764557"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3229260908, "name": "Sabrina Del Bello", "screen_name": "SabrinaSueDB", "lang": "en", "location": "null", "create_at": date("2015-05-28"), "description": "LB Wilson Varsity XC Captain⛵️", "followers_count": 109, "friends_count": 121, "statues_count": 135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cinemark At The Pike", "id": "07d9e3abf5c83002", "name": "Cinemark At The Pike", "place_type": "poi", "bounding_box": rectangle("-118.1932351,33.7645569 -118.193235,33.764557") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2015-11-25T10:00:23.000Z"), "id": 669576380343037952, "text": "Baby now we got bad blood��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 599792227, "name": "Jeremy", "screen_name": "Jeremyylee_7", "lang": "en", "location": "Georgia, USA", "create_at": date("2012-06-04"), "description": "Jeremiah 29:11 || hoco football", "followers_count": 463, "friends_count": 292, "statues_count": 6615 }, "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-25T10:00:23.000Z"), "id": 669576380800065536, "text": "@_Reetheloc him and Casey dropping a whole mixtape together", "in_reply_to_status": 669576238332182528, "in_reply_to_user": 2620428362, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2620428362 }}, "user": { "id": 1169712512, "name": "Jiggy㊙️", "screen_name": "DOPEITSJAYEE", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-02-11"), "description": "I'm not allergic to cats I just don't fuck with u cats", "followers_count": 2300, "friends_count": 832, "statues_count": 96021 }, "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("2015-11-25T10:00:24.000Z"), "id": 669576381026672640, "text": "#High5 new JASPERNET WIFI updates - faster speed test results. Much better @MC_ITS https://t.co/93rATMiHZc https://t.co/1REoL2jlkL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "High5" }}, "user_mentions": {{ 538230299 }}, "user": { "id": 538434794, "name": "Jake Holmquist", "screen_name": "HolmquistJake", "lang": "en", "location": "New York, NY", "create_at": date("2012-03-27"), "description": "Technology Evangelist, Change Agent, CIO @ManhattanEdu", "followers_count": 106, "friends_count": 133, "statues_count": 223 }, "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("2015-11-25T10:00:24.000Z"), "id": 669576381223731200, "text": "@befourodeo I definitely don't want your gorilla lookin ass ��", "in_reply_to_status": 669576182774411264, "in_reply_to_user": 1043419279, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1043419279 }}, "user": { "id": 2267169865, "name": "/\\/", "screen_name": "Andrevv_", "lang": "en", "location": "Hesperia ", "create_at": date("2013-12-29"), "description": "'Merica", "followers_count": 493, "friends_count": 255, "statues_count": 21484 }, "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("2015-11-25T10:00:24.000Z"), "id": 669576381345300480, "text": "I love how even after 9 years Valentin Elizalde is still remembered ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1271103920, "name": "dennise ❣", "screen_name": "udenniseee", "lang": "en", "location": "W$LB", "create_at": date("2013-03-15"), "description": "sdsu", "followers_count": 377, "friends_count": 212, "statues_count": 11506 }, "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("2015-11-25T10:00:24.000Z"), "id": 669576381370515456, "text": "Mostly cloudy this afternoon, high 52 (11 C). Low 48 (9 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119751854, "name": "Chicago Weather", "screen_name": "Chicago_WX", "lang": "en", "location": "Chicago, IL", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Chicago, IL", "followers_count": 1055, "friends_count": 68, "statues_count": 6305 }, "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-25T10:00:24.000Z"), "id": 669576381785874432, "text": "#Patience and #Wisdom Use them always. https://t.co/xhSOcTF4g0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Patience", "Wisdom" }}, "user": { "id": 2469884923, "name": "Celina Barajas", "screen_name": "CelinaBarajas1", "lang": "en", "location": "Chicago", "create_at": date("2014-04-29"), "description": "Chicago Artist (Animator/Illustrator) - Art is everywhere\nhttp://www.society6.com/celinabarajas http://www.redbubble.com/people/celinabarajas", "followers_count": 513, "friends_count": 1767, "statues_count": 8352 }, "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-25T10:00:24.000Z"), "id": 669576381915901952, "text": "@sonarriley @stuba2 https://t.co/brkg24lbHB", "in_reply_to_status": -1, "in_reply_to_user": 54731881, "favorite_count": 0, "coordinate": point("-83.750153,42.279341"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 54731881, 15865693 }}, "user": { "id": 169706206, "name": "Honey Boo Boo", "screen_name": "SSFCSnorlax", "lang": "en", "location": "Valparaiso, IN", "create_at": date("2010-07-22"), "description": "Married. Tattooed. Heartland Horde Admin and ECS Away Drummer. Full time brand ambassador. Part time Snorlax.", "followers_count": 636, "friends_count": 785, "statues_count": 58179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Krazy Jim's Blimpy Burger", "id": "07d9d18973487000", "name": "Krazy Jim's Blimpy Burger", "place_type": "poi", "bounding_box": rectangle("-83.75015309999999,42.2793409 -83.750153,42.279341") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26161, "countyName": "Washtenaw", "cityID": 2603000, "cityName": "Ann Arbor" } }
+{ "create_at": datetime("2015-11-25T10:00:24.000Z"), "id": 669576382356332544, "text": "@trinaaaxoxo @VeronikaaWalker ✌��️", "in_reply_to_status": 669575990264352769, "in_reply_to_user": 2536409297, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2536409297, 550528034 }}, "user": { "id": 1259542256, "name": "Arigato", "screen_name": "TSwellySwell", "lang": "en", "location": "City of Wind - Kronos ", "create_at": date("2013-03-11"), "description": "One World One Love 1700 EAT", "followers_count": 747, "friends_count": 538, "statues_count": 36012 }, "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-25T10:00:24.000Z"), "id": 669576382440185857, "text": "#BlackLivesMatter #LatinosUnidos #APIA #NativeAmerican #WarOnWomen #NoH8 #MuslimsAreNotTerorist #CPD #Chicago\n https://t.co/F67JMkF3Ck", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "BlackLivesMatter", "LatinosUnidos", "APIA", "NativeAmerican", "WarOnWomen", "NoH8", "MuslimsAreNotTerorist", "CPD", "Chicago" }}, "user": { "id": 1566650000, "name": "Raj Balasubramanian", "screen_name": "Solutioneer72", "lang": "en", "location": "Chicago, IL, USA", "create_at": date("2013-07-03"), "description": "After 14+ years as a surveillance target, I am now speaking out to get liberty, equality, and justice for all. Join me here, on Facebook, and at my website.", "followers_count": 4402, "friends_count": 3404, "statues_count": 118206 }, "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-25T10:00:24.000Z"), "id": 669576382519857152, "text": "@b696963 even better �� https://t.co/hQ8FQIGoWN", "in_reply_to_status": -1, "in_reply_to_user": 1322542093, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1322542093 }}, "user": { "id": 895412312, "name": "Niko", "screen_name": "the1real_niko", "lang": "en", "location": "West Bab represent ", "create_at": date("2012-10-21"), "description": "Live for happiness.. its not easy to come by Rangers.. Yankees.. Knicks.. Giants 7-11-15", "followers_count": 148, "friends_count": 249, "statues_count": 2455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Bay Shore, NY", "id": "f87e333473584780", "name": "North Bay Shore", "place_type": "city", "bounding_box": rectangle("-73.281541,40.74276 -73.228098,40.777154") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3651495, "cityName": "North Bay Shore" } }
+{ "create_at": datetime("2015-11-25T10:00:24.000Z"), "id": 669576382549131264, "text": "@diegosaurs_ go Diego, go !", "in_reply_to_status": -1, "in_reply_to_user": 1707791742, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1707791742 }}, "user": { "id": 3250003741, "name": "(d) Amen fizzy", "screen_name": "askingsenpai", "lang": "en", "location": "null", "create_at": date("2015-06-19"), "description": "hai", "followers_count": 203, "friends_count": 420, "statues_count": 440 }, "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("2015-11-25T10:00:24.000Z"), "id": 669576382695927808, "text": "kills me 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": 3188051845, "name": "gilly", "screen_name": "gilllato", "lang": "en", "location": "707", "create_at": date("2015-05-07"), "description": "null", "followers_count": 189, "friends_count": 216, "statues_count": 1731 }, "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("2015-11-25T10:00:24.000Z"), "id": 669576382947713026, "text": "I'm in love, such a pretty girl�� https://t.co/AOfevWoA7i", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 625192010, "name": "moon", "screen_name": "e_duarte16", "lang": "en", "location": "where i'm meant to be |IJD|", "create_at": date("2012-07-02"), "description": "There are two things in life you cannot hide. Sneezing and love", "followers_count": 843, "friends_count": 561, "statues_count": 92197 }, "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("2015-11-25T10:00:24.000Z"), "id": 669576383757094913, "text": "@julesdafool @itsbnastyy AHHH WTHH", "in_reply_to_status": 669576217633415168, "in_reply_to_user": 429489235, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 429489235, 615210566 }}, "user": { "id": 3183830991, "name": "Emma Dunlap", "screen_name": "emmakdunlap", "lang": "en", "location": "null", "create_at": date("2015-04-19"), "description": "its just me", "followers_count": 82, "friends_count": 95, "statues_count": 612 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Peachtree City, GA", "id": "fab01c4d14587744", "name": "Peachtree City", "place_type": "city", "bounding_box": rectangle("-84.626167,33.335655 -84.522236,33.449361") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13113, "countyName": "Fayette", "cityID": 1359724, "cityName": "Peachtree City" } }
+{ "create_at": datetime("2015-11-25T10:00:24.000Z"), "id": 669576383849361408, "text": "Partly cloudy this afternoon, high 71 (22 C). Low 53 (12 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119753180, "name": "Jackson Weather", "screen_name": "JacksonWX", "lang": "en", "location": "Jackson, MS", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Jackson, MS", "followers_count": 294, "friends_count": 68, "statues_count": 6254 }, "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("2015-11-25T10:00:24.000Z"), "id": 669576384159739909, "text": "@JanetteJaquez @EmilyEllinwood ������", "in_reply_to_status": 669563754238058496, "in_reply_to_user": 413568162, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 413568162, 2437401740 }}, "user": { "id": 320052725, "name": "Jaqueline❁", "screen_name": "sippin_xo", "lang": "en", "location": "Illinois ", "create_at": date("2011-06-18"), "description": "My dreams stay big and my worries stay small. 19. IG:Jaqueline357 ☾☮♡❋ snapchat:Jaqueline3570 Nursing college student.", "followers_count": 395, "friends_count": 207, "statues_count": 16694 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Silvis, IL", "id": "16e04a464cd5104d", "name": "Silvis", "place_type": "city", "bounding_box": rectangle("-90.432072,41.457399 -90.374431,41.522137") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17161, "countyName": "Rock Island", "cityID": 1769979, "cityName": "Silvis" } }
+{ "create_at": datetime("2015-11-25T10:00:24.000Z"), "id": 669576384319090688, "text": "me and my fuzzy polar bear socks ready to hit the road ☺️ https://t.co/OlxNa2xrWV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1239609367, "name": "☼ A S H L E Y ☼", "screen_name": "infinxtyswift", "lang": "en", "location": "omaha", "create_at": date("2013-03-03"), "description": "△ Your heart is the biggest gift you can give anyone △ fearless tour 8.9.09 speak now tour met TS 5.27.11 red tour 3.13.13 and 1989 tour 10.10.15", "followers_count": 436, "friends_count": 335, "statues_count": 4704 }, "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("2015-11-25T10:00:24.000Z"), "id": 669576384356814848, "text": "Thank you @Thom_Hartmann for being realistic. Bernie fans might be feeling the burn, but we must get a Dem in the WH and #ImWithHer #Hillyes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ImWithHer", "Hillyes" }}, "user_mentions": {{ 21414576 }}, "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": 1436, "friends_count": 2037, "statues_count": 32053 }, "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-25T10:00:24.000Z"), "id": 669576384428273664, "text": "Shoutout to soph on her birthday today ������ @sophie_dinehart https://t.co/bqj3JuVqby", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2737062760 }}, "user": { "id": 3867468196, "name": "Lyneseee", "screen_name": "lyncast2", "lang": "en", "location": "null", "create_at": date("2015-10-04"), "description": "null", "followers_count": 102, "friends_count": 112, "statues_count": 151 }, "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("2015-11-25T10:00:24.000Z"), "id": 669576384885297153, "text": "#Hollywood #audition #actor #Sag Aftra #headshot #lines #fitness #fun #happy https://t.co/yfQP3xMh4t", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hollywood", "audition", "actor", "Sag", "headshot", "lines", "fitness", "fun", "happy" }}, "user": { "id": 1347041785, "name": "Deborah Lynne", "screen_name": "DeborahlynneNYC", "lang": "en", "location": "los angeles ", "create_at": date("2013-04-12"), "description": "Hair/mkup/skincare/fashion expert/accessorydesigner/blogger. Loves travel,meeting positive smart peeps, new cultures, happygolucky adventureseeker, love 2 chill", "followers_count": 203, "friends_count": 518, "statues_count": 401 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-25T10:00:24.000Z"), "id": 669576385027928064, "text": "My throat really bothers me ugh freaking Shelby ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2777149779, "name": "coco", "screen_name": "sandrasandia_", "lang": "en", "location": "null", "create_at": date("2014-09-19"), "description": "WHS/EMCC , @izzzy_0 ❤️ chi-town, 18", "followers_count": 359, "friends_count": 289, "statues_count": 10340 }, "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("2015-11-25T10:00:25.000Z"), "id": 669576385262956544, "text": "I was born with some nappy hair drinking breast milk out a lean cup!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165889254, "name": "Royalty", "screen_name": "iLLRoyalty", "lang": "en", "location": "Cleveland", "create_at": date("2010-07-12"), "description": "| #LongLiveZuse | #LongLiveJen | Producer | marbeatz17@gmail.com | 1/1™", "followers_count": 868, "friends_count": 286, "statues_count": 64562 }, "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("2015-11-25T10:00:25.000Z"), "id": 669576385321537536, "text": "December 26th", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 909716570, "name": "I Work at Clark, hmu", "screen_name": "AsplundJorden", "lang": "en", "location": "the cool table ", "create_at": date("2012-10-28"), "description": "null", "followers_count": 445, "friends_count": 252, "statues_count": 9990 }, "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("2015-11-25T10:00:25.000Z"), "id": 669576385363582978, "text": "@iiQueenMing �������� https://t.co/s5AffPNiyS", "in_reply_to_status": 669576233462665216, "in_reply_to_user": 37594850, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 37594850 }}, "user": { "id": 26174402, "name": "Christian Grey", "screen_name": "Half_Ah_Virgin", "lang": "en", "location": "P H I L L Y ", "create_at": date("2009-03-23"), "description": "My DMs are open Mon-Fri 11am-11pm and I charge $20 for a follow back!", "followers_count": 9481, "friends_count": 972, "statues_count": 314282 }, "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-25T10:00:25.000Z"), "id": 669576385501884416, "text": "Solomon going to jail", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 449502917, "name": "#No$tepsBack", "screen_name": "Jordannmichael_", "lang": "en", "location": "Texas, USA", "create_at": date("2011-12-28"), "description": "$uccess is the best revenge c/o 17 tianaisthebestestbestfriendinthewholewideworld.", "followers_count": 1395, "friends_count": 548, "statues_count": 33842 }, "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("2015-11-25T10:00:25.000Z"), "id": 669576385678151680, "text": "About to sleep for a good 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": 135353774, "name": "Hadji Galore™", "screen_name": "HadjiTheUpsettr", "lang": "en", "location": "Around ™", "create_at": date("2010-04-20"), "description": "Progressing..|717|FreeQuez|CalU|", "followers_count": 1815, "friends_count": 1104, "statues_count": 150897 }, "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("2015-11-25T10:00:25.000Z"), "id": 669576386022088704, "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": 128874275, "name": "tee", "screen_name": "teresa_terlato", "lang": "en", "location": "new york ", "create_at": date("2010-04-02"), "description": "WFHS 17' • happiest lil ginger you'll ever meet•", "followers_count": 523, "friends_count": 915, "statues_count": 7883 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shirley, NY", "id": "7d5f383dc2e0241d", "name": "Shirley", "place_type": "city", "bounding_box": rectangle("-72.90615,40.737278 -72.847502,40.852084") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3667070, "cityName": "Shirley" } }
+{ "create_at": datetime("2015-11-25T10:00:25.000Z"), "id": 669576386139439105, "text": "�������������� https://t.co/hqsJj14SKN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2265235430, "name": "✨ADaddy✨", "screen_name": "LovelyyKeys", "lang": "en", "location": "null", "create_at": date("2013-12-27"), "description": "Living life", "followers_count": 655, "friends_count": 448, "statues_count": 19665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-25T10:00:25.000Z"), "id": 669576386231709696, "text": "@bvbblez im thirsty lol https://t.co/ge6OZRQcLI", "in_reply_to_status": 669576239661715456, "in_reply_to_user": 1206610483, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1206610483 }}, "user": { "id": 341938605, "name": "seastar✨", "screen_name": "emmzpen", "lang": "en", "location": "pdx", "create_at": date("2011-07-24"), "description": "im not sad anymore, im just tired of this place @haydenndent • @bvbblez • mafia☕️", "followers_count": 918, "friends_count": 338, "statues_count": 20306 }, "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("2015-11-25T10:00:25.000Z"), "id": 669576386269552641, "text": "Hate when folks talk to me like they died once, got all the answers and came back to tell facts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46760451, "name": "King Sleep", "screen_name": "InFaREDDot", "lang": "en", "location": "Detroit ", "create_at": date("2009-06-12"), "description": "When you want to succeed as bad as you want to breathe, then you will be successful - Eric Thomas", "followers_count": 3146, "friends_count": 3010, "statues_count": 31823 }, "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-25T10:00:25.000Z"), "id": 669576386324127745, "text": "It's Lit https://t.co/q0l2NNZ6LH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 814935858, "name": "Chae", "screen_name": "chaemcconaghy_", "lang": "en", "location": "daydreaming", "create_at": date("2012-09-10"), "description": "Which do you prefer, Dijon or regular mustard gas?", "followers_count": 482, "friends_count": 457, "statues_count": 381 }, "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("2015-11-25T10:00:25.000Z"), "id": 669576386382864385, "text": "Only thing good about coming home is food && meeting up with my plugs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 403481385, "name": "bspells", "screen_name": "BSpellsJr", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2011-11-02"), "description": "null", "followers_count": 728, "friends_count": 765, "statues_count": 10715 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vineland, NJ", "id": "ecc2e1285c7d074f", "name": "Vineland", "place_type": "city", "bounding_box": rectangle("-75.076284,39.401507 -74.945245,39.568715") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34011, "countyName": "Cumberland", "cityID": 3476070, "cityName": "Vineland" } }
+{ "create_at": datetime("2015-11-25T10:00:25.000Z"), "id": 669576386483367936, "text": "Cloudy this afternoon, high 73 (23 C). Low 63 (17 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119754993, "name": "Waco Weather", "screen_name": "WacoWeather", "lang": "en", "location": "Waco, TX", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Waco, TX", "followers_count": 809, "friends_count": 68, "statues_count": 6234 }, "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.069786,31.639828") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4876000, "cityName": "Waco" } }
+{ "create_at": datetime("2015-11-25T10:00:25.000Z"), "id": 669576386571427840, "text": "Smells like the dank @ Jess and her grandpa just showed up ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 283341735, "name": "D A B B Y", "screen_name": "AbbyDukes", "lang": "en", "location": "ATX", "create_at": date("2011-04-16"), "description": "gas in my blunt", "followers_count": 951, "friends_count": 521, "statues_count": 14023 }, "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("2015-11-25T10:00:25.000Z"), "id": 669576387406274560, "text": "My nigga I'm legit crying", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3394737827, "name": "Ken", "screen_name": "SosaStuntin", "lang": "en", "location": "West Virginia, USA", "create_at": date("2015-07-29"), "description": "wvu18", "followers_count": 220, "friends_count": 199, "statues_count": 1469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North East, MD", "id": "01394a901ecfda99", "name": "North East", "place_type": "city", "bounding_box": rectangle("-76.01977,39.556031 -75.886302,39.663376") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24015, "countyName": "Cecil", "cityID": 2456450, "cityName": "North East" } }
+{ "create_at": datetime("2015-11-25T10:00:25.000Z"), "id": 669576387582230528, "text": "No threat to Americans message while Putin is moving SAM missiles to Syria. How will America respond? https://t.co/J1uBCWu6Yq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43623440, "name": "CM JAVIER", "screen_name": "cmjavier", "lang": "en", "location": "United States", "create_at": date("2009-05-30"), "description": "The UNCOMMON MAN\r\nFrom Humble Beginnings to a Confidante of President FERDINAND E. MARCOS of the Philippines\r\nTo Order, visit http://www.cmjavier.com", "followers_count": 78, "friends_count": 191, "statues_count": 2145 }, "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("2015-11-25T10:00:25.000Z"), "id": 669576387800358912, "text": "@jace_Curry25 I'll be checking the books ����", "in_reply_to_status": 669576129070497792, "in_reply_to_user": 3526156408, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3526156408 }}, "user": { "id": 361033207, "name": "A$AP Tymon", "screen_name": "Tymon_Anderson", "lang": "en", "location": "Norwood Young America, MN", "create_at": date("2011-08-23"), "description": "I pulled up to the house about 7 or 8", "followers_count": 547, "friends_count": 507, "statues_count": 5479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwood Young America, MN", "id": "e3cc493c0bd59124", "name": "Norwood Young America", "place_type": "city", "bounding_box": rectangle("-93.939825,44.762281 -93.893861,44.790203") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27019, "countyName": "Carver", "cityID": 2747520, "cityName": "Norwood Young America" } }
+{ "create_at": datetime("2015-11-25T10:00:25.000Z"), "id": 669576387905241089, "text": "This dude @mickjenkins making a pot pie on snapchat funny as shit ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 79053251 }}, "user": { "id": 911694654, "name": "©®ûž", "screen_name": "cruz_villarreal", "lang": "en", "location": "McKinney, TX", "create_at": date("2012-10-28"), "description": "┳╱╱┓┳━┓┳━┓╱╱┳╭━╮ ┃╱╱┃┣┫╱┣┫╱╱╱┃╰━╮ ┻━┛┛┻╱╱┻━┛╱╱┻╰━╯ ╭━━╮╭━━╮╭━━╮┳━━╮ ┃┏━┓┃╱╱┃┃╱╱┃┃╱╱┃ ╰━━╯╰━━╯╰━━╯┻━━╯", "followers_count": 148, "friends_count": 109, "statues_count": 4480 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Melissa, TX", "id": "002f05221ac47644", "name": "Melissa", "place_type": "city", "bounding_box": rectangle("-96.605443,33.260111 -96.53793,33.310683") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4847496, "cityName": "Melissa" } }
+{ "create_at": datetime("2015-11-25T10:00:25.000Z"), "id": 669576387921969152, "text": "@odddelights thanks Marin!!", "in_reply_to_status": 669518523874590720, "in_reply_to_user": 1368522410, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1368522410 }}, "user": { "id": 32185860, "name": "Colleen Conway", "screen_name": "colleenaconway", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-04-16"), "description": "Book Rep for Penguin Young Readers at PRH. Reader, NH native, Seahawks fan, baker & jam maker. Owns persnickety cat. Tweets are my own", "followers_count": 1675, "friends_count": 1835, "statues_count": 18037 }, "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-25T10:00:25.000Z"), "id": 669576388257710081, "text": "Wonder if my bestfriend wanna do it then", "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": "Trevonn24", "lang": "en", "location": "null", "create_at": date("2014-03-02"), "description": "If it don't make dollars then it don't make sense #TrueStory ...... IG: Trevondwill", "followers_count": 1172, "friends_count": 719, "statues_count": 48253 }, "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("2015-11-25T10:00:25.000Z"), "id": 669576388349812737, "text": "I just got sent home from work because my cough is gross", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2768389184, "name": "Mina", "screen_name": "minahernandezz", "lang": "en", "location": "null", "create_at": date("2014-08-25"), "description": "I'm a Star Wars, Obsessed with 80's Music", "followers_count": 369, "friends_count": 359, "statues_count": 2757 }, "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("2015-11-25T10:00:25.000Z"), "id": 669576388488380416, "text": "DEAD ASS https://t.co/Rvy6HmfpMI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3429449902, "name": "lil pee wee", "screen_name": "beanbrand", "lang": "en", "location": "315", "create_at": date("2015-08-17"), "description": "help me meet the Thompson brothers and I'll help you with immortality :)", "followers_count": 120, "friends_count": 156, "statues_count": 2029 }, "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("2015-11-25T10:00:25.000Z"), "id": 669576388521791488, "text": "When one of your side niggas hitting you up with ( why don't you talk to me anymore) �� fuck outta here boi✌", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 760321220, "name": "kαrєєnnхσ_ ♡", "screen_name": "kareennxo", "lang": "en", "location": "El Paso, TX", "create_at": date("2012-08-15"), "description": "O824O14 ♡ O1O7O15 ♡ el paso tx . ✌ ✨", "followers_count": 447, "friends_count": 386, "statues_count": 8318 }, "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("2015-11-25T10:00:25.000Z"), "id": 669576388584857601, "text": "last night during #bigkrit's #kriticallyacclaimedtour • photo by #gregnoire @ Warehouse Live https://t.co/7Kw44bOW6I", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.354332,29.7522106"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "bigkrit", "kriticallyacclaimedtour", "gregnoire" }}, "user": { "id": 26010290, "name": "JPEG GREG", "screen_name": "GregNoire", "lang": "en", "location": "where the lighting is lowkey", "create_at": date("2009-03-23"), "description": "classic. info@gregnoire.com", "followers_count": 1838, "friends_count": 701, "statues_count": 71194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:25.000Z"), "id": 669576389088051200, "text": "the drive to this mountain is killer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2189311172, "name": "Gracie❀", "screen_name": "gracieloulouu", "lang": "en", "location": "null", "create_at": date("2013-11-11"), "description": "/// @imkourtneeokay ///", "followers_count": 223, "friends_count": 199, "statues_count": 4595 }, "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-25T10:00:25.000Z"), "id": 669576389280952320, "text": "@Alejandraa_Boo THANK U LMAO UR TWO DAYS OLDER THAN ME SMH", "in_reply_to_status": 669564551210274817, "in_reply_to_user": 458568443, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 458568443 }}, "user": { "id": 339455504, "name": "ashy", "screen_name": "ashypls", "lang": "en", "location": "null", "create_at": date("2011-07-20"), "description": "accept me into college", "followers_count": 583, "friends_count": 494, "statues_count": 14118 }, "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-25T10:00:26.000Z"), "id": 669576389369184256, "text": "Me neither lol https://t.co/Y4hkL9asdK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 277782273, "name": "DJ Thunder", "screen_name": "IAMDJThunder", "lang": "en", "location": "Behind the Booth ", "create_at": date("2011-04-05"), "description": "Young Broke Chico #YBC #FAU #305Raised Booking 813 516 7134 #PangeaMiami Nov 28th Water and Bring Your Own Drinks Party", "followers_count": 4783, "friends_count": 3419, "statues_count": 91633 }, "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("2015-11-25T10:00:26.000Z"), "id": 669576389578915840, "text": "Can't Wait https://t.co/Wi8LEKBPmY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 64131562, "name": "Joey Sanders", "screen_name": "JoeySandersNY", "lang": "en", "location": "null", "create_at": date("2009-08-09"), "description": "null", "followers_count": 1442, "friends_count": 290, "statues_count": 117011 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marco Island, FL", "id": "398e489d0705ff47", "name": "Marco Island", "place_type": "city", "bounding_box": rectangle("-81.750726,25.90813 -81.673991,25.974477") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12021, "countyName": "Collier", "cityID": 1243083, "cityName": "Marco Island" } }
+{ "create_at": datetime("2015-11-25T10:00:26.000Z"), "id": 669576389612277761, "text": "Nike Air Foamposites drop today $230 we still have a sz 8.5 , 9 ,9.5 left #playmakers #nike #foams… https://t.co/QXvj3eRRdL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.9803288,43.09263521"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "playmakers", "nike", "foams" }}, "user": { "id": 973095734, "name": "playmakers", "screen_name": "playmakersmilw", "lang": "en", "location": "null", "create_at": date("2012-11-26"), "description": "null", "followers_count": 36, "friends_count": 35, "statues_count": 672 }, "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("2015-11-25T10:00:26.000Z"), "id": 669576389645893632, "text": "@Thatcoolkid817 ����������", "in_reply_to_status": 669573200905420800, "in_reply_to_user": 42553160, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 42553160 }}, "user": { "id": 50111109, "name": "David Edge", "screen_name": "DavidxEdge", "lang": "en", "location": "San Jose, CA", "create_at": date("2009-06-23"), "description": "I'm that guy who thinks he's fuuny. Straight edge and pro wrestling is my life. Instagram/Vine/Snapchat: Davidxedge #mountmarkmore", "followers_count": 1414, "friends_count": 722, "statues_count": 11177 }, "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("2015-11-25T10:00:26.000Z"), "id": 669576389683642368, "text": "Sunny this afternoon, high 61 (16 C). Low 45 (7 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119755728, "name": "Louisville Weather", "screen_name": "Louisville_WX", "lang": "en", "location": "Louisville, KY", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Louisville, KY", "followers_count": 1302, "friends_count": 68, "statues_count": 6261 }, "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-25T10:00:26.000Z"), "id": 669576389700419585, "text": "Thanksgiving pre-game @ Stiles Switch BBQ & Brew https://t.co/FtoWKp8dSY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.7216873,30.3345203"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 9286622, "name": "Melanie Haupt", "screen_name": "boxingoctopus", "lang": "en", "location": "austin, tx", "create_at": date("2007-10-06"), "description": "Ain't no thing like me 'cept me.", "followers_count": 805, "friends_count": 556, "statues_count": 14982 }, "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-25T10:00:26.000Z"), "id": 669576389750845441, "text": "@AbnInfVet Appreciate RT. Army wives requested to participate in Army approved study. Participation is online. https://t.co/uOSxTvgmMs", "in_reply_to_status": -1, "in_reply_to_user": 43037025, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 43037025 }}, "user": { "id": 366931475, "name": "Patrick", "screen_name": "Patrick0215", "lang": "en", "location": "Auburn, AL", "create_at": date("2011-09-02"), "description": "Navy & CG vet working for the Army. I keep Facebook posts tame, but let my conservative views fly freely on Twitter. Oh,...and #WarEagle!", "followers_count": 724, "friends_count": 1443, "statues_count": 4591 }, "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("2015-11-25T10:00:26.000Z"), "id": 669576390073647104, "text": "@WSHHFANS @Emilyxo_XO", "in_reply_to_status": 669395300222705664, "in_reply_to_user": 1370986902, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1370986902, 360170050 }}, "user": { "id": 607838703, "name": "Dominic Salerno", "screen_name": "DomSalerno24", "lang": "en", "location": "0-0", "create_at": date("2012-06-13"), "description": "Jack O'Callahan, Charlestown, Mass. Boston University. PESLIYL Tweet of the year recipient 2015", "followers_count": 320, "friends_count": 294, "statues_count": 3591 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beckett, NJ", "id": "00a250140f3a65ea", "name": "Beckett", "place_type": "city", "bounding_box": rectangle("-75.387633,39.745695 -75.341385,39.79828") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34015, "countyName": "Gloucester", "cityID": 3404400, "cityName": "Beckett" } }
+{ "create_at": datetime("2015-11-25T10:00:26.000Z"), "id": 669576390337925124, "text": "I'm glad we have brave #Americans protecting our borders because our government won't! #MakeAmericaGreatAgain", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Americans", "MakeAmericaGreatAgain" }}, "user": { "id": 2900938994, "name": "Clinton Rusthoven", "screen_name": "clintsessentia1", "lang": "en", "location": "Montana, USA", "create_at": date("2014-11-15"), "description": "#Author of 'Self-Disclosure: Being Bi-Gender and My Journey.' #Amazon. Please check out my #erotic #stories on #amazon! Make each day count!", "followers_count": 2716, "friends_count": 3260, "statues_count": 26448 }, "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("2015-11-25T10:00:26.000Z"), "id": 669576390354841601, "text": "Primary care business models are evolving, survey finds https://t.co/Km1LRquy2e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3047925973, "name": "Ian Weissman, DO", "screen_name": "DrIanWeissman", "lang": "en", "location": "Milwaukee, WI", "create_at": date("2015-02-27"), "description": "Radiologist. Advocate of improving the patient experience, healthcare innovation, leadership best practices, value-based medicine. Opinions are my own.", "followers_count": 2965, "friends_count": 2999, "statues_count": 5560 }, "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("2015-11-25T10:00:26.000Z"), "id": 669576390925287424, "text": "im 18 and my birthday is December 19th �� remember to spoil me !! https://t.co/urbJrY9Hax", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3821835677, "name": "cyrstal rae", "screen_name": "cyrstaalrae", "lang": "en", "location": "null", "create_at": date("2015-09-29"), "description": "for booking inquiries only 18+ #moneyqueen lifesabeachent@gmail.com", "followers_count": 5138, "friends_count": 83, "statues_count": 1037 }, "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("2015-11-25T10:00:26.000Z"), "id": 669576391021629441, "text": "Accident, right lane blocked in #ElPaso on I 25 NB south of Bijou St, stop and go traffic back to Cimarron St, delay of 2 mins #traffic", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.8304,38.83692"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ElPaso", "traffic" }}, "user": { "id": 930711835, "name": "TTN Colorado Springs", "screen_name": "TotalTrafficCOS", "lang": "en", "location": "Colorado Springs, CO", "create_at": date("2012-11-06"), "description": "Traffic information for Colorado Springs! Powered by Total Traffic Network.", "followers_count": 241, "friends_count": 263, "statues_count": 8320 }, "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("2015-11-25T10:00:26.000Z"), "id": 669576391260794880, "text": "Tis the season to shop small!! Check out all our locally made items that are perfect for gift giving… https://t.co/pcA4dUrdIK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.56621122,35.56881826"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3027773636, "name": "Milo & Lily", "screen_name": "miloandlilyokc", "lang": "en", "location": "OKC", "create_at": date("2015-02-18"), "description": "Storefront opening August 4th with a Grand Opening Event August 8th (tax free weekend)! 9612 N May Avenue 73120", "followers_count": 111, "friends_count": 461, "statues_count": 326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Village, OK", "id": "4c7d3c109c481e6a", "name": "The Village", "place_type": "city", "bounding_box": rectangle("-97.575986,35.557923 -97.540303,35.580222") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4073250, "cityName": "The Village" } }
+{ "create_at": datetime("2015-11-25T10:00:26.000Z"), "id": 669576391352934402, "text": "You're my favorite, dork.�� https://t.co/LCnO1vYh19", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1043574727, "name": "Gänseblümchen", "screen_name": "BethsDahlia96", "lang": "en", "location": "Ashtabula, OH", "create_at": date("2012-12-28"), "description": "@CaptainTathy is the light of my life♑️♋️ / @imAWOL17 is my spirit animal and sister / 12195-101011❤️/ @coy_kristen is mine of 14 years", "followers_count": 409, "friends_count": 197, "statues_count": 7558 }, "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("2015-11-25T10:00:26.000Z"), "id": 669576391457918977, "text": "�� �� https://t.co/rM7s4KSUhr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4042044563, "name": "5 : 25", "screen_name": "JaayTooMuch", "lang": "en", "location": "null", "create_at": date("2015-10-25"), "description": "M E E C H Y ' S .. W O R L D", "followers_count": 191, "friends_count": 403, "statues_count": 682 }, "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("2015-11-25T10:00:26.000Z"), "id": 669576392036585472, "text": "Preaching to make disciples is like going to the nursery and spraying the crying babies with milk and saying that you just fed the kids.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 122739632, "name": "Jarrod Billings", "screen_name": "JarrodBillings", "lang": "en", "location": "null", "create_at": date("2010-03-13"), "description": "Make disciples who make disciples. #LoveGod #LoveOthers #GreatCommission #MakeDisciples", "followers_count": 637, "friends_count": 1419, "statues_count": 5566 }, "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("2015-11-25T10:00:26.000Z"), "id": 669576392049213440, "text": "I'm trying to be in somebody tattoo shop Sunday night����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 624458538, "name": "November 29♐️", "screen_name": "mytatyanaaa_", "lang": "en", "location": "laughingatyouhoesss❤️", "create_at": date("2012-07-01"), "description": "'16seniorrrr❤️", "followers_count": 1426, "friends_count": 1246, "statues_count": 51630 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Violet, LA", "id": "f6158506a43e47cc", "name": "Violet", "place_type": "city", "bounding_box": rectangle("-89.907768,29.875438 -89.874714,29.924759") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22087, "countyName": "St. Bernard", "cityID": 2278855, "cityName": "Violet" } }
+{ "create_at": datetime("2015-11-25T10:00:26.000Z"), "id": 669576392330244097, "text": "Uncle: When you gunna have some kids \nMe: When you gunna meet yours? \nUncle: https://t.co/YDC63zdYDj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 353360401, "name": "Connor Gawryszewski", "screen_name": "biggzoo", "lang": "en", "location": "Belton, TX", "create_at": date("2011-08-11"), "description": "UMHB DB Transfer/ Sophomore .. Selfie King Snapchat:ConnorAlan123 Insta:Connor_zooski", "followers_count": 839, "friends_count": 451, "statues_count": 21208 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellmead, TX", "id": "d26e48982854dc8d", "name": "Bellmead", "place_type": "city", "bounding_box": rectangle("-97.1233,31.572896 -97.070011,31.644987") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4807408, "cityName": "Bellmead" } }
+{ "create_at": datetime("2015-11-25T10:00:26.000Z"), "id": 669576392342814720, "text": "hahhhhah me ������ https://t.co/T4j3baUyec", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 734886686, "name": "lisa", "screen_name": "x_lisanicole", "lang": "en", "location": "null", "create_at": date("2012-08-03"), "description": "battle's", "followers_count": 1939, "friends_count": 980, "statues_count": 6102 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:26.000Z"), "id": 669576392611201024, "text": "��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 296839017, "name": "Carol ❤️", "screen_name": "__Jacarol", "lang": "en", "location": "null", "create_at": date("2011-05-11"), "description": "Grand Valley State University 19' ⚓️ #lakerforalifetime", "followers_count": 387, "friends_count": 440, "statues_count": 7636 }, "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-25T10:00:26.000Z"), "id": 669576392661708800, "text": "https://t.co/HrzBdsZLig", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3010008897, "name": "FP", "screen_name": "fpiv513", "lang": "en", "location": "null", "create_at": date("2015-02-01"), "description": "Every Damn Night, on the Telephone", "followers_count": 95, "friends_count": 246, "statues_count": 6029 }, "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("2015-11-25T10:00:26.000Z"), "id": 669576392967856128, "text": "When you go see the weeknd again...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1325804888, "name": "killa", "screen_name": "shannonkilbride", "lang": "en", "location": "Dracut, MA ", "create_at": date("2013-04-03"), "description": "sc-shannon_kilbrid ☯", "followers_count": 1015, "friends_count": 587, "statues_count": 38394 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hudson, NH", "id": "01a216c0183feb44", "name": "Hudson", "place_type": "city", "bounding_box": rectangle("-71.463685,42.69879 -71.360807,42.824745") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33011, "countyName": "Hillsborough", "cityID": 3337860, "cityName": "Hudson" } }
+{ "create_at": datetime("2015-11-25T10:00:26.000Z"), "id": 669576393039024128, "text": "Thanksgiving and end of the month specials! Call us today! 972-403-4499 / 469-662-8688! ��… https://t.co/orsFKuH5NL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.80906556,32.93331644"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2174958433, "name": "D&G Warehouse", "screen_name": "DandGWarehouse", "lang": "en", "location": "Dallas, TX", "create_at": date("2013-11-04"), "description": "Wholesale Diamond Dealers Dallas, Loose Diamond Wholesalers, Diamond Engagement Rings, Custom Designs, Diamond Jewelry Store Based in Dallas, TX.", "followers_count": 201, "friends_count": 180, "statues_count": 134 }, "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-25T10:00:26.000Z"), "id": 669576393055956992, "text": "#park \n#brooklyn #nyc #vsco \n#leica #leicam #leicamp240 \n#bw #blackandwhite #life #sidewalk @ Marlow… https://t.co/6SNtspByCz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9650497,40.7106285"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "park", "brooklyn", "nyc", "vsco", "leica", "leicam", "leicamp240", "bw", "blackandwhite", "life", "sidewalk" }}, "user": { "id": 99579091, "name": "unknwnkid", "screen_name": "unknownkid00", "lang": "en", "location": "#brooklyn ", "create_at": date("2009-12-26"), "description": "#NYC", "followers_count": 51, "friends_count": 142, "statues_count": 1066 }, "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-25T10:00:26.000Z"), "id": 669576393190187008, "text": "I'm at Daily Dozen Doughnut Co in Seattle, WA https://t.co/ZHw2GikGRt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.34043077,47.608873"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4246761, "name": "PJ Andrews", "screen_name": "aolbites", "lang": "en", "location": "Sacramento, CA", "create_at": date("2007-04-11"), "description": "I am an atheist as no evidence exists for any god. I am a skeptical humanist who follows the evidence & believes that people are the ones we should care about.", "followers_count": 134, "friends_count": 290, "statues_count": 9218 }, "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-25T10:00:27.000Z"), "id": 669576393555058688, "text": "@lowbeyonder @cchauvet Prove it. What lands are in your deck?", "in_reply_to_status": 669565543762931712, "in_reply_to_user": 101561695, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 101561695, 15416277 }}, "user": { "id": 161968653, "name": "Doctor Superstition", "screen_name": "DSuperstition", "lang": "en", "location": "#mtgunderground ", "create_at": date("2010-07-02"), "description": "Love Not Law", "followers_count": 499, "friends_count": 359, "statues_count": 18356 }, "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("2015-11-25T10:00:27.000Z"), "id": 669576393672540160, "text": "I'm spending the whole day with 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": 44774988, "name": "Lizzy Liz✨", "screen_name": "KM_Heartbreaker", "lang": "en", "location": "null", "create_at": date("2009-06-04"), "description": "Veni, vidi, vici❤️", "followers_count": 1086, "friends_count": 1194, "statues_count": 61211 }, "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("2015-11-25T10:00:27.000Z"), "id": 669576393836118017, "text": "Everybody wanna be Fetty Wap, but don't nobody wanna lose an eyeball ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2700233032, "name": "Kid Slum", "screen_name": "Shaunyy_2xxs", "lang": "en", "location": "-The Man Cave-", "create_at": date("2014-07-11"), "description": "We Starved Together, We Eat Together!! Follow Bae @shorty_beanzz", "followers_count": 369, "friends_count": 635, "statues_count": 7311 }, "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("2015-11-25T10:00:27.000Z"), "id": 669576393915666432, "text": "i look like a bitch w/out my beard smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160084583, "name": "Diego Mendez ⚽️", "screen_name": "_fifalordmendez", "lang": "en", "location": "Bonverse, TX ", "create_at": date("2010-06-26"), "description": "Stmu 18 #25 ⚽️ snapchat: fifalordmendez", "followers_count": 316, "friends_count": 208, "statues_count": 9339 }, "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-25T10:00:27.000Z"), "id": 669576394230353920, "text": "@theycallmemara fetty wap is my spirit animal ��", "in_reply_to_status": 669573043124047873, "in_reply_to_user": 70023008, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 70023008 }}, "user": { "id": 518054413, "name": "Lauren Bacho", "screen_name": "Visual_Bacho", "lang": "en", "location": "Athens, OH", "create_at": date("2012-03-07"), "description": "Ohio University '18 | Photojournalism & French | @ThePost | Photo Editor | Columnist", "followers_count": 304, "friends_count": 315, "statues_count": 5431 }, "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("2015-11-25T10:00:27.000Z"), "id": 669576394444111872, "text": "What do we talk about in my English class?\n\nStoners & eating psychedelic mushrooms on thanksgiving ��\n\nThis is a fun conversation ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 331862270, "name": "Wreck It", "screen_name": "Lyssa_J24", "lang": "en", "location": "Grants Pass, OR", "create_at": date("2011-07-08"), "description": "⭐️ || 20 || 541 || ig: alyssa_j24 || ⭐️|| Alaska grown || (BGCRV) || Proud sister of a Sailor and a Marine ||", "followers_count": 494, "friends_count": 375, "statues_count": 15790 }, "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("2015-11-25T10:00:27.000Z"), "id": 669576394788093952, "text": "Accident, left lane blocked in #Westborough on Mass Pike WB after Westboro Service Area #traffic https://t.co/AoTr5OsSgQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.592,42.24392"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Westborough", "traffic" }}, "user": { "id": 249393984, "name": "TTN Boston", "screen_name": "TotalTrafficBOS", "lang": "en", "location": "Boston, MA", "create_at": date("2011-02-08"), "description": "null", "followers_count": 1379, "friends_count": 1, "statues_count": 21635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.59695,42.183866 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2015-11-25T10:00:27.000Z"), "id": 669576394876153857, "text": "@georgelopez @RealFPJr @BillSimmons Thar was a Bully who we all knew in our barrio that just kept poinding you til you gave up #unChingaso", "in_reply_to_status": 669372216287457280, "in_reply_to_user": 49717874, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "unChingaso" }}, "user_mentions": {{ 49717874, 2777356364, 32765534 }}, "user": { "id": 3189400592, "name": "My Info", "screen_name": "mannytopdog60", "lang": "en", "location": "Ventura, CA", "create_at": date("2015-05-08"), "description": "Laker Fan 4Life, Dodger fan, LA Kings fan", "followers_count": 4, "friends_count": 19, "statues_count": 139 }, "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("2015-11-25T10:00:27.000Z"), "id": 669576395002093568, "text": "@McKJayHorsley I'm not... Too quiet in 3rd block without you ����", "in_reply_to_status": 669573996418060292, "in_reply_to_user": 989676769, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 989676769 }}, "user": { "id": 965194434, "name": "Debra Donohue", "screen_name": "jerzeehoosier", "lang": "en", "location": "Granger, IN", "create_at": date("2012-11-22"), "description": "Educational Engineer and molder of young minds!", "followers_count": 173, "friends_count": 341, "statues_count": 2962 }, "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("2015-11-25T10:00:27.000Z"), "id": 669576396130283520, "text": "Ain't nobody safe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 318170462, "name": "Oga Izu", "screen_name": "ThatBoyIzu", "lang": "en", "location": "Nigeria", "create_at": date("2011-06-15"), "description": "3P's.", "followers_count": 463, "friends_count": 331, "statues_count": 24109 }, "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("2015-11-25T10:00:27.000Z"), "id": 669576396482740225, "text": "@maria_mcnair https://t.co/2m9Xpk7DtW", "in_reply_to_status": 669510770766585856, "in_reply_to_user": 542900419, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 542900419 }}, "user": { "id": 1119465510, "name": "Hip-HopCrates✯", "screen_name": "Kev_Cloud", "lang": "en", "location": "Long Island, NY", "create_at": date("2013-01-25"), "description": "▪️They say live every day like its your last. ▪️Well I'm just going along for the ride. ▪️Age: 23 | Rating: Awesome | ♉", "followers_count": 601, "friends_count": 579, "statues_count": 6942 }, "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("2015-11-25T10:00:27.000Z"), "id": 669576396554018817, "text": "Is there a better feeling than turning OFF your alarm for the rest of the week?!? #thanksvegas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thanksvegas" }}, "user": { "id": 500373876, "name": "Miss Hayes", "screen_name": "misshayes520", "lang": "en", "location": "null", "create_at": date("2012-02-22"), "description": "null", "followers_count": 195, "friends_count": 294, "statues_count": 1353 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Windsor Locks, CT", "id": "b6cd125e971e59ef", "name": "Windsor Locks", "place_type": "city", "bounding_box": rectangle("-72.718049,41.897596 -72.616067,41.956129") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 987140, "cityName": "Windsor Locks" } }
+{ "create_at": datetime("2015-11-25T10:00:27.000Z"), "id": 669576396600160257, "text": "THANKSGIVING TWERKY #HAM # #TWERK #twerky ryaustinhair elenamiglino @ Brooklyn Greenpoint NY https://t.co/vX3u5pUCUd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.95643806,40.73383163"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HAM", "TWERK", "twerky" }}, "user": { "id": 20546736, "name": "Parson James", "screen_name": "iamparson", "lang": "en", "location": "ÜT: 40.684189,-73.977854", "create_at": date("2009-02-10"), "description": "hold on to your seats. im up on my feet. TEMPLE & Stole the Show out now!", "followers_count": 3717, "friends_count": 857, "statues_count": 7977 }, "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-25T10:00:27.000Z"), "id": 669576396784705541, "text": "Words I call upon to keep myself moving. I thank the lord for the people I have found... https://t.co/KdYDzxzVWa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1268057976, "name": "mama kass", "screen_name": "frannectarine", "lang": "en", "location": "null", "create_at": date("2013-03-14"), "description": "throw kindness around like confetti", "followers_count": 330, "friends_count": 351, "statues_count": 7181 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forest Hills Gardens, Queens", "id": "46d546c6d47d42e8", "name": "Forest Hills Gardens", "place_type": "neighborhood", "bounding_box": rectangle("-73.858382,40.706288 -73.831037,40.727129") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2015-11-25T10:00:27.000Z"), "id": 669576396864286720, "text": "It's nice that my grandparents aren't grilling me on boys 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": 179525532, "name": "Dominique Marshall", "screen_name": "dominique32115", "lang": "en", "location": "San Antonio", "create_at": date("2010-08-17"), "description": "Fajita tell me what's wrong. I've never seen such sorrow iiiiiiiiin your eyes and the wedding is tomorrow", "followers_count": 238, "friends_count": 171, "statues_count": 8636 }, "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("2015-11-25T10:00:27.000Z"), "id": 669576397036380162, "text": "Had to https://t.co/gOxh87yTbk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 66811297, "name": "Dominique Hubbard", "screen_name": "DominiqueSaysss", "lang": "en", "location": "Columbus, OH", "create_at": date("2009-08-18"), "description": "O$U class of '16. instagram: dominiquehubbard_", "followers_count": 741, "friends_count": 446, "statues_count": 16848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Centerville, OH", "id": "edacd050f402302c", "name": "Centerville", "place_type": "city", "bounding_box": rectangle("-84.21092,39.579164 -84.098555,39.674261") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39113, "countyName": "Montgomery", "cityID": 3913190, "cityName": "Centerville" } }
+{ "create_at": datetime("2015-11-25T10:00:27.000Z"), "id": 669576397149462529, "text": "lol same https://t.co/QAoGKfW990", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 441578682, "name": "baddest putà✨", "screen_name": "KeAndreaK_", "lang": "en", "location": "cameron is super crazy ", "create_at": date("2011-12-19"), "description": "everything they say, I'm already doing.. #beyhive #camhive", "followers_count": 4289, "friends_count": 2049, "statues_count": 96656 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:27.000Z"), "id": 669576397380153345, "text": "@RT_com Sure I believe #Lavrov & #Putin, they never lied. #Russia is bombing innocent civilians in #Syria as we speak. #Russia out of Syria.", "in_reply_to_status": 669481262680285184, "in_reply_to_user": 64643056, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Lavrov", "Putin", "Russia", "Syria", "Russia" }}, "user_mentions": {{ 64643056 }}, "user": { "id": 409622922, "name": "Fawaz Elmasri", "screen_name": "FawazElmasri", "lang": "en", "location": "Arcadia CA", "create_at": date("2011-11-10"), "description": "Like Soccer, Activism helps keep me young.\n Race: Human, Religion: Divine, Nationality: Universal, Ideology: Theology of life.", "followers_count": 354, "friends_count": 721, "statues_count": 2873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-25T10:00:27.000Z"), "id": 669576397405319169, "text": "Krueger Tattoo studio gift certificates are here for the holiday season!! Give the gift that keeps on… https://t.co/RhgSZgoo5a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-91.4689789,44.8058205"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 272745394, "name": ".KRUEGER.", "screen_name": "Kruegertattoo", "lang": "en", "location": "Eau Claire, WI", "create_at": date("2011-03-26"), "description": "Tattooer/ Painter/ Power Lifter. Working at my trade since 2007. Have 2 kids with the best wife ever!!! Designer for Solace Skate Co. Apple nerd.", "followers_count": 218, "friends_count": 204, "statues_count": 4090 }, "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("2015-11-25T10:00:27.000Z"), "id": 669576397602447364, "text": "@Chiefsmafiaaa @brian8556utube @Russgoat @rocketsffl the #17 QB lmao he aint doing shit on our defense", "in_reply_to_status": 669574336529850368, "in_reply_to_user": 4294195512, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4294195512, 572981684, 2990611854, 3406586975 }}, "user": { "id": 3282839594, "name": "TyGod Taylor", "screen_name": "BSarow08", "lang": "en", "location": "null", "create_at": date("2015-07-17"), "description": "FL born. NC raised. H-Town made. English major at UofH. SPORTS. #OneBuffalo @HoustonRockets @UNC_Basketball @BuffaloBills @BuffaloSabres @Indians @GatorsFB", "followers_count": 430, "friends_count": 1406, "statues_count": 14785 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:27.000Z"), "id": 669576397619228673, "text": "Accident cleared in #Bklyn:OnTheB.q.e. on I-278 WB at Metropolitan Ave, stop and go traffic back to The L.I.E., delay of 4 mins #traffic", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.94829,40.71769"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Bklyn", "traffic" }}, "user": { "id": 42640432, "name": "Total Traffic NYC", "screen_name": "TotalTrafficNYC", "lang": "en", "location": "New York City & New Jersey", "create_at": date("2009-05-26"), "description": "Total Traffic Network delivers real-time traffic via in-car and portable navigation, broadcast media, wireless, & Internet. Call Tips Hands-Free to 212-377-7757", "followers_count": 1973, "friends_count": 366, "statues_count": 303327 }, "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-25T10:00:28.000Z"), "id": 669576397787009025, "text": "@juliazepeda_ let's hang out.", "in_reply_to_status": -1, "in_reply_to_user": 623754764, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 623754764 }}, "user": { "id": 298614200, "name": "octobersveryown.", "screen_name": "JayHardiee", "lang": "en", "location": "texas ", "create_at": date("2011-05-14"), "description": "jaylyn.", "followers_count": 774, "friends_count": 881, "statues_count": 22791 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576397820551169, "text": "@jrcaballero2 this why we have trust issues lmao https://t.co/Zd8NlCHOOs", "in_reply_to_status": -1, "in_reply_to_user": 717821326, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 717821326 }}, "user": { "id": 615428332, "name": "9 Gawd.", "screen_name": "joancastillo_", "lang": "en", "location": "New Mexico, USA.", "create_at": date("2012-06-22"), "description": "con mis amistadas siempre estoy al cien. #CowboysNation.", "followers_count": 728, "friends_count": 320, "statues_count": 43581 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576397858299904, "text": "https://t.co/40ujA6exYQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1447422110, "name": "jessica vey", "screen_name": "VeyJessica", "lang": "en", "location": "San Diego, CA", "create_at": date("2013-05-21"), "description": "take it sleazy", "followers_count": 388, "friends_count": 179, "statues_count": 2711 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576397917167617, "text": "someone let me come over������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3171386508, "name": "Em❤️", "screen_name": "em_xoxo311", "lang": "en", "location": "null", "create_at": date("2015-04-24"), "description": "vete a la mierda", "followers_count": 281, "friends_count": 320, "statues_count": 10546 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576398055583744, "text": "Been living Reckless I hate it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 79861549, "name": "✈YoungWavy✈", "screen_name": "GzUp224", "lang": "en", "location": "Los Angeles, CA ✈️Columbus OH", "create_at": date("2009-10-04"), "description": "California Living ✈️ to Columbus,OH Live Fa$t Die Young! Sip Slow Live Fa$t #DREAMCHASER #RIPGRANDMA #RIPRJ224 #RIPFA$E", "followers_count": 634, "friends_count": 662, "statues_count": 31075 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576398059790336, "text": "https://t.co/O4TNvIb4kQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 595972564, "name": "kurt schmidt", "screen_name": "kurtschmidt5", "lang": "en", "location": "null", "create_at": date("2012-05-31"), "description": "i love boobs and thick woman", "followers_count": 1367, "friends_count": 2051, "statues_count": 103759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Collingdale, PA", "id": "8c2c71cbc11fc837", "name": "Collingdale", "place_type": "city", "bounding_box": rectangle("-75.291543,39.905801 -75.263234,39.923218") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42045, "countyName": "Delaware", "cityID": 4215232, "cityName": "Collingdale" } }
+{ "create_at": datetime("2015-11-25T10:00:28.000Z"), "id": 669576398374178817, "text": "#ghettofabulous @ The Home Depot https://t.co/Yg4kAEHtlg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.19850104,40.73183137"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ghettofabulous" }}, "user": { "id": 222677841, "name": "Daniel Munro", "screen_name": "danielmunro1988", "lang": "en", "location": "Newark, NJ", "create_at": date("2010-12-03"), "description": "Freelance sound mixer looking for work in the greater New York City area", "followers_count": 92, "friends_count": 372, "statues_count": 2758 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576398390951936, "text": "This year I'm thankful that I'm being allowed to use granular synthesis on a pop record @ Vinegar… https://t.co/epwTOFjxih", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9848175,40.7033043"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 58634215, "name": "Eben™", "screen_name": "ebendamico", "lang": "en", "location": "top of the middle ", "create_at": date("2009-07-20"), "description": "Creator, collaborator, has all original teeth", "followers_count": 465, "friends_count": 332, "statues_count": 6762 }, "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-25T10:00:28.000Z"), "id": 669576398651150337, "text": "Every day i think about my next move", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2645811723, "name": "Tiaaaamarie", "screen_name": "MsKillaT_", "lang": "en", "location": "null", "create_at": date("2014-06-26"), "description": "Bosston. 24", "followers_count": 236, "friends_count": 499, "statues_count": 3285 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576398730727424, "text": "@bieberxking Follow for DM", "in_reply_to_status": -1, "in_reply_to_user": 56006745, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 56006745 }}, "user": { "id": 327622460, "name": "Joe", "screen_name": "JoeDoyleWSU", "lang": "en", "location": "Kent, WA", "create_at": date("2011-07-01"), "description": "Business Development Manager. WSU Grad, -- #GoCougs #GoHawks, #GoMariners, Über-enthusiast of sandwiches.", "followers_count": 721, "friends_count": 385, "statues_count": 22776 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576398785404934, "text": "@kiixooox @Callmee_snapple Y'all Quoted Me, Y'all Was In MY Notifications. DISMISSED", "in_reply_to_status": 669576234930741248, "in_reply_to_user": 3084904763, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3084904763, 578509050 }}, "user": { "id": 131060568, "name": "Zae ✨", "screen_name": "Badgal_zae", "lang": "en", "location": "Cali ✈ Clevietnam", "create_at": date("2010-04-08"), "description": "Israel's World | LongLiveAshont'e", "followers_count": 5373, "friends_count": 4086, "statues_count": 14575 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576398873485312, "text": "we get it, you vape. https://t.co/i2hvLRY7Po", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2677778393, "name": "ShawtyLiotti••", "screen_name": "nell_bell16", "lang": "en", "location": "Harrisburg, NC", "create_at": date("2014-07-05"), "description": "null", "followers_count": 369, "friends_count": 236, "statues_count": 4244 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576398890250240, "text": "Dumping wood Guts all Out the Window married to the money I ain't never Let'n Go ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1710814081, "name": "Mr.6⃣0⃣9⃣", "screen_name": "TownBoyStakz", "lang": "en", "location": "Trenton ", "create_at": date("2013-08-29"), "description": "Surround Yourself with People of Good Taste. live Life. #Jersey #609 IG:@gold2good #RIPAB", "followers_count": 1451, "friends_count": 1186, "statues_count": 13513 }, "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.325071 -74.619759,40.37435") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34021, "countyName": "Mercer", "cityID": 3460900, "cityName": "Princeton" } }
+{ "create_at": datetime("2015-11-25T10:00:28.000Z"), "id": 669576399049494528, "text": "Before you sit down at the table watch who you eating with.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 512968974, "name": "Treshawn Lambert", "screen_name": "Trey_Lamb", "lang": "en", "location": "James Madison University", "create_at": date("2012-03-03"), "description": "JMU ig:trey_lamb Rest In Paradise Grandma,Grandpa,and Uncle. I do this for you.", "followers_count": 1713, "friends_count": 1487, "statues_count": 64708 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576399112544262, "text": "@CasillaLater People freaking out about the Cap 3 trailer. I can't judge them. I am also freaking out about the Cap 3 trailer.", "in_reply_to_status": 669575997243584516, "in_reply_to_user": 17415717, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17415717 }}, "user": { "id": 7961132, "name": "SetPhasmaToStunning", "screen_name": "momebie", "lang": "en", "location": "Boston, MA", "create_at": date("2007-08-04"), "description": "Comics. Tattoos. Poetry. Wee giraffes. Intense selfie taking.", "followers_count": 1032, "friends_count": 897, "statues_count": 114747 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576399309549568, "text": "@quietAZmoney i told my father Mr. James Cervantez about you have u told ur sis Anthony Cruz; Precious Cruz bout me i want a Big Mac NOW��", "in_reply_to_status": -1, "in_reply_to_user": 20409967, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20409967 }}, "user": { "id": 3199413230, "name": "Ms. Preciosa", "screen_name": "5e3e26ad31bc4a1", "lang": "en", "location": "Ohio, USA", "create_at": date("2015-05-17"), "description": "smmfh", "followers_count": 60, "friends_count": 173, "statues_count": 1387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Cleveland, OH", "id": "36ba7375e4498732", "name": "East Cleveland", "place_type": "city", "bounding_box": rectangle("-81.598231,41.515406 -81.555538,41.548046") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3923380, "cityName": "East Cleveland" } }
+{ "create_at": datetime("2015-11-25T10:00:28.000Z"), "id": 669576399351607296, "text": "Receive FREE Daily Advent Refelctions >>> https://t.co/m4z9Q7SDaV https://t.co/hUJnQsIFe6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 140129683, "name": "uCatholic", "screen_name": "uCatholic", "lang": "en", "location": "Cleveland, Ohio", "create_at": date("2010-05-04"), "description": "http://www.uCatholic.com is dedicated to providing traditional Catholic information in the modern world.", "followers_count": 6083, "friends_count": 103, "statues_count": 7183 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576399376793600, "text": "I literally just got into an argument over juice w my roommate", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 299381441, "name": "MICROSOFT WORD THOT", "screen_name": "nyranology", "lang": "en", "location": "nyc", "create_at": date("2011-05-15"), "description": "nah", "followers_count": 279, "friends_count": 325, "statues_count": 15530 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576399577968640, "text": "Laura Send Me A Friend Request on Facebook & Sc & I Ain't Adding Her ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1967996322, "name": "PabloP", "screen_name": "pobs_perez2013", "lang": "en", "location": "Evergreen State Washington", "create_at": date("2013-10-17"), "description": "Moneys My Motivation Sc . pops_2013", "followers_count": 305, "friends_count": 223, "statues_count": 10844 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576399590699009, "text": "@morinap the one of the kid in Cleveland, no one even checked to see if he was dead. Just left him there for minutes.", "in_reply_to_status": 669575023758319616, "in_reply_to_user": 5152561, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 5152561 }}, "user": { "id": 14493902, "name": "Wendy Packard", "screen_name": "WendEclair", "lang": "en", "location": "Chicago, IL, USA", "create_at": date("2008-04-23"), "description": "Chicago. Engineer, Lawyer, trying to escape to California.", "followers_count": 379, "friends_count": 388, "statues_count": 44704 }, "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-25T10:00:28.000Z"), "id": 669576399657660416, "text": "I'm over here really missing my baby���� my sister needs to get off nowwaaaa��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 632513186, "name": "Lexis", "screen_name": "themcakestho", "lang": "en", "location": "null", "create_at": date("2012-07-10"), "description": "SC:lexis_texas19 IG:lexis_monet ✨", "followers_count": 1036, "friends_count": 644, "statues_count": 31330 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576399661826048, "text": "Thanksgiving with my family is NOTHING but arguing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 837628274, "name": "J00lian", "screen_name": "mellamosaxxy", "lang": "en", "location": "San Antonio, TX", "create_at": date("2012-09-21"), "description": "WHS Band • VP • EDM", "followers_count": 381, "friends_count": 275, "statues_count": 8233 }, "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-25T10:00:28.000Z"), "id": 669576400798633984, "text": "Gave thanks to my fans out there", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 214572543, "name": "TheGatewayClanGamer", "screen_name": "Streetofgateway", "lang": "en", "location": "www.YouTube.com/Lil G ", "create_at": date("2010-11-11"), "description": "Love doing Rap Career playing video games and playing Street ball watching the game and wrestling movies BET showing people and fans having good time", "followers_count": 80, "friends_count": 447, "statues_count": 1540 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576401134198784, "text": "With that being said", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 446463241, "name": "chuck tweets", "screen_name": "ChuckHunnas", "lang": "en", "location": "wherever a yung hunna be", "create_at": date("2011-12-25"), "description": "lebron time", "followers_count": 678, "friends_count": 1236, "statues_count": 17138 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576401264189440, "text": "Ann Arbor bound ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 298291021, "name": "Annie", "screen_name": "_ayyrudz", "lang": "en", "location": "Michigan", "create_at": date("2011-05-13"), "description": "Dessert enthusiast. WMU '18", "followers_count": 353, "friends_count": 178, "statues_count": 11051 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576401318711297, "text": "when your fav cousin get to the house for thanksgiving �������� https://t.co/oWKPo47Xim", "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": "Cam Curry❗️3️⃣0️⃣", "screen_name": "ayooocam", "lang": "en", "location": "Pickerington, OH", "create_at": date("2011-04-11"), "description": "Everyone calls me Cam, but you can call me tonight :). Combo Guard @ Ohio U Lancaster!!LOADING..... #30 RIP Will & Q", "followers_count": 13972, "friends_count": 10955, "statues_count": 101664 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576401515847680, "text": "STUPID DOG , YOU MAKE ME LOOK BAD , UGABUBAUBUGAUBUAA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 1559093083, "name": "@-♫Mentor-beats♫-#", "screen_name": "MentorBeats_", "lang": "en", "location": "Sunrise, FL", "create_at": date("2013-06-30"), "description": "Your Friendly Neighborhood Producer! Contact Me: Gripstate@gmail.com", "followers_count": 139, "friends_count": 77, "statues_count": 8287 }, "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("2015-11-25T10:00:28.000Z"), "id": 669576401545072640, "text": "@nicklockwood @tadeuzagallo ����", "in_reply_to_status": 669508387583971328, "in_reply_to_user": 22920215, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 22920215, 584229130 }}, "user": { "id": 233679538, "name": "Daniel SABBATICson", "screen_name": "dantoml", "lang": "en", "location": "London - Travelling: San Francisco", "create_at": date("2011-01-03"), "description": "Freelance iOS Engineer currently on sabbatical. \nPronoun: they. \n❤︎ @henriwatson. \nYou can email me: dan@tomlinson.io", "followers_count": 1717, "friends_count": 523, "statues_count": 38205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Union Square, San Francisco", "id": "84484af0f9d1cded", "name": "Union Square", "place_type": "neighborhood", "bounding_box": rectangle("-122.413931,37.780905 -122.401991,37.792852") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2015-11-25T10:00:28.000Z"), "id": 669576401574625280, "text": "that sad moment when you're hair dryer breaks so you have to dry your hair in the car", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3025738894, "name": "karissa penrod", "screen_name": "PenrodKarissa", "lang": "en", "location": "Kettering, OH", "create_at": date("2015-02-08"), "description": "null", "followers_count": 277, "friends_count": 411, "statues_count": 7068 }, "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("2015-11-25T10:00:29.000Z"), "id": 669576401939337216, "text": "@FreeRangeMegg LOL.Say whatever your heart desires. I am no fan of Asad. Just think that Syrian problem can not be solved without Asad.:))", "in_reply_to_status": 669574815624396801, "in_reply_to_user": 2717487895, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2717487895 }}, "user": { "id": 2750009240, "name": "Zaffar A", "screen_name": "SZaffar5", "lang": "en", "location": " WA, USA", "create_at": date("2014-08-20"), "description": "A passionate Aerospace engineer. Against oppression/injustice. US my land of opportunity, PAK my land of hope .A free man, yet happy to be enslaved by LAW /LOVE", "followers_count": 1177, "friends_count": 1807, "statues_count": 9705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mukilteo, WA", "id": "00acddf65083d6cc", "name": "Mukilteo", "place_type": "city", "bounding_box": rectangle("-122.328148,47.859946 -122.219961,47.951882") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5347735, "cityName": "Mukilteo" } }
+{ "create_at": datetime("2015-11-25T10:00:29.000Z"), "id": 669576402367356932, "text": "Nimbus Fish Hatchery Weir. Removable barrier, only in place during the salmon run, prevents the… https://t.co/sPvxQodz7Z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.22499658,38.63434121"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20911042, "name": "Chris", "screen_name": "cclauson", "lang": "en", "location": "Fair Oaks, CA", "create_at": date("2009-02-15"), "description": "Husband. Father. Cyclist. Runner. Love my girls. #GoBears", "followers_count": 89, "friends_count": 171, "statues_count": 1800 }, "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("2015-11-25T10:00:29.000Z"), "id": 669576402560094209, "text": "https://t.co/0PB7wmKXkv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4269164113, "name": "heatmefor", "screen_name": "amazinggirls17", "lang": "en", "location": "null", "create_at": date("2015-11-17"), "description": "null", "followers_count": 2, "friends_count": 52, "statues_count": 55 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit Beach, MI", "id": "018864367d6cf2fb", "name": "Detroit Beach", "place_type": "city", "bounding_box": rectangle("-83.342625,41.906474 -83.317784,41.939387") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26115, "countyName": "Monroe", "cityID": 2622040, "cityName": "Detroit Beach" } }
+{ "create_at": datetime("2015-11-25T10:00:29.000Z"), "id": 669576402694352896, "text": "This morning's sharing of my New York with with a Korean travel magazine. Reminded the world is so… https://t.co/DW9tIPCK2F", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.96555556,40.76916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14247990, "name": "Willa Kammerer", "screen_name": "willakammerer", "lang": "en", "location": "New York City / New England", "create_at": date("2008-03-28"), "description": "FIRESTARTER | STORYTELLER | Founder @firestarterIX | explorer, traveler, life/style enthusiast, wellness warrior & nature girl", "followers_count": 981, "friends_count": 972, "statues_count": 6216 }, "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("2015-11-25T10:00:29.000Z"), "id": 669576402748837889, "text": "*tia trying to get me over it* \n\"He's ugly\" \n\"He's 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": 1140478566, "name": "haley", "screen_name": "long_haleyy", "lang": "en", "location": "friendswood, tx", "create_at": date("2013-02-01"), "description": "hmtfu", "followers_count": 654, "friends_count": 498, "statues_count": 7769 }, "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("2015-11-25T10:00:29.000Z"), "id": 669576402757283840, "text": "I can't trust a soul", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2727458631, "name": "WOLF", "screen_name": "iamdeahvian", "lang": "en", "location": "Wichita, KS", "create_at": date("2014-07-28"), "description": "By any means necessary", "followers_count": 1041, "friends_count": 1283, "statues_count": 26831 }, "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("2015-11-25T10:00:29.000Z"), "id": 669576403591958528, "text": "Leo https://t.co/T9L9RfOV0E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 514658394, "name": "King of Kings™", "screen_name": "Iam_Bry22", "lang": "en", "location": "null", "create_at": date("2012-03-04"), "description": "(901) ✈ (402) . “There may be people that have more talent than you, but theres no excuse for anyone to work harder than you do.” – Derek Jeter #EK3 #TKT #DR", "followers_count": 559, "friends_count": 456, "statues_count": 15855 }, "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("2015-11-25T10:00:29.000Z"), "id": 669576403990396928, "text": "Thanks Christian!! https://t.co/7kOsLLC2Se", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 318656995, "name": "Jessica Valadez", "screen_name": "jayvaladez", "lang": "en", "location": "California, USA", "create_at": date("2011-06-16"), "description": "19. GCU'17⚽. Mexican National Team. Seleccion Mexicana Femenil. snapchat: jessvaladez", "followers_count": 3115, "friends_count": 554, "statues_count": 22283 }, "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("2015-11-25T10:00:29.000Z"), "id": 669576404174921728, "text": "Anybody else remember when moshing was called \"slam dancing\"?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1400554945, "name": "LordAsmodeus", "screen_name": "Lrodgers74", "lang": "en", "location": "null", "create_at": date("2013-05-03"), "description": "Grim and Frostbitten Banana", "followers_count": 492, "friends_count": 593, "statues_count": 2138 }, "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-25T10:00:29.000Z"), "id": 669576404179226624, "text": "���������������������� https://t.co/gixWNskIpL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 156110658, "name": "188-HOT-LEGS", "screen_name": "iwinsoiwin", "lang": "en", "location": "null", "create_at": date("2010-06-15"), "description": "You trying to tell me you're a bigger knob than you've been letting on?", "followers_count": 526, "friends_count": 453, "statues_count": 85954 }, "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-25T10:00:29.000Z"), "id": 669576404212817920, "text": "No one is too busy It's just a matter of priorities.. @ Philadelphia, Pennsylvania https://t.co/i8hPU7kWcy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.1448,39.998"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 779520006, "name": "Eman Alshami", "screen_name": "Eman_a7", "lang": "ar", "location": "Ohio, USA", "create_at": date("2012-08-24"), "description": "Instagram: eman23alshami", "followers_count": 49, "friends_count": 61, "statues_count": 1267 }, "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-25T10:00:29.000Z"), "id": 669576404326068224, "text": "MY BABY �������� https://t.co/qPFzM54pUH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 636366231, "name": "lilly", "screen_name": "AguilarLilly", "lang": "en", "location": "Dade City, FL", "create_at": date("2012-07-15"), "description": "null", "followers_count": 283, "friends_count": 534, "statues_count": 13577 }, "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("2015-11-25T10:00:29.000Z"), "id": 669576404388945921, "text": "having little feet is a blessing lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 411277029, "name": "Unessciaaa", "screen_name": "_unknownuser___", "lang": "en", "location": "Clearwater,FL", "create_at": date("2011-11-12"), "description": "patiently waiting on my prince or princess to arrive ❤️", "followers_count": 3444, "friends_count": 2479, "statues_count": 121413 }, "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("2015-11-25T10:00:29.000Z"), "id": 669576404686737408, "text": "I really do hate the dentist…. Like nick open your mouth and I’m like no I don’t swing that way sorry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.35718584,41.04927347"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 390313549, "name": "Nick Graziani", "screen_name": "GrazianiN", "lang": "en", "location": "Clarion university", "create_at": date("2011-10-13"), "description": "#italian #22 #CLARION UNIV. #FOLLOWME #Instagram:@GRAZIANI1823\n#snapchat: Graz1823", "followers_count": 1087, "friends_count": 1913, "statues_count": 18459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Castle, PA", "id": "2a903a64e4815dbe", "name": "New Castle", "place_type": "city", "bounding_box": rectangle("-80.40045,40.937588 -80.26887,41.070935") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42073, "countyName": "Lawrence", "cityID": 4253368, "cityName": "New Castle" } }
+{ "create_at": datetime("2015-11-25T10:00:29.000Z"), "id": 669576404783120384, "text": "The HELP! #BeatlesAFilm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BeatlesAFilm" }}, "user": { "id": 2605930386, "name": "Jeffrey Chavez", "screen_name": "area51jeff", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-07-05"), "description": "null", "followers_count": 520, "friends_count": 327, "statues_count": 6997 }, "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("2015-11-25T10:00:29.000Z"), "id": 669576404904706048, "text": "It's gonna hurt. It's gonna hurt because it matters.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3267495902, "name": "¯\\_(ツ)_/¯", "screen_name": "cambree_jade", "lang": "en", "location": "null", "create_at": date("2015-07-03"), "description": "I tweet what I agree with not what I'm going through people tend to confuse the two.", "followers_count": 156, "friends_count": 153, "statues_count": 1655 }, "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("2015-11-25T10:00:29.000Z"), "id": 669576405051555841, "text": "Menorah Park Center for Senior L... #Nursing #Job: Resident Associate (#Beachwood, OH) https://t.co/IycnKgGXJ8 #STNA #training #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4899224,41.5011199"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "Job", "Beachwood", "STNA", "training", "Jobs", "Hiring" }}, "user": { "id": 214122801, "name": "MenorahParkJobs", "screen_name": "MenorahParkJobs", "lang": "en", "location": "Beachwood, Ohio", "create_at": date("2010-11-10"), "description": "Menorah Park Center for Senior Living is one of the largest senior living communities in the state of Ohio.", "followers_count": 127, "friends_count": 82, "statues_count": 1378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beachwood, OH", "id": "fa5bd676191716ee", "name": "Beachwood", "place_type": "city", "bounding_box": rectangle("-81.532635,41.449669 -81.48037,41.501448") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3904500, "cityName": "Beachwood" } }
+{ "create_at": datetime("2015-11-25T10:00:29.000Z"), "id": 669576405059923968, "text": "You pay me how much YOU think you should lol or however much we talk about. You don't just ask how much money I want. I want 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": 236738184, "name": "Supreme Kate Moss", "screen_name": "BluntDoc_", "lang": "en", "location": "OREGON", "create_at": date("2011-01-10"), "description": "[i rt fuckery] Biracial Butterfly, Yung Mama #SASSYSADGRL", "followers_count": 3347, "friends_count": 1889, "statues_count": 125592 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, OR", "id": "4be92bcdb7106ede", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-122.842306,42.263198 -122.798235,42.293361") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41029, "countyName": "Jackson", "cityID": 4157500, "cityName": "Phoenix" } }
+{ "create_at": datetime("2015-11-25T10:00:29.000Z"), "id": 669576405252972544, "text": "reason #8482 why I shouldn't babysit: \"I wish aliens would abduct you so I only had to watch your brother and sister and 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": 177428373, "name": "lex", "screen_name": "lparrilli_", "lang": "en", "location": "shitcago", "create_at": date("2010-08-11"), "description": "19 and still terrified of the dark", "followers_count": 535, "friends_count": 326, "statues_count": 52874 }, "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-25T10:00:29.000Z"), "id": 669576405257203712, "text": "#TOTM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "TOTM" }}, "user": { "id": 43482933, "name": "RIP Javion", "screen_name": "Bankroll_way", "lang": "en", "location": "null", "create_at": date("2009-05-29"), "description": "#BRMBP #MoBB #RipJavion #JerkGangBIPBabyJ Guddababy229@gmail.com", "followers_count": 1808, "friends_count": 1689, "statues_count": 181597 }, "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("2015-11-25T10:00:29.000Z"), "id": 669576405508755456, "text": "Selfieeeee���� https://t.co/Jds22TW8eF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2366383902, "name": "kaylynnnnn :)", "screen_name": "KaylynConnertyy", "lang": "en", "location": "Stigler, OK", "create_at": date("2014-02-28"), "description": "man, I'm just living", "followers_count": 513, "friends_count": 424, "statues_count": 7312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Goose Creek, SC", "id": "7a789fc97175b3f1", "name": "Goose Creek", "place_type": "city", "bounding_box": rectangle("-80.101267,32.961647 -79.967041,33.063457") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45015, "countyName": "Berkeley", "cityID": 4529815, "cityName": "Goose Creek" } }
+{ "create_at": datetime("2015-11-25T10:00:29.000Z"), "id": 669576405613543425, "text": "Want to work in #Chandler, Arizona? View our latest opening: https://t.co/niWOfcgScJ #Retail #Job #Jobs #Hiring https://t.co/g0z9euyUGd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.8412502,33.3061605"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Chandler", "Retail", "Job", "Jobs", "Hiring" }}, "user": { "id": 614471194, "name": "Nordstrom Jobs", "screen_name": "JobsAtNordstrom", "lang": "en", "location": "null", "create_at": date("2012-06-21"), "description": "null", "followers_count": 159, "friends_count": 0, "statues_count": 5743 }, "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-25T10:00:29.000Z"), "id": 669576406075056128, "text": "Laid Ent x Hear 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": 2730415702, "name": "Tj Bryson", "screen_name": "Poloteamtj", "lang": "en", "location": "null", "create_at": date("2014-07-31"), "description": "taken !!", "followers_count": 726, "friends_count": 741, "statues_count": 3381 }, "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("2015-11-25T10:00:30.000Z"), "id": 669576406754430976, "text": "https://t.co/ZBURTE203U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 442186923, "name": "j", "screen_name": "jazIynn", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-12-20"), "description": "i neED Me A HyNa ~~", "followers_count": 854, "friends_count": 214, "statues_count": 7887 }, "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("2015-11-25T10:00:30.000Z"), "id": 669576406771195904, "text": "need new people to text", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 354188511, "name": "N.", "screen_name": "nayonceeeee", "lang": "en", "location": "DLGJ ❤️", "create_at": date("2011-08-13"), "description": "humble. & all about shemar. ❤️", "followers_count": 3570, "friends_count": 2588, "statues_count": 90125 }, "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("2015-11-25T10:00:30.000Z"), "id": 669576407375200256, "text": "Pulp bi 5 sene once de bana bu kadar iyi geliyor muydu, yoksa su anda zamanlamasi mi harika?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "user": { "id": 251569134, "name": "Eva", "screen_name": "godzilladosun", "lang": "en", "location": "Seattle, WA", "create_at": date("2011-02-13"), "description": "LL.M. Student at @UWSchoolofLaw, a member of Istanbul Bar Association, #Destinythegame fan", "followers_count": 301, "friends_count": 144, "statues_count": 10606 }, "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-25T10:00:30.000Z"), "id": 669576407396311041, "text": "@tibbslexi DON'T LISTEN TO THE \"all brown\" GROUP. PUT THOSE VOTES IN THE BLUE GROUL", "in_reply_to_status": 669570357678993408, "in_reply_to_user": 2984327806, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2984327806 }}, "user": { "id": 568710703, "name": "♊Riss♊", "screen_name": "Boo_Bear_18", "lang": "en", "location": "✌️Maine✌️", "create_at": date("2012-05-01"), "description": "Swim, Dance, Netflix, & Younglife #RyderTribe {Snapchat: rissabear18}\nRomans 8:31", "followers_count": 353, "friends_count": 271, "statues_count": 21984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sanford, ME", "id": "46634c3ba5a46fdb", "name": "Sanford", "place_type": "city", "bounding_box": rectangle("-70.809516,43.414359 -70.749894,43.462986") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23031, "countyName": "York", "cityID": 2365725, "cityName": "Sanford" } }
+{ "create_at": datetime("2015-11-25T10:00:30.000Z"), "id": 669576407404576769, "text": "@wizzyjr @SKTV_ lmaooooooooo on cue", "in_reply_to_status": 669543320004227072, "in_reply_to_user": 30556102, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 30556102, 279156307 }}, "user": { "id": 61366106, "name": "He Is I And I Am Him", "screen_name": "Flyer_thanu2", "lang": "en", "location": "Chicago/Blue Island, IL", "create_at": date("2009-07-29"), "description": "28 years old #80sBaby I'm only here for laughs, sports, and music. Forget a debate #49ers #Lakers #OKC Instagram: Kenny_DatDude", "followers_count": 4748, "friends_count": 3990, "statues_count": 534627 }, "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("2015-11-25T10:00:30.000Z"), "id": 669576407467450368, "text": "See our latest #Towson, MD #job and click to apply: Dessert Finisher/Espresso Maker - https://t.co/t69B7EHNMl #Hospitality #Veterans", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.59934,39.40377"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Towson", "job", "Hospitality", "Veterans" }}, "user": { "id": 22454634, "name": "TMJ-BAL HRTA Jobs", "screen_name": "tmj_bal_hrta", "lang": "en", "location": "Baltimore, MD", "create_at": date("2009-03-01"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Baltimore, MD. Need help? Tweet us at @CareerArc!", "followers_count": 388, "friends_count": 293, "statues_count": 401 }, "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("2015-11-25T10:00:30.000Z"), "id": 669576407576506368, "text": "@naaaaaomeeei ��", "in_reply_to_status": 669576251561000960, "in_reply_to_user": 3345743914, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3345743914 }}, "user": { "id": 261931991, "name": "lil neicy mane❣", "screen_name": "__lilneicyyy", "lang": "en", "location": "Detroit2Dallas2Houston", "create_at": date("2011-03-06"), "description": "i like mula don't like rumors", "followers_count": 2042, "friends_count": 1329, "statues_count": 53423 }, "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("2015-11-25T10:00:30.000Z"), "id": 669576407622795264, "text": "@camry_k3 HAHAHAHAHAHHHAHHA that hair thou", "in_reply_to_status": 669560761203556352, "in_reply_to_user": 3270330402, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 3270330402 }}, "user": { "id": 393029648, "name": "Maria Macaluso", "screen_name": "maria_macaluso", "lang": "en", "location": "probably on youtube", "create_at": date("2011-10-17"), "description": "Just a radical dame who likes to play games", "followers_count": 477, "friends_count": 366, "statues_count": 7810 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hollow Creek, KY", "id": "75d0bdd5ab88e72d", "name": "Hollow Creek", "place_type": "city", "bounding_box": rectangle("-85.629439,38.148145 -85.618228,38.157413") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2137576, "cityName": "Hollow Creek" } }
+{ "create_at": datetime("2015-11-25T10:00:30.000Z"), "id": 669576407836528640, "text": "@CokeBoyBluerr I went the the other night car full of niggas one of em told me he seen me like 4 times and where do I stay", "in_reply_to_status": 669575651230257152, "in_reply_to_user": 1398937831, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1398937831 }}, "user": { "id": 512838848, "name": "Cris", "screen_name": "Roll_Aswisher", "lang": "en", "location": "null", "create_at": date("2012-03-02"), "description": "null", "followers_count": 439, "friends_count": 270, "statues_count": 31497 }, "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("2015-11-25T10:00:30.000Z"), "id": 669576407878672384, "text": "Better Late than Never: 'His Girl Friday' (1940) https://t.co/hMKG2nMiSm", "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": 271167530, "name": "The Guys From", "screen_name": "guysfromblank", "lang": "en", "location": "Calgary, Alberta", "create_at": date("2011-03-23"), "description": "Home of sarcastic insight into film, tv, sports, news and gaming. Plus news on The Guys From {BLANK} comedy", "followers_count": 246, "friends_count": 18, "statues_count": 482 }, "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("2015-11-25T10:00:30.000Z"), "id": 669576407908024321, "text": "Shoutout to Treece for not sewing this shit in tight ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3950721389, "name": "2 more days✨", "screen_name": "Ashhhhlayyy", "lang": "en", "location": "Starbucks", "create_at": date("2015-10-13"), "description": "• Matthew 7:1• Beyonce & Chance & JB❤️", "followers_count": 288, "friends_count": 256, "statues_count": 9599 }, "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-25T10:00:30.000Z"), "id": 669576408088342528, "text": "Aggy lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2159186074, "name": "Chloe Rogers", "screen_name": "_callmechloeeee", "lang": "en", "location": "Graceville, FL", "create_at": date("2013-10-29"), "description": "Marley ❤️ || JCM.RLAB.BMD. || GHS || 15", "followers_count": 443, "friends_count": 633, "statues_count": 9052 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Graceville, FL", "id": "01b6f2db63f115e2", "name": "Graceville", "place_type": "city", "bounding_box": rectangle("-85.530538,30.945965 -85.495801,30.97536") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12063, "countyName": "Jackson", "cityID": 1227000, "cityName": "Graceville" } }
+{ "create_at": datetime("2015-11-25T10:00:30.000Z"), "id": 669576408423923713, "text": "It's hard for me to trust cuz niggas fake to much ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 368036793, "name": "♒️", "screen_name": "_PrettySade", "lang": "en", "location": "Somewhere Getting Money", "create_at": date("2011-09-04"), "description": "First You Get That Money Then You Get That POWER ‼️", "followers_count": 1480, "friends_count": 2089, "statues_count": 6830 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland, TN", "id": "6ad5fc2baffd0f57", "name": "Cleveland", "place_type": "city", "bounding_box": rectangle("-84.945705,35.120064 -84.786995,35.266608") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47011, "countyName": "Bradley", "cityID": 4715400, "cityName": "Cleveland" } }
+{ "create_at": datetime("2015-11-25T10:00:30.000Z"), "id": 669576408532959232, "text": "i still be pretending like im cool with these niggas i know is hoe niggas. it aint even pretendin foreal i just know yall gettin left.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3371826807, "name": "Left Earth Again", "screen_name": "HeyPablo_Felipe", "lang": "en", "location": "I was never really here", "create_at": date("2015-07-11"), "description": "This never really happened", "followers_count": 92, "friends_count": 86, "statues_count": 3036 }, "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-25T10:00:30.000Z"), "id": 669576408578945026, "text": "@giannabelluomin thank you����", "in_reply_to_status": 669448753825320960, "in_reply_to_user": 1626860984, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1626860984 }}, "user": { "id": 437630414, "name": "Hannah", "screen_name": "hannertimeee", "lang": "en", "location": "null", "create_at": date("2011-12-15"), "description": "UNR '19 #wasntme", "followers_count": 810, "friends_count": 334, "statues_count": 12675 }, "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("2015-11-25T10:00:30.000Z"), "id": 669576408729960448, "text": "You know you're an adult when the shit you're asking for for Christmas is shit you actually NEED instead of shit you want ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.04590117,39.94099761"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 865311019, "name": "◡̈C NA$TY◡̈", "screen_name": "cassandradearr", "lang": "en", "location": "•CO•", "create_at": date("2012-10-06"), "description": "✌Ride or Die✌", "followers_count": 433, "friends_count": 280, "statues_count": 18492 }, "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-25T10:00:30.000Z"), "id": 669576408755273728, "text": "How was it ? �� https://t.co/UmrF0vltyZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 105671137, "name": "Nate The Great™", "screen_name": "natethegoat_", "lang": "en", "location": "null", "create_at": date("2010-01-16"), "description": "I'm Nate The Great, and there ain't no greater, but God. Follow on Snap @thegreatnate_1 East Carolina University '18", "followers_count": 720, "friends_count": 1777, "statues_count": 10304 }, "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("2015-11-25T10:00:30.000Z"), "id": 669576408780447746, "text": "Darkness Looms - New Comics Wednesday https://t.co/HfbJuuGeFC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-106.6700458,52.1332144"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4170126613, "name": "Aaron Munro", "screen_name": "aaronamunro", "lang": "en", "location": "Saskatoon, Saskatchewan", "create_at": date("2015-11-08"), "description": "SK family man. Lover of God, humanity & earth. Religious, spiritual and nerdy. Blog at http://aaronamunro.com - Podcast at @whatdoimean_pc w/ @dublinsomeday.", "followers_count": 309, "friends_count": 1232, "statues_count": 159 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Saskatoon, Saskatchewan", "id": "4bb41f9d86e16416", "name": "Saskatoon", "place_type": "city", "bounding_box": rectangle("-106.824954,52.069634 -106.503696,52.230521") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2015-11-25T10:00:30.000Z"), "id": 669576408893648896, "text": "How to make a Rob Roy https://t.co/7e7Hf0HERT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9271644,40.7595044"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2513457279, "name": "BYBER", "screen_name": "byberapp", "lang": "en", "location": "New York", "create_at": date("2014-04-27"), "description": "BYBER helps you meet new people, connect with friends and explore what your neighborhood has to offer. \n\n\n\n\nMeet - Connect - Explore!", "followers_count": 363, "friends_count": 83, "statues_count": 558 }, "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("2015-11-25T10:00:30.000Z"), "id": 669576409065627648, "text": "@graciie__xo who, where ya @ button at hummmmmm", "in_reply_to_status": 669562399599558656, "in_reply_to_user": 1492560271, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1492560271 }}, "user": { "id": 125789625, "name": "zee", "screen_name": "cutienomber1", "lang": "en", "location": "philly", "create_at": date("2010-03-23"), "description": "#25 #2016 #newyork #philly #teammixed work hard play harder keep ya head up and never give up #musician", "followers_count": 355, "friends_count": 407, "statues_count": 2697 }, "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-25T10:00:30.000Z"), "id": 669576409094860800, "text": "Get you PJ Rose apparel here! #official @WindyCityLIVE is having a today-only sale! https://t.co/ObmnfovG8E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "official" }}, "user_mentions": {{ 252173791 }}, "user": { "id": 64573701, "name": "Prince", "screen_name": "_iPrince", "lang": "en", "location": "Chicago", "create_at": date("2009-08-10"), "description": "Snapchat: i.prince // Husband. Dad. Christian. Est. 1984. Uncompromised. #Team #TeamBRINGIT #Yeshua", "followers_count": 328, "friends_count": 328, "statues_count": 29573 }, "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("2015-11-25T10:00:30.000Z"), "id": 669576409170489348, "text": "So that's what we're doing today?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17017064, "name": "JadeBrieanne, Author", "screen_name": "BlackNerdJade", "lang": "en", "location": "Shooting in the gym", "create_at": date("2008-10-28"), "description": "I'm a great writer with an epic potty mouth. http://www.toysldrs.com #RIPYUSUF MY BOOK! ~ http://amzn.to/1IMIndM MY SITE!~ http://www.jadebrieanne.com", "followers_count": 2614, "friends_count": 820, "statues_count": 196784 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garner, NC", "id": "0a960df1a2b0abaf", "name": "Garner", "place_type": "city", "bounding_box": rectangle("-78.680992,35.649142 -78.546396,35.731468") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3725480, "cityName": "Garner" } }
+{ "create_at": datetime("2015-11-25T10:00:30.000Z"), "id": 669576409178906624, "text": "While my ravioli dough is resting, I might as well have lunch! (@ Brookland's Finest Bar & Kitchen - @blandsfinest) https://t.co/s3OIwvxWyA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.99094296,38.92940226"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1385189442 }}, "user": { "id": 339265530, "name": "Michael Fortunato", "screen_name": "MJFortunato", "lang": "en", "location": "Arlington, VA\n", "create_at": date("2011-07-20"), "description": "SEC Reporting at Computer Sciences Corporation", "followers_count": 56, "friends_count": 143, "statues_count": 1082 }, "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("2015-11-25T10:00:30.000Z"), "id": 669576409224998912, "text": "Never.. https://t.co/vYV3mWr0P4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 169247085, "name": "Clarence Mac™", "screen_name": "2ndstboi", "lang": "en", "location": "East Coast", "create_at": date("2010-07-21"), "description": "I'm living the American Dream!!!! Try it.....", "followers_count": 174, "friends_count": 183, "statues_count": 23522 }, "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("2015-11-25T10:00:30.000Z"), "id": 669576409627492352, "text": "@ItsReginaG @chrisstalita", "in_reply_to_status": 669546344449753088, "in_reply_to_user": 469517478, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 469517478, 170198000 }}, "user": { "id": 225613571, "name": "sandy⚡️", "screen_name": "ohsandythatgirl", "lang": "en", "location": "null", "create_at": date("2010-12-11"), "description": "Dream a little dream of me.", "followers_count": 134, "friends_count": 111, "statues_count": 17216 }, "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("2015-11-25T10:00:30.000Z"), "id": 669576409652838400, "text": "@kccustoms2001 @cristinalaila1 regardless, warnings were issued when Putin decided to enter Syria with arms, fighter jets, etc.", "in_reply_to_status": 669573844097761280, "in_reply_to_user": 62193172, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 62193172, 608471241 }}, "user": { "id": 3381987010, "name": "James M. Nelson", "screen_name": "Jmonroenelson", "lang": "en", "location": "Staten Island, NY", "create_at": date("2015-07-18"), "description": "Exec prot & limo Co owner. Motorcade coordinator & Advance 4 Reagan WH in NYC. CTAP. DOS. ATF. USSS. Customs. NYPD.", "followers_count": 48, "friends_count": 37, "statues_count": 932 }, "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-25T10:00:30.000Z"), "id": 669576410005159936, "text": "@VicoLudovico @PatriziaRametta @Papryka5 @AdrianaCioci @asemra Una vera meraviglia. .grazie Ludovico.", "in_reply_to_status": 669571477855932418, "in_reply_to_user": 2322261107, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user_mentions": {{ 2322261107, 479810905, 61292316, 1913671255, 145576592 }}, "user": { "id": 1644683106, "name": "Alberto Petroccione", "screen_name": "albertopetro2", "lang": "it", "location": "null", "create_at": date("2013-08-04"), "description": "Il coraggio è difendere certi valori sempre e comunque,pur sapendo di apparire scomodo a chi dice di condividere le tue idee.", "followers_count": 4757, "friends_count": 4936, "statues_count": 31864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Revere, MA", "id": "407b2c699286aab5", "name": "Revere", "place_type": "city", "bounding_box": rectangle("-71.033007,42.388503 -70.959648,42.4499") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2556585, "cityName": "Revere" } }
+{ "create_at": datetime("2015-11-25T10:00:30.000Z"), "id": 669576410189717509, "text": "@KPierceTV Thank you for all you do!", "in_reply_to_status": -1, "in_reply_to_user": 1310616553, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1310616553 }}, "user": { "id": 2207450964, "name": "DAAA-1A", "screen_name": "daaa_1a", "lang": "en", "location": "Detroit", "create_at": date("2013-11-21"), "description": "Educating, advocating, and promoting healthy aging for seniors since 1980. #theseniorsolution", "followers_count": 267, "friends_count": 699, "statues_count": 2919 }, "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-25T10:00:31.000Z"), "id": 669576410332295168, "text": "Use thighs as ear muffs szn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 162276142, "name": "Netflix n Drill", "screen_name": "qui_good", "lang": "en", "location": "East Orange, NJ", "create_at": date("2010-07-02"), "description": "'Cuse Alum / DB Nupe", "followers_count": 1869, "friends_count": 1750, "statues_count": 165771 }, "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("2015-11-25T10:00:31.000Z"), "id": 669576410491695104, "text": "Why Holidays Are Good For Your Business https://t.co/1qMhuJH8LU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.1182951,40.9818873"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3058328650, "name": "Bright and Happy", "screen_name": "brighthappyplnr", "lang": "en", "location": "San Diego", "create_at": date("2015-02-23"), "description": "Marketing strategist & Virtual Assistant | I help small businesses create marketing strategy plans, automate systems and manage social media.", "followers_count": 123, "friends_count": 56, "statues_count": 2545 }, "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("2015-11-25T10:00:31.000Z"), "id": 669576410864877568, "text": "@ryanbroekhoff ne biliyim hoca gözden çıkardı mı acaba diye düşündüm", "in_reply_to_status": 669576278702473217, "in_reply_to_user": 169185404, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "user_mentions": {{ 169185404 }}, "user": { "id": 1961629416, "name": "Ismail Can Yigit", "screen_name": "ICYigit", "lang": "en", "location": "Ann Arbor, MI", "create_at": date("2013-10-14"), "description": "Istanbul & New Brunswick & San Diego & Ann Arbor // Besiktas & Sport & Music & Nature // University of Michigan Ann Arbor - Aerospace Engineering", "followers_count": 73, "friends_count": 81, "statues_count": 8376 }, "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-25T10:00:31.000Z"), "id": 669576410881777665, "text": "Lost in the sauce fam https://t.co/S04pBzZ5RY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2313298324, "name": "12|10 ♐️", "screen_name": "1youngpharaoh", "lang": "en", "location": "Out my mind•803 ✈️ 864", "create_at": date("2014-01-29"), "description": "It feel good waking up to money in the bank #BWG•#TheField 〽️•#TrillLegacy", "followers_count": 1271, "friends_count": 1054, "statues_count": 16888 }, "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("2015-11-25T10:00:31.000Z"), "id": 669576411150028800, "text": "@TalkinCowboys - the 2 thanksgiving games that stand out 2 me both involve backup qbs. Garrett & Longly. Greats games!", "in_reply_to_status": -1, "in_reply_to_user": 2595387888, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2595387888 }}, "user": { "id": 1872351318, "name": "Brent", "screen_name": "BFite22", "lang": "en", "location": "null", "create_at": date("2013-09-16"), "description": "null", "followers_count": 52, "friends_count": 69, "statues_count": 46 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ingleside, TX", "id": "00749ed44f1ee829", "name": "Ingleside", "place_type": "city", "bounding_box": rectangle("-97.242995,27.836632 -97.170111,27.892585") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48409, "countyName": "San Patricio", "cityID": 4836008, "cityName": "Ingleside" } }
+{ "create_at": datetime("2015-11-25T10:00:31.000Z"), "id": 669576411250737152, "text": "@Tveez_World you ain't ���� \n\nIn a good way tho lol", "in_reply_to_status": 669576304790892544, "in_reply_to_user": 1265092778, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1265092778 }}, "user": { "id": 368735686, "name": "Chris", "screen_name": "TheLakeShOwKinG", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-09-05"), "description": "subscribe to my podcast http://ipocketcast.com/pwnav/podcast.rss I love the 49ers, Lakers, SF Giants, USC, Black & Latina Queens", "followers_count": 4764, "friends_count": 3540, "statues_count": 231192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-25T10:00:31.000Z"), "id": 669576411619991552, "text": "bitch go to hell. that shit not fucking funny https://t.co/E2YJG89wqk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 963408168, "name": "11.30 4️⃣ my cammy❤️", "screen_name": "queenxgg", "lang": "en", "location": "✨chicaglo✨", "create_at": date("2012-11-21"), "description": "finesse god❤️❤️❤️", "followers_count": 382, "friends_count": 196, "statues_count": 9259 }, "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-25T10:00:31.000Z"), "id": 669576411867410433, "text": "Play me at night they won't act right �� https://t.co/75LE3AiYUA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 350397660, "name": "Ge'Kyra", "screen_name": "GeKyraDavis", "lang": "en", "location": "null", "create_at": date("2011-08-07"), "description": "Ge'Kyra T. Davis | 17 | February 2nd | R.I.P John ❤️", "followers_count": 1111, "friends_count": 854, "statues_count": 35077 }, "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("2015-11-25T10:00:31.000Z"), "id": 669576411871444992, "text": "Want to work in #Danvers, MA? View our latest opening: https://t.co/xXohCkydn0 #Sales #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.9300507,42.5750946"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Danvers", "Sales", "Job", "Jobs", "Hiring" }}, "user": { "id": 28720800, "name": "Advantage Careers", "screen_name": "asmcareers", "lang": "en", "location": "North America", "create_at": date("2009-04-03"), "description": "Advantage Sales and Marketing has a career for you! \nCome Join Our Team Today!", "followers_count": 1989, "friends_count": 1250, "statues_count": 29122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Danvers, MA", "id": "898265bec13bd843", "name": "Danvers", "place_type": "city", "bounding_box": rectangle("-71.002897,42.540366 -70.904091,42.612824") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25009, "countyName": "Essex", "cityID": 2516285, "cityName": "Danvers" } }
+{ "create_at": datetime("2015-11-25T10:00:31.000Z"), "id": 669576411997274112, "text": "Woke up at 9:52 and my first class is at 10 bye I can't get up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2398096081, "name": "Gina Murillo", "screen_name": "ginalizete", "lang": "en", "location": "null", "create_at": date("2014-03-19"), "description": "fierce Jesus lover | princesssssa ☻", "followers_count": 311, "friends_count": 365, "statues_count": 16135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-25T10:00:31.000Z"), "id": 669576412001673219, "text": "Hahahaha when none of you \"best friends\" from high school want to hang out with you during break ����������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 410900130, "name": "Maiz ☄", "screen_name": "uhMAIZIng_", "lang": "en", "location": "Nude on the moon ", "create_at": date("2011-11-12"), "description": "And she'll tease you, she'll unease you", "followers_count": 631, "friends_count": 432, "statues_count": 16208 }, "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("2015-11-25T10:00:31.000Z"), "id": 669576412903243776, "text": "Zahra in her wedding finery. skysong8 #bride #bridalportrait #bridalphotography #bridalphotoshoot… https://t.co/dVmzWJJVpa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.97202415,40.66275966"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "bride", "bridalportrait", "bridalphotography", "bridalphotoshoot" }}, "user": { "id": 22120132, "name": "Steven Rosen", "screen_name": "RamblinRosen", "lang": "en", "location": "Brooklyn Heights", "create_at": date("2009-02-27"), "description": "Award-winning wedding and portrait photographer", "followers_count": 196, "friends_count": 50, "statues_count": 340 }, "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-25T10:00:31.000Z"), "id": 669576413012455424, "text": "This day started off great and it's not gonna 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": 2858047511, "name": "Goldy Locks", "screen_name": "Dakota_Walker_", "lang": "en", "location": "null", "create_at": date("2014-11-02"), "description": "8/22/15❤️", "followers_count": 258, "friends_count": 233, "statues_count": 2435 }, "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("2015-11-25T10:00:31.000Z"), "id": 669576413033406465, "text": "Thank you ! �� https://t.co/x4gmsNMhAP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1599923414, "name": "itsMYbirthday", "screen_name": "Kayylaaaa__", "lang": "en", "location": "New Orleans, LA", "create_at": date("2013-07-16"), "description": "Jayni❤️. Nigerian-American theWinningTeam", "followers_count": 1462, "friends_count": 939, "statues_count": 23385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Terrytown, LA", "id": "9c97662afdd448aa", "name": "Terrytown", "place_type": "city", "bounding_box": rectangle("-90.043765,29.878501 -90.008313,29.92917") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2275180, "cityName": "Terrytown" } }
+{ "create_at": datetime("2015-11-25T10:00:31.000Z"), "id": 669576413075271681, "text": "It's not fair when guys ask you out in front of lots of people cause it's putting a ton of pressure on u & if you say no u look like 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": 509609848, "name": "B❁", "screen_name": "bianca_mac75", "lang": "en", "location": "null", "create_at": date("2012-02-29"), "description": "day dreamer☀️, night thinker TX➡️AZ", "followers_count": 790, "friends_count": 460, "statues_count": 30187 }, "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("2015-11-25T10:00:31.000Z"), "id": 669576413113028609, "text": "@mike_6ix what gags", "in_reply_to_status": 669576317218590724, "in_reply_to_user": 197922812, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 197922812 }}, "user": { "id": 1727304210, "name": "Dylan", "screen_name": "DylanxBeats", "lang": "en", "location": "null", "create_at": date("2013-09-03"), "description": "null", "followers_count": 426, "friends_count": 415, "statues_count": 18995 }, "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("2015-11-25T10:00:31.000Z"), "id": 669576413125611520, "text": "#TeamCap https://t.co/0OBlRH0SNd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "TeamCap" }}, "user": { "id": 3268657274, "name": "Middle Class Batman", "screen_name": "MidClassBatman", "lang": "en", "location": "The BatManor", "create_at": date("2015-07-04"), "description": "I left my fortune and crime fighting. Got married. Moved to the suburbs, had some kids (SIX KIDS YO), and hang out on Periscope. I'm Middle Class Batman.", "followers_count": 177, "friends_count": 264, "statues_count": 540 }, "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-25T10:00:31.000Z"), "id": 669576413339648001, "text": "@TheRealCamG215 oh my god this is great", "in_reply_to_status": 669550292753637376, "in_reply_to_user": 400888223, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 400888223 }}, "user": { "id": 105790457, "name": "Marty McFly", "screen_name": "madznotsadz", "lang": "en", "location": "nyc // dc // ie", "create_at": date("2010-01-17"), "description": "•a ginger with a soul and a lil bit of rock n'roll•", "followers_count": 470, "friends_count": 976, "statues_count": 14454 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waltham, MA", "id": "0633ab4aa27b0bf7", "name": "Waltham", "place_type": "city", "bounding_box": rectangle("-71.289902,42.352885 -71.194241,42.436021") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2572600, "cityName": "Waltham" } }
+{ "create_at": datetime("2015-11-25T10:00:31.000Z"), "id": 669576413398216704, "text": "The kids keep 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": 30996977, "name": "alyss", "screen_name": "alyssamagro", "lang": "en", "location": "♡♡♡♡♡", "create_at": date("2009-04-13"), "description": "If I ever let my head down, it will be just to admire my shoes.", "followers_count": 578, "friends_count": 760, "statues_count": 15510 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Saugus, MA", "id": "48ef19debe0f30ca", "name": "Saugus", "place_type": "city", "bounding_box": rectangle("-71.053569,42.429752 -70.972136,42.505195") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25009, "countyName": "Essex", "cityID": 2560050, "cityName": "Saugus" } }
+{ "create_at": datetime("2015-11-25T10:00:31.000Z"), "id": 669576413763248128, "text": "don't know what to eat for lunch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 342207362, "name": "tianaliz", "screen_name": "tianaliz_15", "lang": "en", "location": "null", "create_at": date("2011-07-25"), "description": "18.", "followers_count": 629, "friends_count": 284, "statues_count": 22342 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irondequoit, NY", "id": "31bfc1f1d5544e1b", "name": "Irondequoit", "place_type": "city", "bounding_box": rectangle("-77.623126,43.166743 -77.521784,43.256791") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3637737, "cityName": "Irondequoit" } }
+{ "create_at": datetime("2015-11-25T10:00:31.000Z"), "id": 669576413784051712, "text": "Experienced RN Hiring Event - Miami Children's Health System: (#Miami, FL) https://t.co/CMn2q9DF0Z #Nursing #Nurse #Florida #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.2940652,25.7414669"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Miami", "Nursing", "Nurse", "Florida", "Job", "Jobs" }}, "user": { "id": 42622658, "name": "Miami Childrens Jobs", "screen_name": "MCHCareers", "lang": "en", "location": "Miami, Florida", "create_at": date("2009-05-26"), "description": "Nicklaus Children’s Hospital, formerly Miami Children's Hospital, is S.Fla’s only licensed pediatric specialty hospital.", "followers_count": 1071, "friends_count": 97, "statues_count": 826 }, "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("2015-11-25T10:00:31.000Z"), "id": 669576414237192192, "text": "@joespiel holy God, means I'm even more lucky than I thought! Yay!", "in_reply_to_status": 669562024976850944, "in_reply_to_user": 45070715, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 45070715 }}, "user": { "id": 65897576, "name": "Mariya Biktimirova", "screen_name": "Manka__Kasha", "lang": "en", "location": "Russia", "create_at": date("2009-08-15"), "description": "-be serious\n-I'm wild", "followers_count": 55, "friends_count": 258, "statues_count": 1396 }, "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("2015-11-25T10:00:32.000Z"), "id": 669576414493020160, "text": "I ❤️ Hallmark", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 463283214, "name": "Kaitlyn Matheson", "screen_name": "kaitlyn0329", "lang": "en", "location": "Johnson City, TN", "create_at": date("2012-01-13"), "description": "ETSU ~ Alpha Delta Pi", "followers_count": 1325, "friends_count": 1160, "statues_count": 6431 }, "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.259078 -82.300523,36.430258") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47179, "countyName": "Washington", "cityID": 4738320, "cityName": "Johnson City" } }
+{ "create_at": datetime("2015-11-25T10:00:32.000Z"), "id": 669576414576816128, "text": "#Pleasanton, CA #BusinessMgmt #Job: Consulting Project Technical Director at Oracle https://t.co/vRpoCYRqTd #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.8746789,37.6624312"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Pleasanton", "BusinessMgmt", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 24315640, "name": "TMJ-SJC Mgmt. Jobs", "screen_name": "tmj_sjc_mgmt", "lang": "en", "location": "San Jose, CA", "create_at": date("2009-03-13"), "description": "Follow this account for geo-targeted Business/Mgmt. job tweets in San Jose, CA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 392, "friends_count": 256, "statues_count": 333 }, "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("2015-11-25T10:00:32.000Z"), "id": 669576414757171201, "text": "@Mwerlinger11 undefeated With Romo at QB ��", "in_reply_to_status": 669567348580323328, "in_reply_to_user": 47186417, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 47186417 }}, "user": { "id": 479150546, "name": "Average Joe", "screen_name": "Ropie30", "lang": "en", "location": "Parts Unknown", "create_at": date("2012-01-30"), "description": "Ea Sports Game Changer and Youtuber. \nFor Madden ultimate team Tips tricks and Videos Visit me at http://Youtube.com/user/BigRopie", "followers_count": 5504, "friends_count": 203, "statues_count": 29182 }, "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("2015-11-25T10:00:32.000Z"), "id": 669576414778249216, "text": "The fact we got a cactus lamp in our living room ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 794218160, "name": "Alisa Victoria ❤️", "screen_name": "alisa_newsom", "lang": "en", "location": "Burton, MI", "create_at": date("2012-08-31"), "description": "| volleyball | basketball | AHS | You aint shit (Quetta) | 18 | SEN16R ⚫️⚪️ | love my bff Allie | Quetta is my best friend 11.5.14 |", "followers_count": 517, "friends_count": 312, "statues_count": 7292 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burton, MI", "id": "d4f8c782da0a3b7b", "name": "Burton", "place_type": "city", "bounding_box": rectangle("-83.692863,42.959625 -83.573753,43.048152") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2612060, "cityName": "Burton" } }
+{ "create_at": datetime("2015-11-25T10:00:32.000Z"), "id": 669576414920753152, "text": "Can't eat on thanksgiving", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2538167940, "name": "K-Jones", "screen_name": "krjonesss", "lang": "en", "location": "null", "create_at": date("2014-05-31"), "description": "null", "followers_count": 294, "friends_count": 116, "statues_count": 857 }, "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("2015-11-25T10:00:32.000Z"), "id": 669576415046582272, "text": "@pweezzzy alright in gonna leave me class rn", "in_reply_to_status": 669576128567209984, "in_reply_to_user": 2917084084, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2917084084 }}, "user": { "id": 74886254, "name": "Kelly.", "screen_name": "XO_KFF", "lang": "en", "location": "Califournia", "create_at": date("2009-09-16"), "description": "Hi I'm Kelly and I've met Abel", "followers_count": 629, "friends_count": 599, "statues_count": 37121 }, "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("2015-11-25T10:00:32.000Z"), "id": 669576415168319488, "text": "Dead @ my last tweet", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 472517940, "name": "sinwala is basic", "screen_name": "thesee_dimples", "lang": "en", "location": "null", "create_at": date("2012-01-23"), "description": "uh•lawn•is", "followers_count": 498, "friends_count": 476, "statues_count": 7681 }, "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("2015-11-25T10:00:32.000Z"), "id": 669576415210287104, "text": "@Cbreunig14 I know, I took the little bro inside with Julie and came back out.", "in_reply_to_status": 669576164567044097, "in_reply_to_user": 430142194, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 430142194 }}, "user": { "id": 1902571070, "name": "mads", "screen_name": "madsmcdonald15", "lang": "en", "location": "null", "create_at": date("2013-09-24"), "description": "banned from h&m for life and I paint a little", "followers_count": 692, "friends_count": 925, "statues_count": 7508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, OH", "id": "00b05917162d2084", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-84.412341,39.369307 -84.352449,39.422516") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39091, "countyName": "Logan", "cityID": 3936988, "cityName": "Huntsville" } }
+{ "create_at": datetime("2015-11-25T10:00:32.000Z"), "id": 669576415399043072, "text": "Throwback Wednesday with this jam. https://t.co/CooGFqdWlI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2232215989, "name": "Erin Richter", "screen_name": "erincrichter", "lang": "en", "location": "Chicago, IL", "create_at": date("2013-12-05"), "description": "Studying music business and PR at @columbiachi. Consumer of coffee and pop culture. Probably too witty for my own good.", "followers_count": 226, "friends_count": 312, "statues_count": 7040 }, "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-25T10:00:32.000Z"), "id": 669576415474421760, "text": "@whatwasithnkn32 @ESPNCFB that's as real as it get", "in_reply_to_status": 669576031859142661, "in_reply_to_user": 2454182388, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2454182388, 57100687 }}, "user": { "id": 518220308, "name": "KCW", "screen_name": "ClayGotSushi", "lang": "en", "location": "Salina, KS", "create_at": date("2012-03-07"), "description": "Okc ✈️ Kansas | College Athlete | KWU", "followers_count": 1701, "friends_count": 451, "statues_count": 84480 }, "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("2015-11-25T10:00:32.000Z"), "id": 669576415507910656, "text": "@WoodyBruh https://t.co/MvJImoMadD", "in_reply_to_status": -1, "in_reply_to_user": 2774203055, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2774203055 }}, "user": { "id": 790367372, "name": "ili", "screen_name": "ilianazfalcon", "lang": "en", "location": "null", "create_at": date("2012-08-29"), "description": "null", "followers_count": 877, "friends_count": 477, "statues_count": 2682 }, "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-25T10:00:32.000Z"), "id": 669576415805898758, "text": "@MsNaChan Trying to order? Learn how to enable Tweet Ordering in your Pizza Profile: https://t.co/zUmEcYM652 [13:00:40 EST]", "in_reply_to_status": 669576360294125568, "in_reply_to_user": 300431976, "favorite_count": 0, "coordinate": point("-83.6828352,42.318795"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 300431976 }}, "user": { "id": 31444922, "name": "Domino's Pizza", "screen_name": "dominos", "lang": "en", "location": "Ann Arbor, MI", "create_at": date("2009-04-15"), "description": "The home of Domino's Pizza on Twitter. Hosted by the crew at Domino's HQ.", "followers_count": 943721, "friends_count": 45411, "statues_count": 202851 }, "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-25T10:00:32.000Z"), "id": 669576415897976832, "text": "I double text, triple text, 27th text.. Ya know. No shame.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 440459464, "name": "Kalea Padilla", "screen_name": "TheycallmeKale", "lang": "en", "location": "null", "create_at": date("2011-12-18"), "description": "I look like I'm 12", "followers_count": 1335, "friends_count": 715, "statues_count": 37109 }, "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("2015-11-25T10:00:32.000Z"), "id": 669576416053202944, "text": "@askanyone Simple retweet her home��@rosariodawson @traceyeedmonds @ashanti Happy Thanksgiving @jlo #DrinkUpAshanti https://t.co/CqjffgMaFe", "in_reply_to_status": 669575473995870209, "in_reply_to_user": 27699224, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DrinkUpAshanti" }}, "user_mentions": {{ 27699224, 82939583, 256242377, 67982898, 85603854 }}, "user": { "id": 841421474, "name": "ErickCribb", "screen_name": "ErickscCribb", "lang": "en", "location": "Charleston, South Carolina", "create_at": date("2012-09-23"), "description": "If u see this page it means u have hit the point of no return and must #follow to turn back - Leave your mark here", "followers_count": 919, "friends_count": 0, "statues_count": 5366 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charleston, SC", "id": "7422f29331ca8257", "name": "Charleston", "place_type": "city", "bounding_box": rectangle("-80.134867,32.673231 -79.879867,32.866916") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45019, "countyName": "Charleston", "cityID": 4513330, "cityName": "Charleston" } }
+{ "create_at": datetime("2015-11-25T10:00:32.000Z"), "id": 669576416384696320, "text": "exactly https://t.co/1g5GuTr9xc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 544887888, "name": "Nicki Carson", "screen_name": "Nicolettalynn", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2012-04-03"), "description": "don't hide. live. follow the sun. you'll make it to tomorrow.", "followers_count": 926, "friends_count": 593, "statues_count": 28482 }, "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-25T10:00:32.000Z"), "id": 669576416493633537, "text": "Oh no @itsannahuff https://t.co/uo46LQcw0R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 308867792 }}, "user": { "id": 114358707, "name": "Maddie Huffman", "screen_name": "maddiehuffman77", "lang": "en", "location": "KC ✈️ Phoenix ☀️", "create_at": date("2010-02-14"), "description": "null", "followers_count": 473, "friends_count": 401, "statues_count": 4534 }, "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("2015-11-25T10:00:32.000Z"), "id": 669576416586002432, "text": "@DanPultzMGMT I know, but he was underthrowing before he got hurt too.", "in_reply_to_status": 669576143822036993, "in_reply_to_user": 28203167, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28203167 }}, "user": { "id": 603332749, "name": "Mushin", "screen_name": "MetalMushin", "lang": "en", "location": "Orange, CT", "create_at": date("2012-06-08"), "description": "#Broncos, #Yankees, #Metalhead, #Guitarist, #Rocker, #Jedi, Professional Wise-Ass. May the Force be with you. You can call me Mushin. The .Gif King.", "followers_count": 2041, "friends_count": 2009, "statues_count": 142967 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CT", "id": "7b62343e287fb7c0", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-73.085611,41.245045 -72.979996,41.31407") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 957670, "cityName": "Orange" } }
+{ "create_at": datetime("2015-11-25T10:00:32.000Z"), "id": 669576416678182912, "text": "@eluxari_ @DVS_Malkarii @isT_Tzeriel @bakedleech6 HAHAHAHAHA.. yeah. I know that feeling", "in_reply_to_status": 669575924283801600, "in_reply_to_user": 2886827726, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2886827726, 37568327, 52711285, 69362555 }}, "user": { "id": 270728196, "name": "Screwy of Lewse", "screen_name": "screwlewseWoW", "lang": "en", "location": "Bay Area, CA", "create_at": date("2011-03-22"), "description": "your adorable creeper at your service. innapropriate 90% of the time.", "followers_count": 697, "friends_count": 342, "statues_count": 50402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downtown North, Palo Alto", "id": "50d5ad4407f8e5aa", "name": "Downtown North", "place_type": "neighborhood", "bounding_box": rectangle("-122.17038,37.443156 -122.156808,37.453941") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 655282, "cityName": "Palo Alto" } }
+{ "create_at": datetime("2015-11-25T10:00:32.000Z"), "id": 669576416938192896, "text": "joce", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1028049954, "name": "wilhelm", "screen_name": "15wilallen15", "lang": "en", "location": "nola", "create_at": date("2012-12-22"), "description": "15⚾️ with no love", "followers_count": 297, "friends_count": 195, "statues_count": 19174 }, "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("2015-11-25T10:00:32.000Z"), "id": 669576416938201088, "text": "@weedtuna_ November baby's rule the world", "in_reply_to_status": 669576048900571137, "in_reply_to_user": 333018201, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 333018201 }}, "user": { "id": 421526619, "name": "henney hardaway", "screen_name": "duttysamg", "lang": "en", "location": "glomaha ", "create_at": date("2011-11-25"), "description": "yeezy taught me", "followers_count": 780, "friends_count": 540, "statues_count": 42078 }, "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("2015-11-25T10:00:32.000Z"), "id": 669576417017921537, "text": "me when my family asks me how my grades are going tomorrow https://t.co/yK16HkCJJM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1687358694, "name": "addie", "screen_name": "addison_angelo", "lang": "en", "location": "az", "create_at": date("2013-08-20"), "description": "you may all go to hell, and I will go to texas", "followers_count": 840, "friends_count": 716, "statues_count": 2603 }, "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-25T10:00:32.000Z"), "id": 669576417139527681, "text": "Border Wars on @NatGeo is my life right now #borderwars #borderpatrol #searchandfind #USMexico", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "borderwars", "borderpatrol", "searchandfind", "USMexico" }}, "user_mentions": {{ 17471979 }}, "user": { "id": 633188388, "name": "Sarah Binder", "screen_name": "SarahBinder22", "lang": "en", "location": "null", "create_at": date("2012-07-11"), "description": "Lindenwood student working towards a degree in Health Management, Cardinals fan, and self proclaimed top chef.", "followers_count": 32, "friends_count": 305, "statues_count": 190 }, "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("2015-11-25T10:00:32.000Z"), "id": 669576417244352514, "text": "One day, I'll go to Italy! ���� https://t.co/6yRTPNAmNK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 370379927, "name": "KyleRuerat", "screen_name": "KyleRuerat", "lang": "en", "location": "null", "create_at": date("2011-09-08"), "description": "Coffee, Skateboarding, and Literature. ☕️", "followers_count": 192, "friends_count": 149, "statues_count": 2430 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, MI", "id": "8cbcad86e88fd71a", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-84.006198,42.568064 -83.909455,42.651203") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26093, "countyName": "Livingston", "cityID": 2639540, "cityName": "Howell" } }
+{ "create_at": datetime("2015-11-25T10:00:32.000Z"), "id": 669576417479229440, "text": "See our latest #Monterey, CA #job and click to apply: Infection Prevention Coord-Quality Mgmt - https://t.co/JZDykDEkr0 #ORjobs #Nursing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.9131935,36.5787155"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Monterey", "job", "ORjobs", "Nursing" }}, "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": 52, "friends_count": 27, "statues_count": 49 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monterey, CA", "id": "49af5b43d4963f4c", "name": "Monterey", "place_type": "city", "bounding_box": rectangle("-121.927024,36.573741 -121.830974,36.618826") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 648872, "cityName": "Monterey" } }
+{ "create_at": datetime("2015-11-25T10:00:32.000Z"), "id": 669576417533779968, "text": "We're #hiring! Read about our latest #job opening here: PICU RN/Travel RN Nights in Wilmington, NC - https://t.co/UdPNevXPkl #Nursing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.9852853,34.3023653"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Nursing" }}, "user": { "id": 23306728, "name": "TMJ-WIL Nursing Jobs", "screen_name": "tmj_wil_nursing", "lang": "en", "location": "Wilmington, NC", "create_at": date("2009-03-08"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Wilmington, NC from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 334, "friends_count": 302, "statues_count": 14 }, "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.32187,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37019, "countyName": "Brunswick" } }
+{ "create_at": datetime("2015-11-25T10:00:32.000Z"), "id": 669576417533894656, "text": "�� https://t.co/Rdw0yEk5Zb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 173588816, "name": "Joc Tarantino", "screen_name": "BigJocATL", "lang": "en", "location": "Behind The Scenes", "create_at": date("2010-08-01"), "description": "Visionary • Curator For @CADArtShow CreativesAfterDark@gmail.com", "followers_count": 3411, "friends_count": 2345, "statues_count": 102905 }, "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("2015-11-25T10:00:32.000Z"), "id": 669576417592672256, "text": "'Play Safe' by Scott T. Kemery\nhttps://t.co/Z1fVCCM9iT\nPaul Morris\nTim Harmon\nTreasure Island Media https://t.co/orM1gUIFy9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2885606546, "name": "Scott T. Kemery", "screen_name": "ScottTKemery", "lang": "en", "location": "Fort Lauderdale, FL", "create_at": date("2014-10-31"), "description": "Writer", "followers_count": 17063, "friends_count": 18121, "statues_count": 52075 }, "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("2015-11-25T10:00:32.000Z"), "id": 669576417663913984, "text": "I wish I could make a group chat with everyone in the world named \"Brett Annelin.\" I would be pretty lonely talking to myself though.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 52131255, "name": "is Brett", "screen_name": "Brett_Annelin", "lang": "en", "location": "Twitter", "create_at": date("2009-06-29"), "description": "Guitarist for Hot Mulligan, Student at Michigan Tech, and a hell of a sequence player. Manistique, Michigan.", "followers_count": 348, "friends_count": 214, "statues_count": 3347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manistique, MI", "id": "10742da23c7a78b9", "name": "Manistique", "place_type": "city", "bounding_box": rectangle("-86.278937,45.942148 -86.226448,45.97978") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26153, "countyName": "Schoolcraft", "cityID": 2650760, "cityName": "Manistique" } }
+{ "create_at": datetime("2015-11-25T10:00:32.000Z"), "id": 669576417877712897, "text": "Suddenly it's 1pm and I'm laying in bed eating pizza flavored gold fish in my underwear questioning my stability. It's fine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1270330746, "name": "JoJo", "screen_name": "jojobee913", "lang": "en", "location": "null", "create_at": date("2013-03-15"), "description": "I have sits ❁", "followers_count": 356, "friends_count": 296, "statues_count": 14009 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corning, NY", "id": "dea740edc22e507b", "name": "Corning", "place_type": "city", "bounding_box": rectangle("-77.077935,42.130906 -77.025832,42.177372") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36101, "countyName": "Steuben", "cityID": 3618256, "cityName": "Corning" } }
+{ "create_at": datetime("2015-11-25T10:00:32.000Z"), "id": 669576418045665281, "text": "Guys omg I'm crying������ He sings so beautifully wtf Yoandri���� @YoandriCabrera_ \nhttps://t.co/dMJcsFLKSO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4095265457 }}, "user": { "id": 2716633245, "name": "Sammi Núñez✨", "screen_name": "ObsessedWithGRR", "lang": "en", "location": "PLANETROYCE2008•XO•SLADOOMY❤", "create_at": date("2014-07-18"), "description": "MIAMI•Royce RTed 7/6/15&7/8/15•|•Daddy Yankee noticed 8/18/15•|•DeLaGeezy s/o on Periscope 8/25/15•|•Nath quoted 11/14/15", "followers_count": 1700, "friends_count": 2245, "statues_count": 57346 }, "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("2015-11-25T10:00:32.000Z"), "id": 669576418200678400, "text": "Join the Jared team! See our latest #SkilledTrade #job opening here: https://t.co/lgeZDG61E8 #Jeweler #CrestviewHills, KY #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.5849438,39.0272829"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SkilledTrade", "job", "Jeweler", "CrestviewHills", "Hiring" }}, "user": { "id": 3248848340, "name": "Jobs at Jared", "screen_name": "JobsAtJared", "lang": "en", "location": "null", "create_at": date("2015-06-18"), "description": "Your Future Starts Here.", "followers_count": 12, "friends_count": 1, "statues_count": 503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crestview Hills, KY", "id": "5886a2113ee1def0", "name": "Crestview Hills", "place_type": "city", "bounding_box": rectangle("-84.590971,39.012275 -84.538381,39.043731") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21117, "countyName": "Kenton", "cityID": 2118442, "cityName": "Crestview Hills" } }
+{ "create_at": datetime("2015-11-25T10:00:32.000Z"), "id": 669576418204909568, "text": "tomorrow will be great�� thanksgiving dinner at the house, then off to anacoco for another ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2325492186, "name": "brittany❤️", "screen_name": "bclg_13", "lang": "en", "location": "null", "create_at": date("2014-02-03"), "description": "@that_girldevin is my boyfriend", "followers_count": 530, "friends_count": 626, "statues_count": 7488 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rosepine, LA", "id": "4aae535e3c33dfca", "name": "Rosepine", "place_type": "city", "bounding_box": rectangle("-93.306131,30.887942 -93.26809,30.963614") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22115, "countyName": "Vernon", "cityID": 2266200, "cityName": "Rosepine" } }
+{ "create_at": datetime("2015-11-25T10:00:32.000Z"), "id": 669576418297270272, "text": "@CrowderKylie happy birthday!!", "in_reply_to_status": -1, "in_reply_to_user": 1107982424, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1107982424 }}, "user": { "id": 2863854163, "name": "dirty dan", "screen_name": "DanielHribal", "lang": "en", "location": "null", "create_at": date("2014-10-18"), "description": "snap chat: Daniel.hairball living the mcdons life", "followers_count": 262, "friends_count": 490, "statues_count": 661 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rockton, IL", "id": "749015dab52e7830", "name": "Rockton", "place_type": "city", "bounding_box": rectangle("-89.089475,42.412296 -89.018477,42.473732") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17201, "countyName": "Winnebago", "cityID": 1765156, "cityName": "Rockton" } }
+{ "create_at": datetime("2015-11-25T10:00:32.000Z"), "id": 669576418372661249, "text": "@BoxerClev @YodeR_9 are you sending or not? We can play for $7.50 but if you send $15 we will accept yall suck", "in_reply_to_status": 669575542618746885, "in_reply_to_user": 3214070099, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1289787234, 617857327 }}, "user": { "id": 3214070099, "name": "Joseph", "screen_name": "Ayyy_Joseph", "lang": "en", "location": "null", "create_at": date("2015-04-27"), "description": "Gold 53 Trophies 6k earned", "followers_count": 233, "friends_count": 117, "statues_count": 8441 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mechanicsville, VA", "id": "91923d1d1be8347e", "name": "Mechanicsville", "place_type": "city", "bounding_box": rectangle("-77.421421,37.576879 -77.293309,37.669732") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51085, "countyName": "Hanover", "cityID": 5150856, "cityName": "Mechanicsville" } }
+{ "create_at": datetime("2015-11-25T10:00:32.000Z"), "id": 669576418485866496, "text": "Can you recommend anyone for this #Retail #job? https://t.co/z0zib4O1Qr #Greenville, AL #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.6396888,31.8542187"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "Greenville", "Hiring", "CareerArc" }}, "user": { "id": 57709710, "name": "TMJ-AL Retail Jobs", "screen_name": "tmj_al_retail", "lang": "en", "location": "Alabama", "create_at": date("2009-07-17"), "description": "Follow this account for geo-targeted Retail job tweets in Alabama. Need help? Tweet us at @CareerArc!", "followers_count": 360, "friends_count": 302, "statues_count": 789 }, "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": 1013, "countyName": "Butler", "cityID": 131912, "cityName": "Greenville" } }
+{ "create_at": datetime("2015-11-25T10:00:32.000Z"), "id": 669576418536370178, "text": "Yesss ������ https://t.co/CZ7LByPtB4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2405936575, "name": "C h y n a r o s e .", "screen_name": "Cee_dollrose", "lang": "en", "location": "C a l i fuckin f o r n i a", "create_at": date("2014-03-22"), "description": "Sorry im a lil childish . \nSc : Cee_macckin . Rt my favs ❤ .", "followers_count": 1813, "friends_count": 1783, "statues_count": 2470 }, "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("2015-11-25T10:00:33.000Z"), "id": 669576418951475201, "text": "@LoLHaruka @LoLDyrus @leena_xu i fry myself if anyone touches MY #tsm #tsmetro @TSMDoublelift i luv ur colors, i died black.. inside", "in_reply_to_status": 669576107801251840, "in_reply_to_user": 4303166900, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "tsm", "tsmetro" }}, "user_mentions": {{ 1040830518, 144701884, 632498857, 344538810 }}, "user": { "id": 4303166900, "name": "O. jacob @ home", "screen_name": "cullen121878", "lang": "en", "location": "cerritos, ca", "create_at": date("2015-11-20"), "description": "#1fan that fuels the flame, i am PASSION of christ #999 im #korean #imported IM ME, I DO ME, AND I CHILL DREAM BIG #KOBE", "followers_count": 244, "friends_count": 866, "statues_count": 2204 }, "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("2015-11-25T10:00:33.000Z"), "id": 669576418955821056, "text": "The constant back and forth with how you feel ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3054609003, "name": "Agatha Macko", "screen_name": "Princess_Macko", "lang": "en", "location": "Worcester, MA", "create_at": date("2015-02-22"), "description": "Burncoat Dance Team | #RestEasyGary | #RestInPeaceChristian | #RestInPeaceBabcia", "followers_count": 757, "friends_count": 699, "statues_count": 2165 }, "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("2015-11-25T10:00:33.000Z"), "id": 669576418997592065, "text": "Got my hair done btw https://t.co/15Ye92TPN2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2568005403, "name": "kiersten green", "screen_name": "kierstengreen9", "lang": "en", "location": "null", "create_at": date("2014-05-27"), "description": "last name? soon to be gerhardt", "followers_count": 262, "friends_count": 350, "statues_count": 2572 }, "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("2015-11-25T10:00:33.000Z"), "id": 669576419467489281, "text": "@1future https://t.co/O2mumw74Rw", "in_reply_to_status": -1, "in_reply_to_user": 51742969, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 51742969 }}, "user": { "id": 632932463, "name": "Madyson", "screen_name": "mad_dawgg20", "lang": "en", "location": "12 tag & PC", "create_at": date("2012-07-11"), "description": "Christian Messer ❤", "followers_count": 954, "friends_count": 888, "statues_count": 10454 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valley, AL", "id": "dd5d995aeda1ccf8", "name": "Valley", "place_type": "city", "bounding_box": rectangle("-85.224667,32.752675 -85.140283,32.856282") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1017, "countyName": "Chambers", "cityID": 178204, "cityName": "Valley" } }
+{ "create_at": datetime("2015-11-25T10:00:33.000Z"), "id": 669576419475787778, "text": "then she hard up https://t.co/76Vpr0xGxT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2428211874, "name": "idccc☺️", "screen_name": "kaenolaa_", "lang": "en", "location": "w/tiwauna,jontrell,jamaria❤️..", "create_at": date("2014-04-04"), "description": "@_bree109 sis!☺️ ilovemybesties❤️. restronald❤️ restnate❤️", "followers_count": 2183, "friends_count": 1703, "statues_count": 43955 }, "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("2015-11-25T10:00:33.000Z"), "id": 669576419614175233, "text": "Cardio and shoulders at bandera gym if anyone'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": 3060414355, "name": "N", "screen_name": "nerinnaaa", "lang": "en", "location": "San Antonio, TX", "create_at": date("2015-03-03"), "description": "probably eating or at the gym.. or both", "followers_count": 1187, "friends_count": 663, "statues_count": 8560 }, "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-25T10:00:33.000Z"), "id": 669576419710648321, "text": "Interested in a #ProductMgmt #job near #Alpharetta, GA? This could be a great fit: https://t.co/F8BLk1sgIH #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.2940899,34.0753762"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ProductMgmt", "job", "Alpharetta", "Hiring", "CareerArc" }}, "user": { "id": 88838027, "name": "TMJ-ATL PM Jobs", "screen_name": "tmj_atl_prod", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-11-09"), "description": "Follow this account for geo-targeted Product Management job tweets in Atlanta, GA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 160, "friends_count": 112, "statues_count": 41 }, "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("2015-11-25T10:00:33.000Z"), "id": 669576419974909952, "text": "Play Fallout or write Fallout fan fiction? CAN'T DECIDE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1518281, "name": "Brian Moon", "screen_name": "lunarobverse", "lang": "en", "location": "Sellwood, OR", "create_at": date("2007-03-19"), "description": "A little to the left of Bernie Sanders. Freelance writer. Native Portlander.", "followers_count": 935, "friends_count": 1439, "statues_count": 50766 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sellwood Moreland, Portland", "id": "16261d425072636b", "name": "Sellwood Moreland", "place_type": "neighborhood", "bounding_box": rectangle("-122.668238,45.457787 -122.636531,45.501491") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2015-11-25T10:00:33.000Z"), "id": 669576420025221120, "text": "������ https://t.co/6TKLipPZ2Z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 83206868, "name": "☕️", "screen_name": "ItsSydneyy", "lang": "en", "location": "LA, CA ", "create_at": date("2009-10-17"), "description": "•one day at a time since 3.25.12 • young and in love• Rest in Paradise, baby brother. I love you Jake. 11.10.97-2.6.15 • Honor Your Truth •", "followers_count": 889, "friends_count": 374, "statues_count": 7118 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-25T10:00:33.000Z"), "id": 669576420218109952, "text": "@YungPapiYaBish @picsforLordA lol", "in_reply_to_status": 669576251279933440, "in_reply_to_user": 1030032793, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1030032793, 2188478460 }}, "user": { "id": 36014941, "name": "Miss Nikki Young", "screen_name": "iamnikkiyoung1", "lang": "en", "location": "KillaforniaUSA", "create_at": date("2009-04-28"), "description": "#TheHipHopPublicistPR - Intern / Red Carpet Host / Writer & Sales Rep for #IndustryInbox - Model INSTA: @domodoespr // @iamnikkiyoung #BlackMafia", "followers_count": 3787, "friends_count": 3585, "statues_count": 116015 }, "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-25T10:00:33.000Z"), "id": 669576420276989952, "text": "I couldn't find a #Thanksgiving #throwback of @sic2treecullen so here is one of me & him at Taylor… https://t.co/GCPvys6bEF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.4607,41.4034"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Thanksgiving", "throwback" }}, "user_mentions": {{ 425017516 }}, "user": { "id": 930851898, "name": "Wendy M.", "screen_name": "WendiPopRock", "lang": "en", "location": "Fairfield County, CT", "create_at": date("2012-11-06"), "description": "News editor by day, fiction writer by night, chick rocker, wife, mom of 3, Triple Neg. Metaplastic breast cancer warrior. https://wendipoprock.wordpress.com/", "followers_count": 794, "friends_count": 1748, "statues_count": 4662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Danbury, CT", "id": "a5b6bdd8008412b1", "name": "Danbury", "place_type": "city", "bounding_box": rectangle("-73.545122,41.352784 -73.400835,41.463455") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 918430, "cityName": "Danbury" } }
+{ "create_at": datetime("2015-11-25T10:00:33.000Z"), "id": 669576420499173376, "text": "I appreciate everything 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": 355854427, "name": "karen lizeth.", "screen_name": "Lizethhxx", "lang": "en", "location": "Oregon, USA", "create_at": date("2011-08-15"), "description": "19 | OR | Vegetarian | I can do everything through Christ, who gives me strength.", "followers_count": 405, "friends_count": 193, "statues_count": 18247 }, "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("2015-11-25T10:00:33.000Z"), "id": 669576421249908738, "text": "#Cranbury, NJ #IT #Job: Platform Architect - Software Engineer URGENT!!! at Robert Half Technology https://t.co/Ybop4UCROP #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.5163889,40.3072222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Cranbury", "IT", "Job", "Jobs", "Hiring" }}, "user": { "id": 21630910, "name": "NJ Software Dev Jobs", "screen_name": "tmj_njc_it", "lang": "en", "location": "Central NJ", "create_at": date("2009-02-22"), "description": "Follow this account for geo-targeted Software Dev. - General/IT job tweets in Central NJ from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 532, "friends_count": 345, "statues_count": 135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cranbury, NJ", "id": "010cea6f5a910a2b", "name": "Cranbury", "place_type": "city", "bounding_box": rectangle("-74.54946,40.288358 -74.476519,40.338744") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3415520, "cityName": "Cranbury" } }
+{ "create_at": datetime("2015-11-25T10:00:33.000Z"), "id": 669576421581295616, "text": "@bardo_rod24 SAME!", "in_reply_to_status": 669574403395420160, "in_reply_to_user": 1335466230, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1335466230 }}, "user": { "id": 706469324, "name": "steph", "screen_name": "stephyysmalls10", "lang": "en", "location": "OC", "create_at": date("2012-07-19"), "description": "uh huh honey", "followers_count": 700, "friends_count": 390, "statues_count": 35195 }, "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("2015-11-25T10:00:33.000Z"), "id": 669576421665280000, "text": "You don't appeal to my pathos", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 635638164, "name": "chezzy jr", "screen_name": "alaina_jonathan", "lang": "en", "location": "Newfane, NY", "create_at": date("2012-07-14"), "description": "null", "followers_count": 645, "friends_count": 403, "statues_count": 14213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newfane, NY", "id": "035517f868f8de84", "name": "Newfane", "place_type": "city", "bounding_box": rectangle("-78.716807,43.252447 -78.687695,43.308758") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36063, "countyName": "Niagara", "cityID": 3650221, "cityName": "Newfane" } }
+{ "create_at": datetime("2015-11-25T10:00:33.000Z"), "id": 669576421698875392, "text": "Cmon now it is 40 sum degrees and ya walking around with Little cut up shirts and shorts for a pep rally?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2442199617, "name": "Sada :)", "screen_name": "sadax0", "lang": "en", "location": "null", "create_at": date("2014-03-28"), "description": "null", "followers_count": 350, "friends_count": 255, "statues_count": 11082 }, "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("2015-11-25T10:00:33.000Z"), "id": 669576421728231428, "text": "@InfoMicDie Aye, thanks for following! I'm a music producer, btw. Feel free to take a listen via my catalog @ https://t.co/wLJiWth2W9", "in_reply_to_status": -1, "in_reply_to_user": 2454684522, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2454684522 }}, "user": { "id": 2694967513, "name": "The B-ST | HQ Beats", "screen_name": "ItsTheBE4ST", "lang": "en", "location": "The Studio, Fam! Where U @?", "create_at": date("2014-07-30"), "description": "#ProudlyMuslim... #HipHop | #Trap | #RnB Music Producer. Email - thebst@outlook.com | My SC - https://soundcloud.com/the_b-st | My Beats ⬇", "followers_count": 2797, "friends_count": 2333, "statues_count": 5915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Exeter, NH", "id": "0a5eca4e60fe6f35", "name": "Exeter", "place_type": "city", "bounding_box": rectangle("-71.001194,42.956811 -70.887799,42.997041") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33015, "countyName": "Rockingham", "cityID": 3325300, "cityName": "Exeter" } }
+{ "create_at": datetime("2015-11-25T10:00:33.000Z"), "id": 669576421761638401, "text": "Can you recommend anyone for this #job? Registered Nurse | RN | OR (Contract) - https://t.co/UWUvPJWJkI #Springfield, MA #Nursing #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.589811,42.1014831"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Springfield", "Nursing", "Hiring" }}, "user": { "id": 36282465, "name": "TMJ-MAW Nursing Jobs", "screen_name": "tmj_maw_nursing", "lang": "en", "location": "Western Area, MA", "create_at": date("2009-04-28"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Western Area, MA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 372, "friends_count": 306, "statues_count": 21 }, "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("2015-11-25T10:00:33.000Z"), "id": 669576421770002432, "text": "@rajkumari_oni @bluemoon_255 @catastrophe_II @adara_art @Vasudha_97 @PaNiHamesha @MishLoveMaNan @pragyaS1192 @sanjanadrealst1 wowwwww suparb", "in_reply_to_status": 669547131104075778, "in_reply_to_user": 3820353018, "favorite_count": 0, "coordinate": point("-118.3430229,34.1843881"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3820353018, 3424632286, 3281317098, 3320589978, 53026001, 3265947072, 1706774149, 373212649, 3148149931 }}, "user": { "id": 2434942777, "name": "Bubly _ hossain", "screen_name": "reefathh", "lang": "en", "location": "California", "create_at": date("2014-04-08"), "description": "MANAN PaNi", "followers_count": 27, "friends_count": 164, "statues_count": 367 }, "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("2015-11-25T10:00:33.000Z"), "id": 669576421912678400, "text": "@rebelliousrocky ohok yeah yeah...", "in_reply_to_status": 669576219340369920, "in_reply_to_user": 2847931842, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2847931842 }}, "user": { "id": 2318178758, "name": "rebecca", "screen_name": "philtyRIchgrl", "lang": "en", "location": "null", "create_at": date("2014-01-29"), "description": "the girl..$", "followers_count": 1157, "friends_count": 682, "statues_count": 70809 }, "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("2015-11-25T10:00:33.000Z"), "id": 669576422390808576, "text": "Bro I want some cool as friends to go to ihop with 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": 944833346, "name": "ANG", "screen_name": "Stefano2society", "lang": "en", "location": "KEIRAQ OREGON", "create_at": date("2012-11-12"), "description": "null", "followers_count": 531, "friends_count": 193, "statues_count": 21437 }, "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("2015-11-25T10:00:33.000Z"), "id": 669576422424338433, "text": "Same https://t.co/NaUynI3v0y", "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": "MVP Lou Chamberlin✨", "screen_name": "LegendJohnson45", "lang": "en", "location": "#TheySleepMovement", "create_at": date("2014-06-07"), "description": "6'5 l D-1 Student-Athlete l Defensive Tackle at #PVAMU #57 l #InnovVtive l Photographer. l BJM.", "followers_count": 2287, "friends_count": 2053, "statues_count": 32064 }, "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("2015-11-25T10:00:33.000Z"), "id": 669576422445314052, "text": "blacked out and told someone I would bring Mac and Cheese to friendsgiving on thanksgiving but like i don't remember who he is �� fuck", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 330764971, "name": "CL", "screen_name": "CourtLangfield", "lang": "en", "location": "null", "create_at": date("2011-07-06"), "description": "Arizona State University Kappa Kappa Gamma member of the daalts. it's a lifestyle", "followers_count": 663, "friends_count": 541, "statues_count": 17309 }, "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("2015-11-25T10:00:33.000Z"), "id": 669576422768291840, "text": "Overhearing Ottawa players talking to each other, \"That guy Varlamov is sick\" #Avalanche", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Avalanche" }}, "user": { "id": 166793604, "name": "J.J. Jerez", "screen_name": "JJoftheYear", "lang": "en", "location": "null", "create_at": date("2010-07-14"), "description": "The cat's pajamas. Mile High Sports AM 1340. CU Grad", "followers_count": 148, "friends_count": 254, "statues_count": 669 }, "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-25T10:00:34.000Z"), "id": 669576422902509568, "text": "The Vitamin Shoppe: Assistant Store Manager (#Hoover, AL) https://t.co/Ainv2UyMPM #Retail #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.804959,33.381831"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hoover", "Retail", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 28467671, "name": "TMJ-BHM Retail Jobs", "screen_name": "tmj_bhm_retail", "lang": "en", "location": "Birmingham, AL", "create_at": date("2009-04-02"), "description": "Follow this account for geo-targeted Retail job tweets in Birmingham, AL. Need help? Tweet us at @CareerArc!", "followers_count": 349, "friends_count": 310, "statues_count": 374 }, "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("2015-11-25T10:00:34.000Z"), "id": 669576423254810624, "text": "I feel like I have to take banquet dress shopping seriously :/", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 68219414, "name": "Chel§ea", "screen_name": "chelseaticonuwu", "lang": "en", "location": "null", "create_at": date("2009-08-23"), "description": "null", "followers_count": 346, "friends_count": 105, "statues_count": 8197 }, "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("2015-11-25T10:00:34.000Z"), "id": 669576423280021504, "text": "on the grounds that they merely participated in a widespread institution of their day,", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 255307986, "name": "Michael Huggins", "screen_name": "MichaelHugg2591", "lang": "en", "location": "Memphis", "create_at": date("2011-02-20"), "description": "DWM, technical writer, avid reader, skeptic, trivia buff, center-right in politics, walks for fitness, likes classical music, museums, good conversation.", "followers_count": 306, "friends_count": 0, "statues_count": 43153 }, "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("2015-11-25T10:00:34.000Z"), "id": 669576423288406016, "text": "Is my family the only ones that only do turkey and ham for thanksgiving ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 579460333, "name": "MyLifeAsAPrincess✨", "screen_name": "Princessjenni_x", "lang": "en", "location": "osvaldo❤️", "create_at": date("2012-05-13"), "description": "♏️✨", "followers_count": 174, "friends_count": 237, "statues_count": 11552 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-25T10:00:34.000Z"), "id": 669576423489691648, "text": "Anyone who does not like holidays do not know the real meaning behind it. So much about holiday's are about family and friends js.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25572164, "name": "JoyDSchmidt", "screen_name": "BTRDriver4eva", "lang": "en", "location": "home is where the heart is ❤️", "create_at": date("2009-03-20"), "description": "Met Heffron Drive 6/29/14 @YumaPig1 DM: 1/6/15 @heffrondrive fave 6/26/15 thanks Ken. Kendall RT 8/12/15 Kendall fave 9/24/15 @challencates followed 11/5/15", "followers_count": 4844, "friends_count": 4888, "statues_count": 235096 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, TN", "id": "003df0cb3e9b4d0e", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-89.536711,35.176596 -89.48108,35.263347") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47047, "countyName": "Fayette", "cityID": 4754920, "cityName": "Oakland" } }
+{ "create_at": datetime("2015-11-25T10:00:34.000Z"), "id": 669576423531646976, "text": "That awkward moment when you are snap chatting someone and a stranger looks at you then smiles ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3023001211, "name": "Marissa Tijerina❥", "screen_name": "mtijerina5_22", "lang": "en", "location": "Mathis,Tx", "create_at": date("2015-02-16"), "description": "⓲ || Isaiah❤️ || Proverbs 3:5-6✞|| †V.XX.MMX†", "followers_count": 265, "friends_count": 234, "statues_count": 1561 }, "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("2015-11-25T10:00:34.000Z"), "id": 669576423665893376, "text": "@notBuntyPatel part jokes but father Patel and I were about to make sangria so this is personal", "in_reply_to_status": 669575000676929536, "in_reply_to_user": 21055325, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21055325 }}, "user": { "id": 21055325, "name": "bunty patel", "screen_name": "notBuntyPatel", "lang": "en", "location": "norman.", "create_at": date("2009-02-16"), "description": "mustard on the beat", "followers_count": 1231, "friends_count": 972, "statues_count": 16088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wilburton, OK", "id": "750e68a1a82251a8", "name": "Wilburton", "place_type": "city", "bounding_box": rectangle("-95.346679,34.895513 -95.257731,34.92759") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40077, "countyName": "Latimer", "cityID": 4081000, "cityName": "Wilburton" } }
+{ "create_at": datetime("2015-11-25T10:00:34.000Z"), "id": 669576423695237120, "text": "But probably the last time.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2307922502, "name": "A.", "screen_name": "Asariah2blessed", "lang": "en", "location": "Pearsall, Texas", "create_at": date("2014-01-23"), "description": "somewhere running from the cops.", "followers_count": 469, "friends_count": 508, "statues_count": 12833 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pearsall, TX", "id": "8e0fcb138f08c738", "name": "Pearsall", "place_type": "city", "bounding_box": rectangle("-99.125353,28.8701 -99.072782,28.911642") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48163, "countyName": "Frio", "cityID": 4856384, "cityName": "Pearsall" } }
+{ "create_at": datetime("2015-11-25T10:00:34.000Z"), "id": 669576423909249024, "text": "I gotta get up and get my own stuff, cant depend on no nigga. Thats always been an priority 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": 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": 1137, "friends_count": 1462, "statues_count": 16971 }, "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("2015-11-25T10:00:34.000Z"), "id": 669576424144150530, "text": "Childrens! matheus_ribaas gabrielkrutri inauegabrielli ���� @ Miami Beach,… https://t.co/cYYDe4LyMp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.1341,25.813"), "retweet_count": 0, "lang": "sk", "is_retweet": false, "user": { "id": 385710091, "name": "A l i s s o n ", "screen_name": "_alissuuuuu", "lang": "pt", "location": "Caxias do Sul ", "create_at": date("2011-10-05"), "description": "http://instagram.com/_alissuuuuu", "followers_count": 351, "friends_count": 575, "statues_count": 23545 }, "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-25T10:00:34.000Z"), "id": 669576424148348928, "text": "@Dougielarge Peter Gammons took a shot at u guys.\"But trade Fernandez? What a clever way to make them as relevant as the Florida Panthers.\"", "in_reply_to_status": 669575305305157633, "in_reply_to_user": 37239551, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 303151202 }}, "user": { "id": 37239551, "name": "Ralph Lucarella", "screen_name": "NCPANTHERS1", "lang": "en", "location": "Franklin, NC", "create_at": date("2009-05-02"), "description": "I'm into South Fla sports, Politics, Current Events and SciFi. Proud Father of Tia Lucarella. Favorite Florida Panther Player Dave Bolland.", "followers_count": 242, "friends_count": 493, "statues_count": 19174 }, "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("2015-11-25T10:00:34.000Z"), "id": 669576424215330816, "text": "I gag when I see people say defiantly when they mean definitely", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 190449616, "name": "lil mari", "screen_name": "msteezy_", "lang": "en", "location": "d06s", "create_at": date("2010-09-13"), "description": "supa silly, supa tired, supa done, gn", "followers_count": 395, "friends_count": 214, "statues_count": 26561 }, "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("2015-11-25T10:00:34.000Z"), "id": 669576424269807616, "text": "@XOXO__Lindsey how...how are you still in bed? I'm in the bed now and you're not here... :+)", "in_reply_to_status": 669576096455778304, "in_reply_to_user": 184667940, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 184667940 }}, "user": { "id": 366307860, "name": "jren ♕", "screen_name": "stybeautifulxx", "lang": "en", "location": "Asgard", "create_at": date("2011-09-01"), "description": "my heart is gold and my hands are cold.", "followers_count": 427, "friends_count": 385, "statues_count": 7390 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Davison, MI", "id": "00ae43d53fe3ca08", "name": "Davison", "place_type": "city", "bounding_box": rectangle("-83.576507,42.975833 -83.488029,43.076749") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2619880, "cityName": "Davison" } }
+{ "create_at": datetime("2015-11-25T10:00:34.000Z"), "id": 669576424571826177, "text": "Clinical Resources #Healthcare : Clinical Sales Representative - Sales Clinician - Clinical Liaison (#Boston, MA) https://t.co/uMhztI6i5S", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.0597732,42.3584308"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "Healthcare", "Boston" }}, "user": { "id": 67911583, "name": "Clinical Resources ", "screen_name": "ClinRes", "lang": "en", "location": "Atlanta, Georgia", "create_at": date("2009-08-22"), "description": "A healthcare staffing and professional services firm operating nationwide! / RTs ≠ endorsements / Call us at 404-343-7227! View Jobs: http://t.co/1VWhFiBPTI", "followers_count": 330, "friends_count": 438, "statues_count": 2489 }, "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("2015-11-25T10:00:34.000Z"), "id": 669576424777318400, "text": "I work 12-9 today, God bless. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 990146905, "name": "cindeezy", "screen_name": "Cindyriveraaa", "lang": "en", "location": "null", "create_at": date("2012-12-04"), "description": "19 • PNW• SSCC• FG1", "followers_count": 176, "friends_count": 137, "statues_count": 4962 }, "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-25T10:00:34.000Z"), "id": 669576424949284864, "text": "@DeannaYujin @saigey_baby", "in_reply_to_status": 669576354958999553, "in_reply_to_user": 3065743729, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3065743729, 618467148 }}, "user": { "id": 3065743729, "name": "D", "screen_name": "DeannaYujin", "lang": "en", "location": "null", "create_at": date("2015-03-06"), "description": "null", "followers_count": 208, "friends_count": 262, "statues_count": 1528 }, "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("2015-11-25T10:00:34.000Z"), "id": 669576425314385920, "text": "Are you thankful for everything? #BlackFriday https://t.co/vzaLIeTigY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BlackFriday" }}, "user": { "id": 1382550824, "name": "Jetty Bo Betty", "screen_name": "jett2whit", "lang": "en", "location": "null", "create_at": date("2013-04-26"), "description": "Child of God. I love my family and good food!!! Always tweeting about bacon!!! Music lover", "followers_count": 836, "friends_count": 711, "statues_count": 20424 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairburn, GA", "id": "0006d55513b0c159", "name": "Fairburn", "place_type": "city", "bounding_box": rectangle("-84.626791,33.509481 -84.531197,33.59282") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1328380, "cityName": "Fairburn" } }
+{ "create_at": datetime("2015-11-25T10:00:34.000Z"), "id": 669576425410809856, "text": "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": 379975007, "name": "Juana", "screen_name": "JuanaTweets_", "lang": "en-gb", "location": "Lincoln Park, MI", "create_at": date("2011-09-25"), "description": "snap me: juanana", "followers_count": 1145, "friends_count": 186, "statues_count": 43670 }, "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("2015-11-25T10:00:34.000Z"), "id": 669576425452662784, "text": "I would've happy if my face wasn't so dry ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1028806279, "name": "☼ kel bel", "screen_name": "keellytomlinson", "lang": "en", "location": "null", "create_at": date("2012-12-22"), "description": "CA ➙ MO | The Aristotle of our generation ✨ ΣΣΣ", "followers_count": 418, "friends_count": 402, "statues_count": 18840 }, "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-25T10:00:34.000Z"), "id": 669576425452773377, "text": "Ready for my shipments to come in so I can finish working on my littles room..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 304105227, "name": "Baby J", "screen_name": "JazJohnsonn", "lang": "en", "location": "DE✈CO✈DE✈GA", "create_at": date("2011-05-23"), "description": "No I'm not lucky, I'm blessed yes!", "followers_count": 702, "friends_count": 447, "statues_count": 27255 }, "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("2015-11-25T10:00:34.000Z"), "id": 669576425603596288, "text": "What are you thankful for ? \n\nIf you are reading this you have a lot to be thankful. \nI'm thankful… https://t.co/0SylbzwMNJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.328,34.1802"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 130777146, "name": "❤️JOSIE❤️", "screen_name": "JOSIEROM", "lang": "en", "location": "LOS ANGELES CA", "create_at": date("2010-04-08"), "description": "LOVE SINGING I ❤ @JENNIRIVERA #J10712 @HOROSCOPOSDEDGO #HDDARMY000225 #ItworksGlobal #Entrepreneur", "followers_count": 2954, "friends_count": 2771, "statues_count": 14901 }, "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("2015-11-25T10:00:34.000Z"), "id": 669576425653993473, "text": "Want to work at Plastipak Packaging? We're #hiring in #Champaign, IL! Click for details: https://t.co/cEuxlVf6Oc #Manufacturing #Veterans", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.2433829,40.1164204"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Champaign", "Manufacturing", "Veterans" }}, "user": { "id": 1157303605, "name": "Plastipak Jobs", "screen_name": "PlastipakJobs", "lang": "en", "location": "null", "create_at": date("2013-02-07"), "description": "Plastipak Packaging is an industry leader in the design and manufacturing of plastic rigid containers of the highest quality.", "followers_count": 236, "friends_count": 30, "statues_count": 6413 }, "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("2015-11-25T10:00:34.000Z"), "id": 669576425716908033, "text": "Smells like #Christmas @peabodymemphis w/ a massive Gingerbread Village #Memphis @ The Peabody Memphis https://t.co/N4zXdKgsIG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.05188191,35.14250665"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Christmas", "Memphis" }}, "user_mentions": {{ 16915511 }}, "user": { "id": 71399928, "name": "Kevin Kern", "screen_name": "SpinZone", "lang": "en", "location": "Spin Zone", "create_at": date("2009-09-03"), "description": "Kern is a world traveling Disney nerd & former TV news reporter turned PR professional based in Memphis with a focus on entertainment & travel publicity.", "followers_count": 876, "friends_count": 788, "statues_count": 863 }, "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("2015-11-25T10:00:34.000Z"), "id": 669576425792479232, "text": "idk 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": 2754040664, "name": "julianna", "screen_name": "juliannamazz", "lang": "en", "location": "null", "create_at": date("2014-08-21"), "description": "null", "followers_count": 299, "friends_count": 162, "statues_count": 1652 }, "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-25T10:00:34.000Z"), "id": 669576425846865920, "text": "\"Siri what's good to eat around here\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1352258622, "name": "Matt Nyitray", "screen_name": "whereisnyitray", "lang": "en", "location": "gcc gauchos ⚽️", "create_at": date("2013-04-14"), "description": "18. stay humble culero", "followers_count": 727, "friends_count": 543, "statues_count": 11169 }, "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("2015-11-25T10:00:34.000Z"), "id": 669576425859518465, "text": "Same �� https://t.co/rbcrEKA7cq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2453793836, "name": "Patty ❤", "screen_name": "Patty_Ramirez03", "lang": "en", "location": "null", "create_at": date("2014-04-19"), "description": "Zacatecas | MN | Snapchat: ramirez009", "followers_count": 138, "friends_count": 129, "statues_count": 885 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Brighton, MN", "id": "66624adcd9ca0231", "name": "New Brighton", "place_type": "city", "bounding_box": rectangle("-93.227785,45.035476 -93.18262,45.093582") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27123, "countyName": "Ramsey", "cityID": 2745430, "cityName": "New Brighton" } }
+{ "create_at": datetime("2015-11-25T10:00:34.000Z"), "id": 669576425905721346, "text": "I'll be eating Reese puffs until I'm 135 years old tbh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 409600400, "name": "RICH HOMIE NATE", "screen_name": "NateTorton", "lang": "en", "location": "North Augusta", "create_at": date("2011-11-10"), "description": "Live and die for the adventure. follow on me twit @NateTorton", "followers_count": 549, "friends_count": 479, "statues_count": 8256 }, "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("2015-11-25T10:00:34.000Z"), "id": 669576426186665985, "text": "@vincenzolandino Temperature?", "in_reply_to_status": 669576046996545536, "in_reply_to_user": 947914190, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 947914190 }}, "user": { "id": 32167968, "name": "Sherrie Rohde", "screen_name": "sherrierohde", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-04-16"), "description": "Community Manager @Magento. Co-Produce @myCMGR + #CMGRHangout. @Hootsuite + @LithiumTech Certified. Terrified of status quo. Motivated by adventure. ENFP. ", "followers_count": 5199, "friends_count": 985, "statues_count": 69743 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-25T10:00:34.000Z"), "id": 669576426312503297, "text": "Ghirardelli: Assistant General Manager - Anaheim (#Anaheim, CA) https://t.co/ahIzul8b0W #BusinessMgmt #management #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.9145036,33.8352932"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Anaheim", "BusinessMgmt", "management", "Job", "Jobs", "Hiring" }}, "user": { "id": 2228943546, "name": "Ghirardelli Careers", "screen_name": "ghirardellijobs", "lang": "en", "location": "null", "create_at": date("2013-12-03"), "description": "Ghirardelli invites you to explore our open #jobs and discover a place where your ideas come to life, as part of our delicious legacy.", "followers_count": 54, "friends_count": 1, "statues_count": 98 }, "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-25T10:00:34.000Z"), "id": 669576426635399169, "text": "some one come over i will cook 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": 2367649297, "name": "chloe alex", "screen_name": "_chloealex", "lang": "en", "location": "california", "create_at": date("2014-03-01"), "description": "null", "followers_count": 489, "friends_count": 311, "statues_count": 861 }, "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("2015-11-25T10:00:34.000Z"), "id": 669576426652332033, "text": "all of these are light especially #2 & #4. thats boxed mac & cheese in #3 too. wtf they a white family or something? https://t.co/0rtQ4tKbLB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 316066489, "name": "soulja dae", "screen_name": "rydaeizm", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2011-06-12"), "description": "@ovolexi_ 's", "followers_count": 1063, "friends_count": 478, "statues_count": 33901 }, "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-25T10:00:34.000Z"), "id": 669576426694152192, "text": "Snapchat: bre_themac��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3993479958, "name": "BreSmoove✨", "screen_name": "bregotthajuice1", "lang": "en", "location": "Cedar Hill, TX", "create_at": date("2015-10-23"), "description": "Hacked at 2K so FOLLOW ME!", "followers_count": 568, "friends_count": 544, "statues_count": 1864 }, "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("2015-11-25T10:00:34.000Z"), "id": 669576426887258116, "text": "@MikeVogel86 @MariaSharapova @Sugarpova I'm okay with that.", "in_reply_to_status": 669570860320227328, "in_reply_to_user": 2200916395, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2200916395, 288333760, 1544256787 }}, "user": { "id": 91781444, "name": "Matthew!", "screen_name": "MRisingStar18", "lang": "en", "location": "null", "create_at": date("2009-11-22"), "description": "I live tweet tennis matches, TV shows, and other ridiculous life events. You should unfollow me. #MariaSharapova needs to buy a new wig", "followers_count": 641, "friends_count": 304, "statues_count": 39409 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden City, NY", "id": "8da7a05f8a865e05", "name": "Garden City", "place_type": "city", "bounding_box": rectangle("-73.682722,40.707292 -73.608713,40.742953") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3628178, "cityName": "Garden City" } }
+{ "create_at": datetime("2015-11-25T10:00:34.000Z"), "id": 669576426958393345, "text": "@PeteSchirrick17 @va21AFC @espn I love Tebow!!! I think he got a bad shake! His issue is he can't read coverages and he makes bad mistakes.", "in_reply_to_status": 669575931971809280, "in_reply_to_user": 720846931, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 720846931, 2964568449, 2557521 }}, "user": { "id": 723176028, "name": "Scott", "screen_name": "cadensdad78", "lang": "en", "location": "null", "create_at": date("2012-07-28"), "description": "null", "followers_count": 28, "friends_count": 219, "statues_count": 149 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-25T10:00:34.000Z"), "id": 669576427059044352, "text": "Daddies at the dmv woah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1870110732, "name": "T", "screen_name": "_tiarajones_", "lang": "en", "location": "909", "create_at": date("2013-09-15"), "description": "forehead big but my heart bigger", "followers_count": 960, "friends_count": 456, "statues_count": 10927 }, "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("2015-11-25T10:00:35.000Z"), "id": 669576427138711552, "text": "@yomoxie ����������", "in_reply_to_status": -1, "in_reply_to_user": 29375776, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 29375776 }}, "user": { "id": 217885601, "name": "Brynn Wallner", "screen_name": "BRYNNTRILL", "lang": "en", "location": "LA ", "create_at": date("2010-11-20"), "description": "spring break ............. foreva", "followers_count": 866, "friends_count": 339, "statues_count": 6756 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-25T10:00:35.000Z"), "id": 669576427176611840, "text": "Loveeee the way @cellestewilson said \"Megan Thrasher\" ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2971264447 }}, "user": { "id": 463766604, "name": "Meg ❊", "screen_name": "MeganThrasherrr", "lang": "en", "location": "probably laughing ", "create_at": date("2012-01-14"), "description": "10-10-15 was the best day of my life. Josh and Colleen mean the world to me.", "followers_count": 430, "friends_count": 466, "statues_count": 19225 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairview, GA", "id": "1dbd6de46c32ef14", "name": "Fairview", "place_type": "city", "bounding_box": rectangle("-85.319391,34.892521 -85.265963,34.961508") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13295, "countyName": "Walker", "cityID": 1328632, "cityName": "Fairview" } }
+{ "create_at": datetime("2015-11-25T10:00:35.000Z"), "id": 669576427193434112, "text": "At moes by myself 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": 312841332, "name": "casey", "screen_name": "MiNameIsCasey", "lang": "en", "location": "just chillin in cedar rapids", "create_at": date("2011-06-07"), "description": "what the heckers", "followers_count": 235, "friends_count": 188, "statues_count": 17109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wyomissing, PA", "id": "1e735f8fb045152d", "name": "Wyomissing", "place_type": "city", "bounding_box": rectangle("-75.988788,40.30576 -75.947615,40.35633") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42011, "countyName": "Berks", "cityID": 4286880, "cityName": "Wyomissing" } }
+{ "create_at": datetime("2015-11-25T10:00:35.000Z"), "id": 669576427268796416, "text": "Aww thanks I really needed that ������ https://t.co/VI6BWWfQPB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 151502751, "name": "PARKER™", "screen_name": "whatif31", "lang": "en", "location": "Dallas,Texas ", "create_at": date("2010-06-03"), "description": "#SFA |Dallas Nac| Management/Criminal Justice Live your life and forget your age.", "followers_count": 541, "friends_count": 1077, "statues_count": 37918 }, "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-25T10:00:35.000Z"), "id": 669576427394703360, "text": "Fam goals @LOLmeganblack https://t.co/zMFM87RNW8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 31206117 }}, "user": { "id": 59774850, "name": "diesel jeans", "screen_name": "brk__blck", "lang": "en", "location": "ny", "create_at": date("2009-07-24"), "description": "null", "followers_count": 393, "friends_count": 188, "statues_count": 22619 }, "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("2015-11-25T10:00:35.000Z"), "id": 669576427583377408, "text": "I feel the same way https://t.co/VY5bk7rOQu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1586446344, "name": "Harry Jackson", "screen_name": "hrhh1725", "lang": "en", "location": "null", "create_at": date("2013-07-11"), "description": "just your typical Puerto Rican from NY. I believe in Karman and try to live my life positive. I give to people what i want to get back from them. Trilingual,", "followers_count": 260, "friends_count": 491, "statues_count": 5623 }, "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("2015-11-25T10:00:35.000Z"), "id": 669576427717591040, "text": "I wish I was a wolf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1661137176, "name": "Javie from Cali", "screen_name": "lbcityjavi", "lang": "en", "location": "somewhere ", "create_at": date("2013-08-10"), "description": "● Just a little dude trying to make it big ● I like all races ● #TXST20", "followers_count": 778, "friends_count": 442, "statues_count": 29682 }, "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("2015-11-25T10:00:35.000Z"), "id": 669576428384481280, "text": "Skipping advisory was a very good idea", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2188788686, "name": "Andrea Luper", "screen_name": "superluper16", "lang": "en", "location": "null", "create_at": date("2013-11-11"), "description": "NTHS | ASB President | Horses | DECA | Running Striving to become the best person I can be Romans 8:38 ϯ", "followers_count": 434, "friends_count": 600, "statues_count": 3453 }, "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("2015-11-25T10:00:35.000Z"), "id": 669576428518817792, "text": "#BarkerDam @joshuatreenp @ Barker Dam Trail Joshua Tree National https://t.co/5vcM7Ng0oY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-116.43115747,34.12035921"), "retweet_count": 0, "lang": "de", "is_retweet": false, "hashtags": {{ "BarkerDam" }}, "user_mentions": {{ 3825684196 }}, "user": { "id": 107060478, "name": "Niki Russell", "screen_name": "niki_russell", "lang": "en", "location": "null", "create_at": date("2010-01-21"), "description": "Artist, curator and writer w/ @wearereactor @weareprimary \nhttp://www.reactor.org.uk\nhttp://www.weareprimary.org", "followers_count": 1189, "friends_count": 962, "statues_count": 2182 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yucca Valley, CA", "id": "78d409ea751c955c", "name": "Yucca Valley", "place_type": "city", "bounding_box": rectangle("-116.480488,34.076028 -116.364729,34.165915") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 687056, "cityName": "Yucca Valley" } }
+{ "create_at": datetime("2015-11-25T10:00:35.000Z"), "id": 669576428711755776, "text": "@asvpsimone no", "in_reply_to_status": 669576332930498560, "in_reply_to_user": 718669573, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 718669573 }}, "user": { "id": 1397178678, "name": "Caramel Delight", "screen_name": "jahova11", "lang": "en", "location": "null", "create_at": date("2013-05-02"), "description": "free bro", "followers_count": 193, "friends_count": 114, "statues_count": 11622 }, "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("2015-11-25T10:00:35.000Z"), "id": 669576428745154560, "text": "Down with piss", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 990116622, "name": "yung cranberry sauce", "screen_name": "sufferances", "lang": "en", "location": "So Cal", "create_at": date("2012-12-04"), "description": "Alex // 18 // gay as hell // pretends to be a bear // shitposting is all I know // INFJ // neighborhood hoe //\nicon by @horrorbuns", "followers_count": 263, "friends_count": 1102, "statues_count": 13381 }, "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-25T10:00:35.000Z"), "id": 669576428774518784, "text": "Me when leaving SB https://t.co/taVe0fXCwy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2495682056, "name": "Mel", "screen_name": "MzMelanieNicole", "lang": "en", "location": "San Francisco, CA", "create_at": date("2014-05-14"), "description": "Photographer | SF | SB", "followers_count": 167, "friends_count": 125, "statues_count": 15250 }, "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-25T10:00:35.000Z"), "id": 669576429223309312, "text": "@Semperfelix @dailydot ... why not?", "in_reply_to_status": 669571877988380672, "in_reply_to_user": 85729788, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 85729788, 211620426 }}, "user": { "id": 25958422, "name": "Andy Mizrahi", "screen_name": "Andcredible_Miz", "lang": "en", "location": "Asbury Park, NJ", "create_at": date("2009-03-22"), "description": "Knick Fan. Director/Producer/Writer. Comedian: Standup/Improvisor. Biz Owner. Sandwich Lover. Snapchat Storyteller @andomiz", "followers_count": 249, "friends_count": 367, "statues_count": 468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Asbury Park, NJ", "id": "68f46439c0f4a207", "name": "Asbury Park", "place_type": "city", "bounding_box": rectangle("-74.027125,40.213215 -73.995775,40.231329") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3401960, "cityName": "Asbury Park" } }
+{ "create_at": datetime("2015-11-25T10:00:35.000Z"), "id": 669576429315555328, "text": "Join the Mandarin Oriental Hotel Las Vegas team! See our latest #job opening here: https://t.co/L1R4JcXqhj #LasVegas, NV #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.1745805,36.1062236"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "LasVegas", "Hiring" }}, "user": { "id": 38052783, "name": "TMJ-LAS Jobs", "screen_name": "tmj_las_jobs", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2009-05-05"), "description": "Follow this account for geo-targeted Other job tweets in Las Vegas, NV. Need help? Tweet us at @CareerArc!", "followers_count": 608, "friends_count": 538, "statues_count": 519 }, "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("2015-11-25T10:00:35.000Z"), "id": 669576429672243200, "text": "Thankful for these people ❤️ @ Knaus Berry Farm https://t.co/XRs1hUvFHG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.4510574,25.5356903"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1609244084, "name": "lil olive", "screen_name": "olive_schuitema", "lang": "en", "location": "null", "create_at": date("2013-07-20"), "description": "worry less, pray more", "followers_count": 196, "friends_count": 342, "statues_count": 2105 }, "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("2015-11-25T10:00:35.000Z"), "id": 669576430091571200, "text": "Want to work in #CharlotteSouth, NC? View our latest opening: https://t.co/ssOLniYaFa #Sales #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.8431267,35.2270869"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CharlotteSouth", "Sales", "Job", "Jobs", "Hiring" }}, "user": { "id": 870560696, "name": "SpectrumBrandsJobs", "screen_name": "WorkAtSpectrum", "lang": "en", "location": "null", "create_at": date("2012-10-09"), "description": "Spectrum Brands hires and rewards great people who understand the importance of a strong culture that values integrity, commitment, teamwork, and diversity.", "followers_count": 276, "friends_count": 6, "statues_count": 2567 }, "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("2015-11-25T10:00:35.000Z"), "id": 669576430594867201, "text": "If I'm picking you up, be ready when I get there", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3193706317, "name": "Madeleine", "screen_name": "MadeleineBodden", "lang": "en", "location": "ATX", "create_at": date("2015-05-12"), "description": "@Drake's official choreographer. Check out my work here: https://www.youtube.com/watch?v=uxpDa-c-4Mc", "followers_count": 419, "friends_count": 668, "statues_count": 4111 }, "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-25T10:00:35.000Z"), "id": 669576430678740992, "text": "@mmaddiejo and I are going to the zoo but were delayed THIRTY MINUTES because she couldn't pee in the stupid cup at the dermatologist", "in_reply_to_status": -1, "in_reply_to_user": 1403941478, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1403941478 }}, "user": { "id": 38584377, "name": "Anna McMaster", "screen_name": "annapie4", "lang": "en", "location": "BYU", "create_at": date("2009-05-07"), "description": "# scabs & sobs", "followers_count": 445, "friends_count": 244, "statues_count": 2248 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midway, UT", "id": "389543e8bdccae1c", "name": "Midway", "place_type": "city", "bounding_box": rectangle("-111.501501,40.496245 -111.450096,40.543048") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49051, "countyName": "Wasatch", "cityID": 4949820, "cityName": "Midway" } }
+{ "create_at": datetime("2015-11-25T10:00:35.000Z"), "id": 669576430909415424, "text": "@jurriaanvr thank you. You too", "in_reply_to_status": 669576255243722753, "in_reply_to_user": 15209654, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15209654 }}, "user": { "id": 16943168, "name": "Anton Stetner", "screen_name": "AntonStetner", "lang": "en", "location": "Everett, WA", "create_at": date("2008-10-23"), "description": "Founder of RESG, Real Estate, Entrepreneur, Investor, Cars, Music, Business, Bball, Snow Skiing, I love it all & welcome everyone to join me my adventures", "followers_count": 14672, "friends_count": 12212, "statues_count": 14109 }, "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("2015-11-25T10:00:35.000Z"), "id": 669576430951485440, "text": "I'm at Doña Paola in H. Matamoros https://t.co/Kkwr0lQTxp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.49675274,25.87777813"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2843072095, "name": "LMT", "screen_name": "ceub07", "lang": "en", "location": "null", "create_at": date("2014-10-06"), "description": "null", "followers_count": 11, "friends_count": 39, "statues_count": 754 }, "place": { "country": "México", "country_code": "México", "full_name": "Matamoros, Tamaulipas", "id": "3309acacf870f6f5", "name": "Matamoros", "place_type": "city", "bounding_box": rectangle("-97.946585,25.047842 -97.148199,26.070144") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4810768, "cityName": "Brownsville" } }
+{ "create_at": datetime("2015-11-25T10:00:35.000Z"), "id": 669576431081553924, "text": "I haven't worked out this long and hard in a long time. I may be in trouble and need help. \nLol. I… https://t.co/FGH9jttvRC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.63918823,37.07516565"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 615915756, "name": "Andrea Johnson", "screen_name": "dreazLLL", "lang": "en", "location": "null", "create_at": date("2012-06-22"), "description": "Love Life,kids & Granddaughter. Rocking Music #Beachbodycoach living #healthy life here 2 #motivate #love #Fitness hope 2 #inspire Love 2 Laugh Dare 2 Dream.", "followers_count": 620, "friends_count": 1994, "statues_count": 7175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Galena, KS", "id": "fa6144555493aba9", "name": "Galena", "place_type": "city", "bounding_box": rectangle("-94.649776,37.048904 -94.607093,37.091579") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20021, "countyName": "Cherokee", "cityID": 2025100, "cityName": "Galena" } }
+{ "create_at": datetime("2015-11-25T10:00:36.000Z"), "id": 669576431395958784, "text": "Join the Sterling Jewelers team! See our latest #Retail #job opening here: https://t.co/P4OeXWICN1 #Paramus, NJ #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0754189,40.9445428"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "Paramus", "Hiring" }}, "user": { "id": 3248774184, "name": "Kay Jewelers", "screen_name": "JobsAtKay", "lang": "en", "location": "null", "create_at": date("2015-06-18"), "description": "Your Future Starts Here.", "followers_count": 5, "friends_count": 0, "statues_count": 1092 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paramus, NJ", "id": "4b6a1f748c5219fa", "name": "Paramus", "place_type": "city", "bounding_box": rectangle("-74.101395,40.908544 -74.038944,40.980598") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3455950, "cityName": "Paramus" } }
+{ "create_at": datetime("2015-11-25T10:00:36.000Z"), "id": 669576431396118530, "text": "*eats Tony's for dinner* *eats Tony's for breakfast* its good to be home #pizza", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "pizza" }}, "user": { "id": 391725921, "name": "V", "screen_name": "vEATaRice", "lang": "en", "location": "the 607 to the 619 ☼ ☀ ", "create_at": date("2011-10-15"), "description": "if i need to give up beer to have abs then i don't want abs.", "followers_count": 309, "friends_count": 112, "statues_count": 17261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Apalachin, NY", "id": "0098eb71044ab24b", "name": "Apalachin", "place_type": "city", "bounding_box": rectangle("-76.202826,42.042754 -76.106353,42.099603") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga", "cityID": 3602308, "cityName": "Apalachin" } }
+{ "create_at": datetime("2015-11-25T10:00:36.000Z"), "id": 669576431454699520, "text": "@ta_naeeeeee you're welcome ��", "in_reply_to_status": 669576368439431168, "in_reply_to_user": 501320678, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 501320678 }}, "user": { "id": 208792853, "name": "brit the goat", "screen_name": "_sacredHeart", "lang": "en", "location": "playing volleyball ", "create_at": date("2010-10-27"), "description": "addicted to breaking hearts & moving on..#DoitforJuan |#UL19| @OBJ_3 ❤️", "followers_count": 3502, "friends_count": 2584, "statues_count": 97082 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wallace, LA", "id": "0058c8ba4e9ee92a", "name": "Wallace", "place_type": "city", "bounding_box": rectangle("-90.687578,30.029095 -90.639209,30.046796") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22095, "countyName": "St. John the Baptist", "cityID": 2279275, "cityName": "Wallace" } }
+{ "create_at": datetime("2015-11-26T10:00:00.000Z"), "id": 669938668157534208, "text": "@akadeathwish very dope, I'm about to roll up to my parents with like three bottles of Liqour LOL", "in_reply_to_status": 669938434249650176, "in_reply_to_user": 1515564157, "favorite_count": 0, "coordinate": point("-97.09856294,32.78610362"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1515564157 }}, "user": { "id": 1522332721, "name": "ALL BASS FIGHT RIFF", "screen_name": "Strifexlife", "lang": "en", "location": "Arlington, TX ", "create_at": date("2013-06-16"), "description": "Just a boy and his pup on a search for their sugar mama.", "followers_count": 1251, "friends_count": 573, "statues_count": 54711 }, "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("2015-11-26T10:00:00.000Z"), "id": 669938668203720705, "text": "Don't make plans with me if you already have plans, you just make me feel like shizz. Thnx ����❤️", "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": 390, "friends_count": 179, "statues_count": 1114 }, "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("2015-11-26T10:00:00.000Z"), "id": 669938668459524096, "text": "Just hanging out with Hannibal the night before Thanksgiving!! @ The Comedy Club on State https://t.co/qyPr5t85oG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.38803997,43.0747889"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 40170415, "name": "Zach Zwadzich", "screen_name": "Zlaka_84", "lang": "en", "location": "Madison, WI ", "create_at": date("2009-05-14"), "description": "31 A Brewers/Garfield/Jennifer Love Hewitt / America Ferrera / Karen Fisher Fan ♥♥♥♥", "followers_count": 31, "friends_count": 175, "statues_count": 8782 }, "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.491889 -86.24955,47.309715") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55025, "countyName": "Dane", "cityID": 5548000, "cityName": "Madison" } }
+{ "create_at": datetime("2015-11-26T10:00:00.000Z"), "id": 669938668476305408, "text": "Happy Thanksgiving to everyone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 236815900, "name": "Ben Berger", "screen_name": "BenBergerBaby", "lang": "en", "location": "null", "create_at": date("2011-01-11"), "description": "Ben is a working Actor in Los Angeles. A graduate of the UniversityOfArizona Ben Is a Wildcat for Life #BearDown. He is a Die Hard Chicago Sports fan.Follow Him", "followers_count": 1543, "friends_count": 2009, "statues_count": 36480 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:00.000Z"), "id": 669938668698738688, "text": "@LutmanEmma love u em", "in_reply_to_status": 669938557755199488, "in_reply_to_user": 2391671185, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2391671185 }}, "user": { "id": 2571350798, "name": "Lindsey", "screen_name": "lihondroulis", "lang": "en", "location": "null", "create_at": date("2014-06-16"), "description": "null", "followers_count": 184, "friends_count": 88, "statues_count": 119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Upper Arlington, OH", "id": "61f73dd1739509f0", "name": "Upper Arlington", "place_type": "city", "bounding_box": rectangle("-83.104102,39.989304 -83.042267,40.063889") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3979002, "cityName": "Upper Arlington" } }
+{ "create_at": datetime("2015-11-26T10:00:00.000Z"), "id": 669938668950306816, "text": "https://t.co/DdTsae1ymi check out my Jr. Year offensive highlights ✊��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1723225549, "name": "Uncle jcooley", "screen_name": "j_cooley253", "lang": "en", "location": "null", "create_at": date("2013-09-02"), "description": "For your hunks!", "followers_count": 602, "friends_count": 488, "statues_count": 4693 }, "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("2015-11-26T10:00:00.000Z"), "id": 669938669034315776, "text": "When it's your first thanksgiving without your dad and your mother Fucks it up even more ��������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 974971158, "name": "Skye", "screen_name": "Skye_babie", "lang": "en", "location": "null", "create_at": date("2012-11-27"), "description": "null", "followers_count": 558, "friends_count": 571, "statues_count": 6149 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Smyrna, TN", "id": "f9eb63ab4f7dd5db", "name": "Smyrna", "place_type": "city", "bounding_box": rectangle("-86.624429,35.879451 -86.458783,36.029213") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47149, "countyName": "Rutherford", "cityID": 4769420, "cityName": "Smyrna" } }
+{ "create_at": datetime("2015-11-26T10:00:00.000Z"), "id": 669938669059309568, "text": "My Auntie Got Me Driving All Around This Town ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 729288666, "name": "Dâñńø™", "screen_name": "YOUNGBIGHAM25", "lang": "en", "location": "Future MV-22", "create_at": date("2012-07-31"), "description": "17 | 2016 | I Play Ball", "followers_count": 600, "friends_count": 1094, "statues_count": 3502 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jasper, TX", "id": "99dc2d8acedd5585", "name": "Jasper", "place_type": "city", "bounding_box": rectangle("-94.060449,30.882109 -93.947336,30.966418") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48241, "countyName": "Jasper", "cityID": 4837420, "cityName": "Jasper" } }
+{ "create_at": datetime("2015-11-26T10:00:00.000Z"), "id": 669938669352972288, "text": "@amaral_crystal thankful for you and all the times you make me laugh and smile! I love you have a blessed day������", "in_reply_to_status": 669219978554380288, "in_reply_to_user": 961741550, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1349362261 }}, "user": { "id": 961741550, "name": "saraline ❦", "screen_name": "sarahaduhh__", "lang": "en", "location": "on the field ", "create_at": date("2012-11-20"), "description": "I win you lose // R.H. ❥", "followers_count": 1171, "friends_count": 1456, "statues_count": 40171 }, "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("2015-11-26T10:00:00.000Z"), "id": 669938669449551872, "text": "I was raised up beneath the shade of a Georgia Pine and thats home you know, sweet tea, pecan pie and homemade wine, where the peaches grow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 151724833, "name": "Amber Dell", "screen_name": "_AmberDell", "lang": "en", "location": "atl", "create_at": date("2010-06-03"), "description": "ignorance can be such bliss", "followers_count": 396, "friends_count": 364, "statues_count": 7825 }, "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("2015-11-26T10:00:00.000Z"), "id": 669938669562777600, "text": "Happy thanksgiving to my family and friends ❤️ so thankful for what I have 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": 1669018640, "name": "▪️Julia Mohamed▪️", "screen_name": "juliafae94", "lang": "en", "location": "null", "create_at": date("2013-08-13"), "description": "null", "followers_count": 144, "friends_count": 275, "statues_count": 2307 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakes of the Four Seasons, IN", "id": "441e82ae8a838fa5", "name": "Lakes of the Four Seasons", "place_type": "city", "bounding_box": rectangle("-87.242047,41.389244 -87.20517,41.420313") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18127, "countyName": "Porter", "cityID": 1841530, "cityName": "Lakes of the Four Seasons" } }
+{ "create_at": datetime("2015-11-26T10:00:00.000Z"), "id": 669938670015672320, "text": "Especially thankful for my sweet sister Emily today! Happy Thanksgiving!… https://t.co/eQioMvPMDu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.2019,30.2639"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 547271465, "name": "Alex Priest", "screen_name": "alexxpriest", "lang": "en", "location": "Deep in the ❤️ of Texas ", "create_at": date("2012-04-06"), "description": "Let Go and Let God • insta: _alexpriest • snapchat: a.priest • LU'19 • AXO", "followers_count": 281, "friends_count": 336, "statues_count": 4836 }, "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("2015-11-26T10:00:00.000Z"), "id": 669938670321971200, "text": "Candied yams have to be 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": 261860843, "name": "NavarusGrant", "screen_name": "ngrant89", "lang": "en", "location": "null", "create_at": date("2011-03-06"), "description": "FL✈️VA Instagram- @navarusg1 #ODU19 Jamaican☀️", "followers_count": 1023, "friends_count": 719, "statues_count": 4962 }, "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("2015-11-26T10:00:00.000Z"), "id": 669938670351314947, "text": "Postgame scene as @KonkreteKids coach Mark Scisly talks to his kids. #lvvarsity https://t.co/rcPN3mzlnr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "lvvarsity" }}, "user_mentions": {{ 967343454 }}, "user": { "id": 63520432, "name": "Keith Groller", "screen_name": "KeithGroller", "lang": "en", "location": "Lehigh Valley, PA", "create_at": date("2009-08-06"), "description": "Senior writer & TV/radio show host for The Morning Call, Allentown PA. Cover PIAA football, basketball, softball .Lehigh football. keith.groller@mcall.com", "followers_count": 4085, "friends_count": 854, "statues_count": 17597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Northampton, PA", "id": "80200fa386aecd14", "name": "Northampton", "place_type": "city", "bounding_box": rectangle("-75.515065,40.66693 -75.471459,40.704633") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42095, "countyName": "Northampton", "cityID": 4254696, "cityName": "Northampton" } }
+{ "create_at": datetime("2015-11-26T10:00:00.000Z"), "id": 669938670363799552, "text": "You see me ballin like Spalding", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 572831144, "name": "Tanz™", "screen_name": "tanner_wilder", "lang": "en", "location": "United States", "create_at": date("2012-05-06"), "description": "They Scream Out My Failures, And Whisper My Accomplishments • #CGOD • Be Humble, But Stay Hungry • #WWYD • #IDGT • Scorpio ♏️ • SC: tanner.wilder", "followers_count": 2186, "friends_count": 1091, "statues_count": 92923 }, "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("2015-11-26T10:00:00.000Z"), "id": 669938670556745728, "text": "We just have to work together. #HappyThanksgivingEveryone ,", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HappyThanksgivingEveryone" }}, "user": { "id": 96056919, "name": "Maria Patafria", "screen_name": "BornfortheFame", "lang": "en", "location": "There, TX", "create_at": date("2009-12-10"), "description": "Living in his kingdom of magical dreams, with his cub ❤", "followers_count": 461, "friends_count": 1615, "statues_count": 26144 }, "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-26T10:00:00.000Z"), "id": 669938670850338816, "text": "chasingmyhalo's photo https://t.co/ESFC0aEy0U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 14641111, "name": "Dianne Bengtson", "screen_name": "dianneb", "lang": "en", "location": "Redmond, WA", "create_at": date("2008-05-03"), "description": "Writer. Photographer. Wanderer. Planning a loooong walk on the Camino de Santiago. Follow me on Periscope @dianneb #TagTribes #dopescope", "followers_count": 662, "friends_count": 1235, "statues_count": 1346 }, "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("2015-11-26T10:00:00.000Z"), "id": 669938670883966976, "text": "If I find a woman that can cook like my momma, ���� I'ma give her 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": 2726958584, "name": "Jevonté Porter", "screen_name": "jp_beenreal", "lang": "en", "location": "null", "create_at": date("2014-08-12"), "description": "The two most important days in your life are the day you were born and the day you figure out why! ~Mark Twain", "followers_count": 456, "friends_count": 365, "statues_count": 1119 }, "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("2015-11-26T10:00:00.000Z"), "id": 669938671177437185, "text": "Wind 0.0 mph ---. Barometer 30.487 in, Rising slowly. Temperature 17.1 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.98388889,47.66361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 67603939, "name": "Bemidji Weather", "screen_name": "BemidjiWx", "lang": "en", "location": "Bemidji, MN, USA", "create_at": date("2009-08-21"), "description": "Local Bemidji Weather", "followers_count": 277, "friends_count": 136, "statues_count": 94723 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minnesota, USA", "id": "9807c5c5f7a2c6ce", "name": "Minnesota", "place_type": "admin", "bounding_box": rectangle("-97.239256,43.499362 -89.483385,49.384359") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27007, "countyName": "Beltrami" } }
+{ "create_at": datetime("2015-11-26T10:00:00.000Z"), "id": 669938671190065152, "text": "PC Activities says Happy Thanksgiving to all PC students, families, and staff- grateful for Pirate Pride @ParkCenterSrHS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 227045777 }}, "user": { "id": 1546110546, "name": "Park Center Pirates", "screen_name": "PCSHActivities", "lang": "en", "location": "null", "create_at": date("2013-06-25"), "description": "null", "followers_count": 633, "friends_count": 199, "statues_count": 1181 }, "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("2015-11-26T10:00:00.000Z"), "id": 669938671219412992, "text": "S/o to the grown ups like me that bring nothing to thanksgiving dinner and just wait for the real adults to serve them ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 22215762, "name": "Christiana Divona", "screen_name": "Christianadvona", "lang": "en", "location": "Santa Monica, CA ", "create_at": date("2009-02-27"), "description": "Video Production for @Interscope", "followers_count": 1310, "friends_count": 585, "statues_count": 1848 }, "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("2015-11-26T10:00:00.000Z"), "id": 669938671383142400, "text": "@noukang fb", "in_reply_to_status": -1, "in_reply_to_user": 723244692, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 723244692 }}, "user": { "id": 52987273, "name": "Same Nigga New Me", "screen_name": "FilmoreBands", "lang": "en", "location": "Cincinnati, OH / Atlanta, GA", "create_at": date("2009-07-01"), "description": "Check out my new video! {Link Below} BOOKING: FilmoreMail@Gmail.com | #Cx3 | IG: @FilmoreBands | SC: @Cxxx3", "followers_count": 1360, "friends_count": 468, "statues_count": 32613 }, "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("2015-11-26T10:00:00.000Z"), "id": 669938671533985792, "text": "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": 151284222, "name": "Chris", "screen_name": "ChrisFowler1223", "lang": "en", "location": "null", "create_at": date("2010-06-02"), "description": "17. Senior. Baseball. South Doyle ⚾️", "followers_count": 586, "friends_count": 561, "statues_count": 5469 }, "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("2015-11-26T10:00:00.000Z"), "id": 669938671840202753, "text": "#NaNaNa plays at the end! @PTXofficial @NBCSuperstore https://t.co/7zotGYi4GL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NaNaNa" }}, "user_mentions": {{ 345522684, 2980239724 }}, "user": { "id": 534722212, "name": "Julisa Theodore", "screen_name": "JDTheodore", "lang": "en", "location": "Mobile, AL", "create_at": date("2012-03-23"), "description": "Follower of Christ and Natural Optimist who's also a Zumba Instructor with an Unbreakable Smile!", "followers_count": 311, "friends_count": 1042, "statues_count": 3610 }, "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("2015-11-26T10:00:00.000Z"), "id": 669938672058376193, "text": "@cierrasuwoo happy birthday love❤️", "in_reply_to_status": -1, "in_reply_to_user": 2547372231, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2547372231 }}, "user": { "id": 1413895837, "name": "Bill Gates", "screen_name": "suu_mula", "lang": "en", "location": "money is the motive ", "create_at": date("2013-05-08"), "description": "To fail to plan is a plan to fail", "followers_count": 469, "friends_count": 336, "statues_count": 15498 }, "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("2015-11-26T10:00:00.000Z"), "id": 669938672091942913, "text": "Happy Thanksgiving Twitter! I'm vegetarian this month so I'm counting on all of you to eat some turkey for me! #GobbleGobble", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.73689388,35.78000257"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GobbleGobble" }}, "user": { "id": 60144418, "name": "Jamie Wasley Oliver", "screen_name": "tapper66", "lang": "en", "location": "Cary, NC", "create_at": date("2009-07-25"), "description": "I love living my glorious adventure!", "followers_count": 37, "friends_count": 103, "statues_count": 1697 }, "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("2015-11-26T10:00:00.000Z"), "id": 669938672297336832, "text": "Being that bitch that posts a collage hahahaha ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2297315768, "name": "hannah", "screen_name": "hannahpred", "lang": "en", "location": "null", "create_at": date("2014-01-17"), "description": "whats good? *nicki minaj voice*", "followers_count": 437, "friends_count": 390, "statues_count": 1949 }, "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("2015-11-26T10:00:01.000Z"), "id": 669938672360296448, "text": "Want to work at OhioHealth? We're #hiring in #Columbus, OH! Click for details: https://t.co/l5NQiTeyCe #Healthcare #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.0092803,39.9602601"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Columbus", "Healthcare", "Job", "Jobs", "CareerArc" }}, "user": { "id": 21195970, "name": "Columbus Health Jobs", "screen_name": "tmj_chh_health", "lang": "en", "location": "Columbus, OH", "create_at": date("2009-02-18"), "description": "Follow this account for geo-targeted Healthcare job tweets in Columbus, OH. Need help? Tweet us at @CareerArc!", "followers_count": 525, "friends_count": 327, "statues_count": 581 }, "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("2015-11-26T10:00:01.000Z"), "id": 669938672423280640, "text": "I wish I was closer to my family sometimes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3022011597, "name": "Chano‼️", "screen_name": "FreshBoyChano", "lang": "en", "location": "null", "create_at": date("2015-02-06"), "description": "Humbled Young Man", "followers_count": 521, "friends_count": 440, "statues_count": 1666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Archdale, NC", "id": "0044c2f4709633a2", "name": "Archdale", "place_type": "city", "bounding_box": rectangle("-80.015734,35.83584 -79.885719,35.925449") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37151, "countyName": "Randolph", "cityID": 3701720, "cityName": "Archdale" } }
+{ "create_at": datetime("2015-11-26T10:00:01.000Z"), "id": 669938672490401792, "text": "Gooble gooble gooble ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 356465405, "name": "Brigitte", "screen_name": "bmarie5276", "lang": "en", "location": "null", "create_at": date("2011-08-16"), "description": "10/06/13❤️what a time to be alive 18", "followers_count": 287, "friends_count": 416, "statues_count": 5351 }, "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-26T10:00:01.000Z"), "id": 669938672687443968, "text": "the only people I've fallen asleep on call with were sam and Yeah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1958367018, "name": "ᴄᴀᴘᴛᴀɪɴ.", "screen_name": "MlRAIS", "lang": "en", "location": "Houston, TX", "create_at": date("2013-10-13"), "description": "river • op & idol games! ♡♡♡ @magicofmassacre ♡♡♡ @heartprinced ✧ @grandkingly ✧ @voidrogued ✧", "followers_count": 602, "friends_count": 250, "statues_count": 145523 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:01.000Z"), "id": 669938673044094976, "text": "HAPPY THANKSGIVING!! Thankful for my partnas ��✊���� https://t.co/4VgMSbkRsj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 450276095, "name": "Life of a Savage", "screen_name": "lildezmon8", "lang": "en", "location": "null", "create_at": date("2011-12-29"), "description": "I love football , Alabama A&M Safety #AAMU19. AUG 14 Atlanta,Ga kik: lildezmon8 BATMAN! JAW!", "followers_count": 752, "friends_count": 829, "statues_count": 8594 }, "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("2015-11-26T10:00:01.000Z"), "id": 669938673060704257, "text": "@Kysus_ ����", "in_reply_to_status": 669921171530387456, "in_reply_to_user": 248983683, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 248983683 }}, "user": { "id": 45994780, "name": "pure organic honey.", "screen_name": "kweenilove", "lang": "en", "location": "850", "create_at": date("2009-06-09"), "description": "she's twenty years old and she ain got time to play.", "followers_count": 851, "friends_count": 754, "statues_count": 49134 }, "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("2015-11-26T10:00:01.000Z"), "id": 669938673081708544, "text": "@jordancastii I love you and miss you so much ��", "in_reply_to_status": 669929425937518592, "in_reply_to_user": 3009674844, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3009674844 }}, "user": { "id": 1273696338, "name": "Claire Sickmen", "screen_name": "claire_sickmen", "lang": "en", "location": "null", "create_at": date("2013-03-16"), "description": "•••SFA'19•••", "followers_count": 137, "friends_count": 240, "statues_count": 81 }, "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("2015-11-26T10:00:01.000Z"), "id": 669938673086046208, "text": "@odalis_hurtado @aliciatagg_ Alicia will crack you relax", "in_reply_to_status": 669938430915321857, "in_reply_to_user": 2324962479, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2324962479, 343723638 }}, "user": { "id": 421585789, "name": "Henry Hill™", "screen_name": "Tristan_985", "lang": "en", "location": "Cranston, RI", "create_at": date("2011-11-25"), "description": "CHSE 16'⚡️ #AppleJuice", "followers_count": 409, "friends_count": 427, "statues_count": 9100 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cranston, RI", "id": "4015afc69a8a8506", "name": "Cranston", "place_type": "city", "bounding_box": rectangle("-71.550463,41.730422 -71.377473,41.807454") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4419180, "cityName": "Cranston" } }
+{ "create_at": datetime("2015-11-26T10:00:01.000Z"), "id": 669938673329246208, "text": "11/26@13:00 - Temp 62.1F, WC 62.1F. Wind 3.0mph SE, Gust 8.0mph. Bar 30.753in, Falling. Rain 0.00in. Hum 63%. UV 2.1. SolarRad 443.", "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": 45093 }, "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("2015-11-26T10:00:01.000Z"), "id": 669938673438343168, "text": "Wind 3.4 mph SW. Barometer 30.53 in, Falling. Temperature 66.6 °F. Rain today 0.00 in. Humidity 40%", "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": 81, "friends_count": 17, "statues_count": 300802 }, "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("2015-11-26T10:00:01.000Z"), "id": 669938673455112192, "text": "�������������������� https://t.co/IBuMZ9Z4l3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2196348242, "name": "baby cam.", "screen_name": "CelestialCam", "lang": "en", "location": "CLE | DC", "create_at": date("2013-11-15"), "description": "penthouse cloud | #HowardU19 | peace,love&melanin", "followers_count": 2175, "friends_count": 1752, "statues_count": 43980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Twinsburg, OH", "id": "65aa0cebdf1cd524", "name": "Twinsburg", "place_type": "city", "bounding_box": rectangle("-81.489172,41.276327 -81.400869,41.349821") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3978050, "cityName": "Twinsburg" } }
+{ "create_at": datetime("2015-11-26T10:00:01.000Z"), "id": 669938673492848640, "text": "All those emojis 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": 154238027, "name": "OYE", "screen_name": "MaamieOye_", "lang": "en", "location": "null", "create_at": date("2010-06-10"), "description": "null", "followers_count": 857, "friends_count": 418, "statues_count": 33445 }, "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-26T10:00:01.000Z"), "id": 669938673517924352, "text": "@karennashleyy @_kcassandra9 Aw look at you guys, best friends! ��", "in_reply_to_status": 669935762247458817, "in_reply_to_user": 73850482, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 73850482, 460387527 }}, "user": { "id": 451024168, "name": "Frances K Cazares", "screen_name": "FrannyCazares", "lang": "en", "location": "Texas State University 2016", "create_at": date("2011-12-30"), "description": "Whiskey in a teacup.", "followers_count": 270, "friends_count": 155, "statues_count": 14584 }, "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("2015-11-26T10:00:01.000Z"), "id": 669938673857638400, "text": "I'm thankful for the people who may not always be my friends but they are forever family��❤️ #missyouguys https://t.co/LHoBVzPRWT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "missyouguys" }}, "user": { "id": 1343025007, "name": "Phia Mia♛", "screen_name": "SophiaTerhaar", "lang": "en", "location": "Fernley, NV", "create_at": date("2013-04-10"), "description": "I'm not a princess I don't need saving, I'm a Queen I got this shit handled✿ 2017", "followers_count": 677, "friends_count": 450, "statues_count": 5917 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fernley, NV", "id": "397b264da04dcb2d", "name": "Fernley", "place_type": "city", "bounding_box": rectangle("-119.284524,39.571978 -119.135682,39.628996") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32019, "countyName": "Lyon", "cityID": 3224900, "cityName": "Fernley" } }
+{ "create_at": datetime("2015-11-26T10:00:01.000Z"), "id": 669938673870336004, "text": "My shortyyyy https://t.co/YVcLKyzLBn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3073190980, "name": "b", "screen_name": "brittnosko", "lang": "en", "location": "null", "create_at": date("2015-03-05"), "description": "21 • happiness is key ☻ •", "followers_count": 783, "friends_count": 217, "statues_count": 914 }, "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("2015-11-26T10:00:01.000Z"), "id": 669938673916338176, "text": "At the park and my phones going to die ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2949179209, "name": "Michel", "screen_name": "Micheeel16_", "lang": "en", "location": "null", "create_at": date("2014-12-28"), "description": "null", "followers_count": 244, "friends_count": 170, "statues_count": 9099 }, "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-26T10:00:01.000Z"), "id": 669938674004553729, "text": "You always got that one uncle that makes his to go plates and it be enough to have lunch for a whole 3 weeks", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 928838490, "name": "Joshua Dunn", "screen_name": "JoshDunn21", "lang": "en", "location": "Lake Wales, Winter Haven", "create_at": date("2012-11-05"), "description": "My circle so small I be talking to myself at times. Live by 1 Corinthians 16:13 Trying to make it out! Soon to be at USF #Loading....", "followers_count": 1008, "friends_count": 532, "statues_count": 3261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Wales, FL", "id": "5a3390f9475a3354", "name": "Lake Wales", "place_type": "city", "bounding_box": rectangle("-81.626231,27.871956 -81.549726,27.969981") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12105, "countyName": "Polk", "cityID": 1238950, "cityName": "Lake Wales" } }
+{ "create_at": datetime("2015-11-26T10:00:01.000Z"), "id": 669938674226696192, "text": "Go Lions!!!! #PHIvsDET", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PHIvsDET" }}, "user": { "id": 25377388, "name": "Nicole Cordero", "screen_name": "Nicole_CB", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-03-19"), "description": "Mom, fabulous, Hispanic Marketing & Media professional, occasional cupcake baker. If found, please return to the wine aisle. Bilingual tweets. \nMex-Canada-LA", "followers_count": 602, "friends_count": 520, "statues_count": 5433 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:01.000Z"), "id": 669938674507890688, "text": "ALSO thankful to be accepted for GlobeMed's GROW internship this summer. Going to be spending 5 weeks in Uganda working with @Setherfree", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 815975028 }}, "user": { "id": 40967435, "name": "still a chano", "screen_name": "GracePetr", "lang": "en", "location": "Washington, DC", "create_at": date("2009-05-18"), "description": "from 'consin to congress", "followers_count": 375, "friends_count": 268, "statues_count": 7003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wauwatosa, WI", "id": "d378ed20e9e8e270", "name": "Wauwatosa", "place_type": "city", "bounding_box": rectangle("-88.067098,43.030534 -87.983987,43.104541") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5584675, "cityName": "Wauwatosa" } }
+{ "create_at": datetime("2015-11-26T10:00:01.000Z"), "id": 669938675124273152, "text": "The Eagles going to catch an L", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3406497471, "name": "Mr. Vinegar Stroke", "screen_name": "trevahhhNstuff", "lang": "en", "location": "Body 660, Soul 757", "create_at": date("2015-08-06"), "description": "Born again Savage from the burbs that had a golden spoon that is now used to eat ass", "followers_count": 206, "friends_count": 189, "statues_count": 13445 }, "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("2015-11-26T10:00:01.000Z"), "id": 669938675224981504, "text": "Thankful for that direct deposit ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 462314840, "name": "Big Kahuna", "screen_name": "Ohh_Mauraah", "lang": "en", "location": "Santa Fe Springs, CA", "create_at": date("2012-01-12"), "description": "| -.Rest Easy Pops.- |", "followers_count": 522, "friends_count": 237, "statues_count": 39276 }, "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("2015-11-26T10:00:01.000Z"), "id": 669938675485163520, "text": "I love you so much�� https://t.co/UoZv29119D", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 572044861, "name": "ōłîvë", "screen_name": "Oliviabucco", "lang": "en", "location": "Probably at Smoothie King", "create_at": date("2012-05-05"), "description": "Lady Elite & Monarchs // everything happens for a reason", "followers_count": 924, "friends_count": 351, "statues_count": 5993 }, "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("2015-11-26T10:00:01.000Z"), "id": 669938675602485248, "text": "I'n taking a nap for the rest 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": 1934460830, "name": "Kristian Claus", "screen_name": "KClausWBOY", "lang": "en", "location": "North-central West Virginia", "create_at": date("2013-10-04"), "description": "WBOY: ABC NBC Stormtracker12 meteorologist via Embry-Riddle. Alderson Broaddus Adjunct. Licensed Pilot. Fox Chapel alum. RT ≠ endorsement.", "followers_count": 2494, "friends_count": 2234, "statues_count": 16052 }, "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("2015-11-26T10:00:01.000Z"), "id": 669938675711512576, "text": "You have 0 followers... lol https://t.co/6JbsFwouN3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1722999780, "name": "Ashton Hansen", "screen_name": "bl3r_mgmt", "lang": "en", "location": "null", "create_at": date("2013-09-02"), "description": "Manager for @bl3rmusic Artist management at @latenation Owner at BL3R Clothing. Contact: ah@latenation.com", "followers_count": 519, "friends_count": 193, "statues_count": 11462 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Snohomish, WA", "id": "c7a6ed0142a419ec", "name": "Snohomish", "place_type": "city", "bounding_box": rectangle("-122.122469,47.902213 -122.033966,47.960067") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5365170, "cityName": "Snohomish" } }
+{ "create_at": datetime("2015-11-26T10:00:01.000Z"), "id": 669938676059779072, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 174674074, "name": "Lima.India.Zulu", "screen_name": "MiChiamoLizzie", "lang": "en", "location": "null", "create_at": date("2010-08-04"), "description": "you are a universe in a universe of universes.", "followers_count": 1121, "friends_count": 806, "statues_count": 195961 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coral Hills, MD", "id": "943aaad83adbb87b", "name": "Coral Hills", "place_type": "city", "bounding_box": rectangle("-76.946773,38.862864 -76.903197,38.88339") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2419825, "cityName": "Coral Hills" } }
+{ "create_at": datetime("2015-11-26T10:00:01.000Z"), "id": 669938676101681153, "text": "https://t.co/qlQdk1l0B7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 629811076, "name": "BuffaloSoldier", "screen_name": "LoveMy_Kickss", "lang": "en", "location": "BuffaloNY✈️AtlantaGA✈️D〽️V", "create_at": date("2012-07-07"), "description": "idgaf. 2jobShawty Makin Bread. 19, Mindset on 25. One of The Real Ones Left. Cool Ahh Nigga Fr, if i FWU i FWU. S/O to my few friends. Bitches Love My Hair. 420", "followers_count": 83, "friends_count": 246, "statues_count": 463 }, "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("2015-11-26T10:00:01.000Z"), "id": 669938676181237760, "text": "I need this kind of hug https://t.co/gZcz3qr9im", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 167459782, "name": "Shells ૐ", "screen_name": "shaayrose", "lang": "en", "location": "null", "create_at": date("2010-07-16"), "description": "Disney isn't a mood but a lifestyle ✨", "followers_count": 519, "friends_count": 1005, "statues_count": 9358 }, "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-26T10:00:01.000Z"), "id": 669938676260974592, "text": "@chhi_nno you're too late on that sale buddy", "in_reply_to_status": 669938600012677121, "in_reply_to_user": 363788865, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 363788865 }}, "user": { "id": 417685562, "name": "Kylo Ren", "screen_name": "BryantChapel", "lang": "en", "location": "null", "create_at": date("2011-11-20"), "description": "I ain't got no manners for no sluts I'ma put my thumb in her butt", "followers_count": 913, "friends_count": 441, "statues_count": 119828 }, "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("2015-11-26T10:00:01.000Z"), "id": 669938676277735424, "text": "Interested in a #Hospitality #job near ? This could be a great fit: https://t.co/GAr2lVURWE #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.9559555,34.1715331"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 88003804, "name": "TMJ-CA HRTA Jobs", "screen_name": "tmj_ca_hrta", "lang": "en", "location": "California", "create_at": date("2009-11-06"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 410, "friends_count": 297, "statues_count": 10856 }, "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("2015-11-26T10:00:01.000Z"), "id": 669938676332392449, "text": "@tristankurtz https://t.co/FgqEaNoM27", "in_reply_to_status": 669936695022874624, "in_reply_to_user": 36108162, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 118881414 }}, "user": { "id": 36108162, "name": "Vardaan Arora", "screen_name": "vardaanarora", "lang": "en", "location": "New York City / Delhi", "create_at": date("2009-04-28"), "description": "professional candle vlogger.", "followers_count": 887, "friends_count": 218, "statues_count": 3509 }, "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("2015-11-26T10:00:01.000Z"), "id": 669938676399386625, "text": "https://t.co/Cl2Mez8ect", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 618532174, "name": "Ryann Kolkmeier☮☀️", "screen_name": "Ryann3laine", "lang": "en", "location": "HTX", "create_at": date("2012-06-25"), "description": "#UST19 BioChem/Premed TJM❤️", "followers_count": 749, "friends_count": 1495, "statues_count": 7775 }, "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("2015-11-26T10:00:02.000Z"), "id": 669938676839882752, "text": "@alexamendezyy �������� vamos a casar turkeys", "in_reply_to_status": 669938558237544448, "in_reply_to_user": 1155330380, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 1155330380 }}, "user": { "id": 3045696004, "name": "Victoria Secret ♌", "screen_name": "_vikayyyy", "lang": "en", "location": "Chicago, IL", "create_at": date("2015-02-19"), "description": "Jalisco and Jaritos ☀️", "followers_count": 254, "friends_count": 323, "statues_count": 2039 }, "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-26T10:00:02.000Z"), "id": 669938677045440513, "text": "@HollyGroff love you������", "in_reply_to_status": 669912930754932737, "in_reply_to_user": 422906691, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 422906691 }}, "user": { "id": 1170840781, "name": "lex", "screen_name": "Leximace2", "lang": "en", "location": "Macedonia, OH", "create_at": date("2013-02-11"), "description": "nordonia", "followers_count": 500, "friends_count": 740, "statues_count": 4252 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Macedonia, OH", "id": "9b8a8b5437a94787", "name": "Macedonia", "place_type": "city", "bounding_box": rectangle("-81.530955,41.276868 -81.463345,41.350133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3945976, "cityName": "Macedonia" } }
+{ "create_at": datetime("2015-11-26T10:00:02.000Z"), "id": 669938677087412224, "text": "In my bag", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 318805217, "name": "TPrice", "screen_name": "T_bojanglex3", "lang": "en", "location": "null", "create_at": date("2011-06-16"), "description": "|A moment of pain is worth a lifetime of glory|♍️|Insta: @tprice_32|•nova•|shepU|", "followers_count": 1886, "friends_count": 1657, "statues_count": 47759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Centreville, VA", "id": "ffcc53c4a4e7a620", "name": "Centreville", "place_type": "city", "bounding_box": rectangle("-77.479597,38.802143 -77.397429,38.880183") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5114440, "cityName": "Centreville" } }
+{ "create_at": datetime("2015-11-26T10:00:02.000Z"), "id": 669938677288730624, "text": "waiting for one thanksgiving text����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2314281719, "name": "chanel", "screen_name": "nelxo_", "lang": "en", "location": "null", "create_at": date("2014-01-30"), "description": "Individual.", "followers_count": 313, "friends_count": 269, "statues_count": 4602 }, "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("2015-11-26T10:00:02.000Z"), "id": 669938677305348097, "text": "did u find a knife yet https://t.co/TtdtHanwo8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1007827885, "name": "vic", "screen_name": "victoriaSNorris", "lang": "en", "location": "be happy", "create_at": date("2012-12-12"), "description": "no, im not lucky im blessed.//", "followers_count": 799, "friends_count": 1000, "statues_count": 11614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boiling Springs, NC", "id": "51cf29f9ebef7949", "name": "Boiling Springs", "place_type": "city", "bounding_box": rectangle("-81.684263,35.232394 -81.622031,35.277061") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37045, "countyName": "Cleveland", "cityID": 3706800, "cityName": "Boiling Springs" } }
+{ "create_at": datetime("2015-11-26T10:00:02.000Z"), "id": 669938677540241408, "text": "@thatsojackie yuh and got the perfect lighting on ur face <3", "in_reply_to_status": 669938541242159104, "in_reply_to_user": 1638451650, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1638451650 }}, "user": { "id": 2703988515, "name": "s!ndy", "screen_name": "xsindy", "lang": "en", "location": "null", "create_at": date("2014-07-12"), "description": "but what if", "followers_count": 208, "friends_count": 112, "statues_count": 7439 }, "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("2015-11-26T10:00:02.000Z"), "id": 669938677565403138, "text": "regrann from @bodybakerytan - Black Friday starts NOW!! �� Enjoy 30% Off all tanning packages and… https://t.co/UlUnn3pfiZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.5330237,34.16881773"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 619436801 }}, "user": { "id": 17154147, "name": "HerbTheEntSpec", "screen_name": "herbtheentspec", "lang": "en", "location": "Hollywood, CA", "create_at": date("2008-11-04"), "description": "@WeFeedTheHungry http://MyFriendsHouseLA.org | Events & Promotions Specialist A High-Yielding Celebrity Commodity", "followers_count": 7309, "friends_count": 5399, "statues_count": 139175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:02.000Z"), "id": 669938677636820996, "text": "Tuning in a Lil late but I love your dress @CariChampion on @SportsCenter . Hair is on fleek����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0161053,41.2197638"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25471833, 26257166 }}, "user": { "id": 1673424901, "name": "Kariemah Shah", "screen_name": "buttadlyte", "lang": "en", "location": "NYC", "create_at": date("2013-08-15"), "description": "Mom, sassy tv/film payroll acct, sports fanatic & lover of all things positive #HamptonUAlum", "followers_count": 137, "friends_count": 193, "statues_count": 2067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thiells, NY", "id": "c4cf536938177f0c", "name": "Thiells", "place_type": "city", "bounding_box": rectangle("-74.051698,41.19411 -73.996007,41.220351") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36087, "countyName": "Rockland", "cityID": 3673583, "cityName": "Thiells" } }
+{ "create_at": datetime("2015-11-26T10:00:02.000Z"), "id": 669938677653475328, "text": "Peep the lil flakes of seasoning lmao https://t.co/xk6ypS7MrR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 232742846, "name": "Sage of the South", "screen_name": "VakoStrange", "lang": "en", "location": "NOLA-HOU", "create_at": date("2010-12-31"), "description": "null", "followers_count": 3289, "friends_count": 509, "statues_count": 128283 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:02.000Z"), "id": 669938677796216837, "text": "@andy_kastrati ilym", "in_reply_to_status": 669938569528627203, "in_reply_to_user": 2874421015, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2874421015 }}, "user": { "id": 600515017, "name": "nïkks", "screen_name": "nicolexvasquez", "lang": "en", "location": "pshs track", "create_at": date("2012-06-05"), "description": "not pregnant just eatin good #SeanStrong #EmptyTheTanks", "followers_count": 1784, "friends_count": 856, "statues_count": 38959 }, "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("2015-11-26T10:00:02.000Z"), "id": 669938677921923073, "text": "75.8F (Feels: 75.8F) - Humidity: 74% - Wind: 12.1mph E - Gust: 18.3mph - Pressure: 1020.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": 215564 }, "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("2015-11-26T10:00:02.000Z"), "id": 669938677955432448, "text": "I'm thankful for all the plugs out there who have come through", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2223652902, "name": "⠀⠀", "screen_name": "N0tSpencer", "lang": "en", "location": "dre Menschen", "create_at": date("2013-11-30"), "description": "Rock n Roll", "followers_count": 243, "friends_count": 160, "statues_count": 6960 }, "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("2015-11-26T10:00:02.000Z"), "id": 669938677972205568, "text": "Temp: 72.1°F Wind:5.5mph Pressure: 30.404hpa 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": 22, "statues_count": 52113 }, "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("2015-11-26T10:00:02.000Z"), "id": 669938677980766208, "text": "@JohnZwerenz Thanks for the follow ��", "in_reply_to_status": -1, "in_reply_to_user": 349163118, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 349163118 }}, "user": { "id": 3492785296, "name": "John c Laslow", "screen_name": "kronk64", "lang": "en", "location": "Lexington, SC", "create_at": date("2015-08-30"), "description": "Originally from Penn Hills,PA. Like Steelers,Pens, zombies,cats and people with a good sense of humor. Don't take life so seriously!", "followers_count": 549, "friends_count": 889, "statues_count": 6810 }, "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("2015-11-26T10:00:02.000Z"), "id": 669938678077063168, "text": "My sister doesn't tell me if she's having a boy or girl ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 944845622, "name": "esme ✨", "screen_name": "itsesmeralda_", "lang": "en", "location": "null", "create_at": date("2012-11-12"), "description": "null", "followers_count": 1320, "friends_count": 677, "statues_count": 12514 }, "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.733319,39.625005") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3271600, "cityName": "Sun Valley" } }
+{ "create_at": datetime("2015-11-26T10:00:02.000Z"), "id": 669938678194642944, "text": "Kart amanhã, trem bala", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 264339986, "name": "Igor", "screen_name": "talksbieberr", "lang": "pt", "location": "Florida, USA", "create_at": date("2011-03-11"), "description": "The lights go on, the music dies, but you don't see me standing here, I just came to say goodbye", "followers_count": 840, "friends_count": 455, "statues_count": 45995 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clermont, FL", "id": "01931c017c8730a9", "name": "Clermont", "place_type": "city", "bounding_box": rectangle("-81.784442,28.492504 -81.682081,28.583835") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12069, "countyName": "Lake", "cityID": 1212925, "cityName": "Clermont" } }
+{ "create_at": datetime("2015-11-26T10:00:02.000Z"), "id": 669938678257614848, "text": "tbt https://t.co/BpCP1Fp0IF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 23689629, "name": "Marlana", "screen_name": "Joy61297", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2009-03-10"), "description": "CL @teencher ☾✩", "followers_count": 6565, "friends_count": 238, "statues_count": 100238 }, "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-26T10:00:02.000Z"), "id": 669938678266003457, "text": "Truly thankful for my beard... Love you buddy #BeardGang", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BeardGang" }}, "user": { "id": 1183004082, "name": "Nick", "screen_name": "yung__boii13", "lang": "en", "location": "Mandingo Tribe ", "create_at": date("2013-02-15"), "description": "#BeardGang", "followers_count": 685, "friends_count": 478, "statues_count": 31984 }, "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("2015-11-26T10:00:02.000Z"), "id": 669938678404370432, "text": "Wind 10.0 mph SE. Barometer 30.231 in, Falling. Temperature 69.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": 116, "friends_count": 0, "statues_count": 109148 }, "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("2015-11-26T10:00:02.000Z"), "id": 669938678609743873, "text": "I also want go to the movies ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2719804592, "name": "Queen.", "screen_name": "_asiiaaa", "lang": "en", "location": "New Orleans, LA", "create_at": date("2014-08-09"), "description": "G..❤️", "followers_count": 4651, "friends_count": 3489, "statues_count": 40404 }, "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("2015-11-26T10:00:02.000Z"), "id": 669938679088078848, "text": "@caplannfl @LRiddickESPN any chance another team gets a home game on thanksgiving beside lions/cowboys ever?!", "in_reply_to_status": -1, "in_reply_to_user": 15729027, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15729027, 1577522323 }}, "user": { "id": 344122149, "name": "joey", "screen_name": "j_paulus16", "lang": "en", "location": "Cleveland, OH", "create_at": date("2011-07-28"), "description": "*God*-*Family*-*Sports*-*BBQ*-*Boating* #Indians #Rangers #Cavs #Blazers #Suns #Browns #Cowboys #Panthers #Buckeyes #AZwildcats - *Pisces*", "followers_count": 162, "friends_count": 1183, "statues_count": 10317 }, "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("2015-11-26T10:00:02.000Z"), "id": 669938679188713472, "text": "Cousins ������ https://t.co/vCBat574lH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25756176, "name": "1999 B Spears", "screen_name": "maryhotter", "lang": "en", "location": "Connecticut", "create_at": date("2009-03-21"), "description": "24, dogs flock to me, comedy geek, 90s r&b queen, Italian ass bitch", "followers_count": 188, "friends_count": 384, "statues_count": 22641 }, "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("2015-11-26T10:00:02.000Z"), "id": 669938679348097026, "text": "Wind 3.0 mph SW. Barometer 30.501 in, Falling. Temperature 64.3 °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": 39, "friends_count": 25, "statues_count": 15677 }, "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("2015-11-26T10:00:02.000Z"), "id": 669938679364775937, "text": "Thankful for @Nukdabomb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1057255670 }}, "user": { "id": 153144782, "name": "Kristen Mori", "screen_name": "KrizzleM", "lang": "en", "location": "West Lafayette, IN", "create_at": date("2010-06-07"), "description": "got a foot in the gutter, a foot in the grave. #PatsNation #BoilerUp", "followers_count": 475, "friends_count": 829, "statues_count": 30098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rockton, IL", "id": "749015dab52e7830", "name": "Rockton", "place_type": "city", "bounding_box": rectangle("-89.089475,42.412296 -89.018477,42.473732") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17201, "countyName": "Winnebago", "cityID": 1765156, "cityName": "Rockton" } }
+{ "create_at": datetime("2015-11-26T10:00:02.000Z"), "id": 669938679792709632, "text": "sometimes I hate myself����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "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": 170, "friends_count": 517, "statues_count": 4880 }, "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("2015-11-26T10:00:02.000Z"), "id": 669938679884943361, "text": "If you go to Loyola hold this #Easily�� https://t.co/1TWFIPdSqS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Easily" }}, "user": { "id": 1688096862, "name": "Kevin", "screen_name": "Mo_ses_", "lang": "en", "location": "Baltimore, MD", "create_at": date("2013-08-21"), "description": "CHC 16 WKU 20", "followers_count": 740, "friends_count": 564, "statues_count": 21189 }, "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-26T10:00:02.000Z"), "id": 669938680019214337, "text": "@dahl_abby awe this is the cutest���� thanks for everything love you so much girl��", "in_reply_to_status": 669937207214477312, "in_reply_to_user": 1072179872, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1072179872 }}, "user": { "id": 520451781, "name": "krystal", "screen_name": "krystalgrudzien", "lang": "en", "location": "null", "create_at": date("2012-03-10"), "description": "Avon 2016 • Snapchat: krysstal • instagram:krystalgrudzien", "followers_count": 598, "friends_count": 556, "statues_count": 2855 }, "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("2015-11-26T10:00:02.000Z"), "id": 669938680035803138, "text": "Thankful for all my friends and family and thank y'all for always being there for me 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": 304706933, "name": "Araceli", "screen_name": "aracelioviedoo", "lang": "en", "location": "null", "create_at": date("2011-05-24"), "description": "Dawson", "followers_count": 686, "friends_count": 467, "statues_count": 12183 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brookside Village, TX", "id": "468172bc6033fb2c", "name": "Brookside Village", "place_type": "city", "bounding_box": rectangle("-95.337364,29.579893 -95.295007,29.599096") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4810648, "cityName": "Brookside Village" } }
+{ "create_at": datetime("2015-11-26T10:00:02.000Z"), "id": 669938680618811392, "text": "*prayer at the table today*\n\nI'm thankful for my brothers, my son Yam's, 6th street, whoever invented twerking and the bitches 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": 628107235, "name": "Tony d", "screen_name": "ohnoitsme_tonyd", "lang": "en", "location": "TEXXXAS", "create_at": date("2012-07-05"), "description": "Millionaireby25 #BLESSED SA⏩ATX", "followers_count": 499, "friends_count": 360, "statues_count": 19928 }, "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-26T10:00:03.000Z"), "id": 669938680828555264, "text": "10 day�� https://t.co/48KIUyCXep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2272298563, "name": "Rubes", "screen_name": "Rruubbyyyyy", "lang": "en", "location": "LBC", "create_at": date("2014-01-01"), "description": "Poly softball⚾️", "followers_count": 629, "friends_count": 744, "statues_count": 16318 }, "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("2015-11-26T10:00:03.000Z"), "id": 669938680979558401, "text": "Rich and makonnen??", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user": { "id": 369839722, "name": "Congo Dandy", "screen_name": "TRAWW__3", "lang": "en", "location": "Los Angeles, California", "create_at": date("2011-09-07"), "description": "West$ide Get The Money IG: traww__3", "followers_count": 484, "friends_count": 452, "statues_count": 20076 }, "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("2015-11-26T10:00:03.000Z"), "id": 669938680979582976, "text": "@KTtheYG you see the one with Cam?", "in_reply_to_status": 669938502297980933, "in_reply_to_user": 1918509481, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1918509481 }}, "user": { "id": 2453470740, "name": "Ocasio", "screen_name": "Jocasio_21", "lang": "en", "location": "null", "create_at": date("2014-04-19"), "description": "Born in New York moved to Dallas...100% Puerto Rican! Money motivated. 19 years young.... Living like Larry", "followers_count": 311, "friends_count": 289, "statues_count": 3767 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Elm, TX", "id": "bd331d141f66eead", "name": "Little Elm", "place_type": "city", "bounding_box": rectangle("-96.983818,33.140885 -96.881861,33.199847") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4843012, "cityName": "Little Elm" } }
+{ "create_at": datetime("2015-11-26T10:00:03.000Z"), "id": 669938681042472960, "text": "success is eminent https://t.co/s1dO7VgVid", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2591529152, "name": "anthony wright", "screen_name": "wrightanthony3", "lang": "en", "location": "Texas, USA", "create_at": date("2014-06-27"), "description": "4/30/15", "followers_count": 190, "friends_count": 375, "statues_count": 1487 }, "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("2015-11-26T10:00:03.000Z"), "id": 669938681143169024, "text": "Injury accident in #Austin on William Cannon Dr at N Bluff Dr #ATXtraffic https://t.co/8laei8AmfZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.77157,30.19094"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Austin", "ATXtraffic" }}, "user": { "id": 244281354, "name": "Total Traffic Austin", "screen_name": "TotalTrafficAUS", "lang": "en", "location": "Austin, TX", "create_at": date("2011-01-28"), "description": "Your up to the minute #traffic updates! Stuck in Traffic? Tell us all about it! Tweet us or call the Total Traffic Tipline 684-7389 #Austin #KeepAustinWeird", "followers_count": 2629, "friends_count": 161, "statues_count": 62352 }, "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-26T10:00:03.000Z"), "id": 669938681197674496, "text": "Wind 6.0 mph ESE. Barometer 30.170 in, Steady. Temperature 55.8 °F. Rain today 0.00 in. Humidity 68%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-106.69694444,34.80694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14656738, "name": "Andrew Madden", "screen_name": "AbqDrew", "lang": "en", "location": "Los Lunas, New Mexico", "create_at": date("2008-05-04"), "description": "31 year old atheist conservative living in Albuquerque, New Mexico. KCCO", "followers_count": 173, "friends_count": 186, "statues_count": 4886 }, "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("2015-11-26T10:00:03.000Z"), "id": 669938681411563520, "text": "Apologize for my stupid cousin's tweets", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2552977110, "name": "preeti:)))", "screen_name": "stassieeann", "lang": "en", "location": "null", "create_at": date("2014-06-07"), "description": "null", "followers_count": 200, "friends_count": 59, "statues_count": 4879 }, "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("2015-11-26T10:00:03.000Z"), "id": 669938681520590849, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4302772944, "name": "Terry Dupuy", "screen_name": "TerryDupuy3", "lang": "en", "location": "Rowlett, TX", "create_at": date("2015-11-20"), "description": "New Twitter!", "followers_count": 84, "friends_count": 52, "statues_count": 17 }, "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("2015-11-26T10:00:03.000Z"), "id": 669938681612861440, "text": "@jarpad hahah that's hilarious! Happy thanksgiving to you & your family Jared!", "in_reply_to_status": 669938145329324032, "in_reply_to_user": 310687757, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 310687757 }}, "user": { "id": 114830693, "name": "Anna", "screen_name": "mistakensinger", "lang": "en", "location": "Arizona", "create_at": date("2010-02-16"), "description": "#akf always keep fighting", "followers_count": 309, "friends_count": 375, "statues_count": 12596 }, "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-26T10:00:03.000Z"), "id": 669938681940054017, "text": "@FunkyreFresh @TheChimpyMan @mumbly_joe It's obvious that Schrødinger needed a dog. (Typing on a portugese keyboard is so fun!)", "in_reply_to_status": 669922341867098112, "in_reply_to_user": 272759604, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 272759604, 229556037, 18741944 }}, "user": { "id": 16166534, "name": "BoatOfVelociraptors", "screen_name": "bagofmice", "lang": "en", "location": "In the Seattle", "create_at": date("2008-09-06"), "description": "Non-cents. My pennies are made of aluminum.", "followers_count": 184, "friends_count": 238, "statues_count": 10377 }, "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("2015-11-26T10:00:03.000Z"), "id": 669938681944236032, "text": "Good morning, Waimea, HI. #UnitedStates https://t.co/tAHmSKhbko https://t.co/mtUOy5yR9U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-159.6708374,21.95861053"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "UnitedStates" }}, "user": { "id": 17111948, "name": "BigZ", "screen_name": "ioDracy", "lang": "zh-cn", "location": "HangZhou", "create_at": date("2008-11-02"), "description": "@Hangzhou China,Android user,Google fan,Geeker,Blogger", "followers_count": 211, "friends_count": 223, "statues_count": 3790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waimea, HI", "id": "406f9f4e07f09285", "name": "Waimea", "place_type": "city", "bounding_box": rectangle("-159.692213,21.951609 -159.655496,21.98181") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15007, "countyName": "Kauai", "cityID": 1578800, "cityName": "Waimea" } }
+{ "create_at": datetime("2015-11-26T10:00:03.000Z"), "id": 669938681977802753, "text": "Just posted a photo @ Pasapoga https://t.co/5DZP3xZ5u8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.44699088,25.70029248"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 848474808, "name": "Carlos Godoy", "screen_name": "cgodoy2930", "lang": "en", "location": "Miami Florida", "create_at": date("2012-09-26"), "description": "null", "followers_count": 216, "friends_count": 1207, "statues_count": 17146 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kendall West, FL", "id": "007ba9db4c5baedf", "name": "Kendall West", "place_type": "city", "bounding_box": rectangle("-80.4965,25.683801 -80.4299,25.728697") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1236121, "cityName": "Kendall West" } }
+{ "create_at": datetime("2015-11-26T10:00:03.000Z"), "id": 669938682023952384, "text": "Late night tattoo 3:30am #ChefLife #KitchenLife https://t.co/dxRmQ68UZ7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ChefLife", "KitchenLife" }}, "user": { "id": 2749937622, "name": "Osmar", "screen_name": "orobles32", "lang": "en", "location": "null", "create_at": date("2014-08-20"), "description": "null", "followers_count": 20, "friends_count": 84, "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("2015-11-26T10:00:03.000Z"), "id": 669938682057617408, "text": "Alicias in VT all day w no service like what the fuck 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": 1702664851, "name": "bacne and menthols", "screen_name": "br0tep", "lang": "en", "location": "64 zoo lane", "create_at": date("2013-08-26"), "description": "writing hits. taking shits.", "followers_count": 701, "friends_count": 85, "statues_count": 80947 }, "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("2015-11-26T10:00:03.000Z"), "id": 669938682166538240, "text": "@Z_Jenkins3 @maddyriggs22 this just seems so inappropriate to me", "in_reply_to_status": 669897936336130048, "in_reply_to_user": 545558009, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 545558009, 1337987089 }}, "user": { "id": 2677763409, "name": "Belote, Gracie", "screen_name": "gracieebel", "lang": "en", "location": "Edmond, USA", "create_at": date("2014-07-05"), "description": "we Oklahoma bound", "followers_count": 1570, "friends_count": 1289, "statues_count": 16765 }, "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("2015-11-26T10:00:03.000Z"), "id": 669938682242211840, "text": "Temp: 76.5°F | Humidity: 68% | Wind: N @ 0.0 mph | Barometer: 30.33 in | Dewpoint: 65.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": 24, "friends_count": 1, "statues_count": 163321 }, "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("2015-11-26T10:00:03.000Z"), "id": 669938682300764161, "text": "I'll be face timing my aunt and mom probably 7 times through out 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": 539339763, "name": "Yessi ➡️ MarshMello", "screen_name": "Yessi_Vanessa", "lang": "en", "location": "lost in colorado #CSU-P", "create_at": date("2012-03-28"), "description": "INSTAGRAM: Yessi_Vanessa FACEBOOK: 'Yessi Vanessa Estrada #EDM Junior #CSUpueblo #gymlife", "followers_count": 1456, "friends_count": 1222, "statues_count": 12554 }, "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("2015-11-26T10:00:03.000Z"), "id": 669938682481270785, "text": "T H E O ⚡️��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "vi", "is_retweet": false, "user": { "id": 355293127, "name": "Brett Hull", "screen_name": "bmhull_15_", "lang": "en", "location": "Oswego", "create_at": date("2011-08-14"), "description": "⚾️", "followers_count": 248, "friends_count": 437, "statues_count": 5925 }, "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("2015-11-26T10:00:03.000Z"), "id": 669938682690797568, "text": "why do I have to be sick today of all days", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 356619366, "name": "LO.♡", "screen_name": "L_moneyy_", "lang": "en", "location": "probs wherever rach is ❣", "create_at": date("2011-08-16"), "description": "null", "followers_count": 785, "friends_count": 636, "statues_count": 40773 }, "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("2015-11-26T10:00:03.000Z"), "id": 669938683261362176, "text": "@Therealkiss #Top5DeadOrAlive ��������", "in_reply_to_status": -1, "in_reply_to_user": 20518710, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Top5DeadOrAlive" }}, "user_mentions": {{ 20518710 }}, "user": { "id": 35343489, "name": "LEVEL 13", "screen_name": "LEVEL_13", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2009-04-25"), "description": "Philly beatmaker. Produced for Kool G Rap, Method Man, Chino XL, Ras Kass, Diabolic, Madchild, DJ Kay Slay. Business Inquiries Hit: beats@level13productions.net", "followers_count": 3082, "friends_count": 657, "statues_count": 38315 }, "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-26T10:00:03.000Z"), "id": 669938683303354369, "text": "Well then", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2760782039, "name": "Cora Mason", "screen_name": "c_masonn", "lang": "en", "location": "null", "create_at": date("2014-09-03"), "description": "#GoNavy", "followers_count": 463, "friends_count": 450, "statues_count": 11966 }, "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("2015-11-26T10:00:03.000Z"), "id": 669938683311603712, "text": "I gotta work 7-3 tomorrow so I'm not gone drink 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": 189596354, "name": "T W E L V E 18♐️❤️", "screen_name": "AvenueB_", "lang": "en", "location": "New Orleans ✈️ Houston ", "create_at": date("2010-09-11"), "description": "You Don't Know Me!!! You Know What I Allow You To See - AveB", "followers_count": 1348, "friends_count": 1073, "statues_count": 82815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:03.000Z"), "id": 669938683324342272, "text": "I am thankful that God took away what I thought I needed and blessed me with someone better ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2524052812, "name": "Ashley Cormier", "screen_name": "Ashleybcormier", "lang": "en", "location": "null", "create_at": date("2014-05-02"), "description": "instagram @ashley_cormier", "followers_count": 267, "friends_count": 26, "statues_count": 689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Toccoa, GA", "id": "90e0916f1f3e69dc", "name": "Toccoa", "place_type": "city", "bounding_box": rectangle("-83.359855,34.53357 -83.282888,34.600613") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13257, "countyName": "Stephens", "cityID": 1376756, "cityName": "Toccoa" } }
+{ "create_at": datetime("2015-11-26T10:00:03.000Z"), "id": 669938683504562180, "text": "His smile melts my heart https://t.co/ryex5nBJoB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1051762988, "name": "A m e r i c a .", "screen_name": "Ameerica_xo", "lang": "en", "location": "Dallas, TX", "create_at": date("2012-12-31"), "description": "xo", "followers_count": 462, "friends_count": 421, "statues_count": 5162 }, "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("2015-11-26T10:00:03.000Z"), "id": 669938683819114498, "text": "Tiff's mom prolly the best cook out there.. like no bs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 282280703, "name": "Ricardo", "screen_name": "Corporate_Rich", "lang": "en", "location": "null", "create_at": date("2011-04-14"), "description": "Cali raised me. Xavier taught me. Alpha made me... Ohio | Cali", "followers_count": 807, "friends_count": 546, "statues_count": 62935 }, "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("2015-11-26T10:00:03.000Z"), "id": 669938684003819520, "text": "@jake_huston14 currently wearing jeans.......��", "in_reply_to_status": 669930568256237568, "in_reply_to_user": 1651963652, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1651963652 }}, "user": { "id": 493378014, "name": "kenzie", "screen_name": "mckenzie_doriot", "lang": "en", "location": "null", "create_at": date("2012-02-15"), "description": "fire up chips 〰 always daydreaming 〰 only person to laugh at my jokes is me 〰 Instagram: mckenzieraedoriot", "followers_count": 349, "friends_count": 347, "statues_count": 2936 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norton Shores, MI", "id": "efae765f7537164f", "name": "Norton Shores", "place_type": "city", "bounding_box": rectangle("-86.320001,43.117747 -86.20517,43.206059") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26121, "countyName": "Muskegon", "cityID": 2659140, "cityName": "Norton Shores" } }
+{ "create_at": datetime("2015-11-26T10:00:03.000Z"), "id": 669938684112805889, "text": "@a_garcia815 @Adeyemi12_ god damnit I was looking for this picture �� happy birthday Adeyemi ������", "in_reply_to_status": 669937338202644481, "in_reply_to_user": 760623241, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 760623241, 419562393 }}, "user": { "id": 2222289418, "name": "ɬαɬíαnα♥︎", "screen_name": "tatianarawrr", "lang": "en", "location": "null", "create_at": date("2013-12-12"), "description": "SHU '19", "followers_count": 233, "friends_count": 216, "statues_count": 8938 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Leonia, NJ", "id": "a0ca3d7968ca1f23", "name": "Leonia", "place_type": "city", "bounding_box": rectangle("-74.006065,40.851262 -73.975615,40.876115") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3440020, "cityName": "Leonia" } }
+{ "create_at": datetime("2015-11-26T10:00:03.000Z"), "id": 669938684242735104, "text": "thankfull for the first amendment that lets this feed be so ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16880995, "name": "David Sitrick", "screen_name": "davidsitrick", "lang": "en", "location": "Minneapolis", "create_at": date("2008-10-20"), "description": "this feed is bad. do not foollow", "followers_count": 458, "friends_count": 370, "statues_count": 31794 }, "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("2015-11-26T10:00:03.000Z"), "id": 669938684431470594, "text": "#ImThankfulFor @DorkyJaniel you are an amazing friend and please don't flirt with me I don't want you and you ship way too many people ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ImThankfulFor" }}, "user_mentions": {{ 4273255212 }}, "user": { "id": 1624925587, "name": "elizabeth / 6 days!", "screen_name": "DOLLHOUSESIVAN", "lang": "en", "location": " 6-13-15 // 10-17-15 :(", "create_at": date("2013-07-27"), "description": "@tyleroakley: hey person reading this, it's all gonna be fine", "followers_count": 1659, "friends_count": 1354, "statues_count": 41205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:03.000Z"), "id": 669938684586668032, "text": "Happy Thanksgiving https://t.co/MiPO8AI2DN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16049918, "name": "Jo§h Wallace", "screen_name": "Joshwallace", "lang": "en", "location": "Tucson, Arizona", "create_at": date("2008-08-29"), "description": "Follow me into the darkness...", "followers_count": 193, "friends_count": 336, "statues_count": 4960 }, "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-26T10:00:03.000Z"), "id": 669938684741816320, "text": "Interested in a #CustomerService #job near #Ozark, MO? This could be a great fit: https://t.co/gBXEeVtjdg #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": {{ "CustomerService", "job", "Ozark", "Hiring" }}, "user": { "id": 72034382, "name": "MO Cust. Srv. Jobs", "screen_name": "tmj_MO_cstsrv", "lang": "en", "location": "Missouri", "create_at": date("2009-09-06"), "description": "Follow this account for geo-targeted Customer Service job tweets in Missouri Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 316, "friends_count": 270, "statues_count": 288 }, "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("2015-11-26T10:00:03.000Z"), "id": 669938684762849280, "text": "Happy Thanksgiving to all from the Brahmas.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2420065896, "name": "Brahma Football", "screen_name": "brahmafootball", "lang": "en", "location": "null", "create_at": date("2014-03-30"), "description": "null", "followers_count": 247, "friends_count": 60, "statues_count": 668 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938685090136065, "text": "Gettysburg! @ Gettysburg National Military Park https://t.co/7HdNTqkrPd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.2216721,39.8121605"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 52046657, "name": "Marcy Gansler Dory", "screen_name": "marcygee", "lang": "en", "location": "Cleveland, OH", "create_at": date("2009-06-29"), "description": "Mother of Velociraptors. \nNeo-independent, cynical optimist, unselfish but self aware, loved and loving, passionate and compassionate.", "followers_count": 74, "friends_count": 143, "statues_count": 2902 }, "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": 42001, "countyName": "Adams" } }
+{ "create_at": datetime("2015-11-26T10:00:04.000Z"), "id": 669938685111046144, "text": "Wind 5.7 mph S. Barometer 30.690 in, Falling Rapidly. Temperature 50.0 °F. Rain today 0.00 in. Humidity 72%", "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": 4658 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938685215817728, "text": "Story of my life... https://t.co/r8MUXxCdTj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 635842225, "name": "Kylie✨", "screen_name": "YesNeverShoutNe", "lang": "en", "location": "null", "create_at": date("2012-07-14"), "description": "c/o '18. sophomore at PHS rest easy mom and kyle", "followers_count": 495, "friends_count": 610, "statues_count": 2626 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938685455040513, "text": "@being_maryam ME?", "in_reply_to_status": 669938494467387393, "in_reply_to_user": 777231931, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 777231931 }}, "user": { "id": 343865967, "name": "lia", "screen_name": "andrUSkoleah", "lang": "en", "location": "null", "create_at": date("2011-07-27"), "description": "spending money I don't have and being sad are my specialities,, 09/28", "followers_count": 523, "friends_count": 230, "statues_count": 12895 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938685505212416, "text": "@AspenLoar whoa okay bye", "in_reply_to_status": 669938166640586753, "in_reply_to_user": 2239823942, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2239823942 }}, "user": { "id": 86423348, "name": "D☄", "screen_name": "Foxxxybrown_", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2009-10-30"), "description": "Saginaw lady rough rider #21", "followers_count": 1704, "friends_count": 1178, "statues_count": 64658 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:04.000Z"), "id": 669938685505245184, "text": "#CareerArc #Retail #Job alert: Retail Store Shift Supervisor | CVS Health | #ElCentro, CA https://t.co/9ZtxYlQbnH #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.5630514,32.792"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CareerArc", "Retail", "Job", "ElCentro", "Jobs", "Hiring" }}, "user": { "id": 59784845, "name": "TMJ-CA Retail Jobs", "screen_name": "tmj_ca_retail", "lang": "en", "location": "California", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 506, "friends_count": 316, "statues_count": 3719 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938685568090112, "text": "Rain this afternoon, high 48 (9 C). Low 21 (-6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15953346, "name": "Amarillo Weather", "screen_name": "AmarilloWeather", "lang": "en", "location": "Amarillo, Texas", "create_at": date("2008-08-22"), "description": "Forecasts and advisories for Amarillo, Texas", "followers_count": 3992, "friends_count": 395, "statues_count": 8591 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938685568159744, "text": "@sammieexo__ @Bboygrape https://t.co/VWZDAIuSJI", "in_reply_to_status": -1, "in_reply_to_user": 844759814, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 844759814, 432784932 }}, "user": { "id": 1312465722, "name": "Bri ||X'O||", "screen_name": "bricedee14", "lang": "en", "location": "XOlifee", "create_at": date("2013-03-28"), "description": "XOTWOD//BBTM✨XO ~until you're used to my face and my mystery fades..@theweeknd //@JColeNC~Cole World #88 #CowBoysNation. R.I.P Adrian.Muniz ~@theweeknd|| AT❤️", "followers_count": 329, "friends_count": 715, "statues_count": 9215 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938686037897217, "text": "Couldn't be more thankful for all these people✨�� https://t.co/K3JtDqkJzO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1567486512, "name": "sofia giba✨", "screen_name": "sofiagiba", "lang": "en", "location": "ntf", "create_at": date("2013-07-03"), "description": "☯Ryan Salomon", "followers_count": 642, "friends_count": 628, "statues_count": 32589 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938686054780928, "text": "@awildtiddy more dancing", "in_reply_to_status": 669938574058496004, "in_reply_to_user": 2908265448, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2908265448 }}, "user": { "id": 1686366211, "name": "thiccflurry", "screen_name": "emothug69", "lang": "en", "location": "Amesbury, Ma", "create_at": date("2013-08-20"), "description": "Mediocre At Best", "followers_count": 474, "friends_count": 279, "statues_count": 16284 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portsmouth, NH", "id": "fb3aafdc1555efcf", "name": "Portsmouth", "place_type": "city", "bounding_box": rectangle("-70.820242,43.013673 -70.737324,43.098556") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33015, "countyName": "Rockingham", "cityID": 3362900, "cityName": "Portsmouth" } }
+{ "create_at": datetime("2015-11-26T10:00:04.000Z"), "id": 669938686138703874, "text": "What is everybody thankful for? Mine is family and friends", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.1626813,39.7733029"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 342457761, "name": "chase budinger", "screen_name": "CBudinger", "lang": "en", "location": "Los Angeles", "create_at": date("2011-07-25"), "description": "play basketball for a living... what a life!", "followers_count": 60379, "friends_count": 84, "statues_count": 2101 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938686201606144, "text": "@NicolasMaduro amor? Desgraciado mandaste a matar a los opositores.", "in_reply_to_status": 669892103267328000, "in_reply_to_user": 1252764865, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 1252764865 }}, "user": { "id": 63851617, "name": "Igor", "screen_name": "igoruiz", "lang": "es", "location": "null", "create_at": date("2009-08-07"), "description": "Abogado. Opositor a Régimen Militante de Primero Justicia y el Foro Penal Venezolano. Demócrata, Defensor de los DDHH. No bajare mi cara ni venderé mis ideales.", "followers_count": 270, "friends_count": 519, "statues_count": 5227 }, "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.957039") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1214895, "cityName": "Country Club" } }
+{ "create_at": datetime("2015-11-26T10:00:04.000Z"), "id": 669938686260187136, "text": "They might as well stay closed since all there customers want to #save50percent https://t.co/th4ZT6fAlj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "save50percent" }}, "user": { "id": 2254761895, "name": "Kris Lewis ", "screen_name": "kris_lewis33", "lang": "en", "location": "null", "create_at": date("2013-12-20"), "description": "null", "followers_count": 360, "friends_count": 180, "statues_count": 387 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938686365077504, "text": "Happy thanksgiving everyone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 171092073, "name": "JCX", "screen_name": "JCXWestCoast", "lang": "en", "location": "San Marcos, CA", "create_at": date("2010-07-26"), "description": "Pennsylvania Born, Connecticut Raised, California Living", "followers_count": 980, "friends_count": 723, "statues_count": 12018 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938686386167808, "text": "@JayGATA24 true", "in_reply_to_status": 669936292776550402, "in_reply_to_user": 1068459642, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1068459642 }}, "user": { "id": 517927481, "name": "(Chill)™", "screen_name": "Golden_Boi17", "lang": "en", "location": "United States", "create_at": date("2012-03-07"), "description": "FOCUS", "followers_count": 1309, "friends_count": 719, "statues_count": 21117 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938686528782337, "text": "#empezamosproceso #deshieloalavion #parapoderdespegar @ Denver International Airport https://t.co/VC7hDMOrMW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.673738,39.855096"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "empezamosproceso", "deshieloalavion", "parapoderdespegar" }}, "user": { "id": 273170179, "name": "Joaquin Barona M.", "screen_name": "Joaco288", "lang": "es", "location": "Mexico", "create_at": date("2011-03-27"), "description": "Dedicado a los espectáculos y la producción de los mismos. No existe la palabra no se puede en producción!, El NO, ya esta dado, siempre hay que buscar el SI!", "followers_count": 489, "friends_count": 304, "statues_count": 4950 }, "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": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2015-11-26T10:00:04.000Z"), "id": 669938686708965376, "text": "@whatabeezy28 huh", "in_reply_to_status": 669925375653187584, "in_reply_to_user": 3242947861, "favorite_count": 0, "coordinate": point("-96.18132994,32.30578201"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3242947861 }}, "user": { "id": 498290686, "name": "Bond", "screen_name": "FullTorqueBondr", "lang": "en", "location": "here, tx", "create_at": date("2012-02-20"), "description": "get sideways, chuck sauce, pound brews #nattys09", "followers_count": 259, "friends_count": 286, "statues_count": 3509 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tool, TX", "id": "5eb056fe666f8218", "name": "Tool", "place_type": "city", "bounding_box": rectangle("-96.205173,32.25983 -96.1524,32.312916") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48213, "countyName": "Henderson", "cityID": 4873352, "cityName": "Tool" } }
+{ "create_at": datetime("2015-11-26T10:00:04.000Z"), "id": 669938686805454848, "text": "@julia_romann love you mom ������", "in_reply_to_status": 669938407775346688, "in_reply_to_user": 1624295785, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1624295785 }}, "user": { "id": 2159837113, "name": "ya girl maiz", "screen_name": "yassomaizy", "lang": "en", "location": "btown", "create_at": date("2013-10-27"), "description": "vball", "followers_count": 574, "friends_count": 378, "statues_count": 11114 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938686868361216, "text": "IM SO EXCITED TO EAT SO MUCH LUMPIA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 985942514, "name": "melanie", "screen_name": "oddititty", "lang": "en", "location": "abq nm", "create_at": date("2012-12-02"), "description": "ring of chain // ehs '18 // virgo // pinoy", "followers_count": 401, "friends_count": 181, "statues_count": 7092 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938686910332928, "text": "You never know who really got you on their mind till the holidays. Mufuckas will wish you happy holidays out the blue lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 140299583, "name": "Lil Baby ✨", "screen_name": "CWeezy710", "lang": "en", "location": "Heaven", "create_at": date("2010-05-04"), "description": "You can call me Cat.", "followers_count": 1581, "friends_count": 829, "statues_count": 184693 }, "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-26T10:00:04.000Z"), "id": 669938686985904128, "text": "Thankful I gave up beanies and scene clothes. https://t.co/1M6W2ZZCxy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 723058994, "name": "kawaii mom", "screen_name": "fauxy_lady_", "lang": "en", "location": "Williamsburg, VA", "create_at": date("2012-07-28"), "description": "joocy currot", "followers_count": 221, "friends_count": 161, "statues_count": 16159 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938687136808960, "text": "@kparkkk_4 look at Normans story��", "in_reply_to_status": -1, "in_reply_to_user": 423550719, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 423550719 }}, "user": { "id": 2255996948, "name": "✨ T d O.G. DA GoD ✨", "screen_name": "tavita580", "lang": "en", "location": "null", "create_at": date("2013-12-20"), "description": "BABY Cr^nK | M8V3IIN | loco1ocho | man ima eat the whole ass", "followers_count": 299, "friends_count": 231, "statues_count": 2257 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938687220649984, "text": "My mom says, \"Do you want to bring anybody to your sister's wedding?\" �� another way to say, \"y el novio?\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 479169711, "name": "Susana", "screen_name": "SusyR_", "lang": "en", "location": "Colorado ", "create_at": date("2012-01-30"), "description": "22. Colorado State University. ΛΘΝ.", "followers_count": 298, "friends_count": 180, "statues_count": 8654 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wellington, CO", "id": "005702d73b6f3244", "name": "Wellington", "place_type": "city", "bounding_box": rectangle("-105.019475,40.675158 -104.990356,40.718061") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8069, "countyName": "Larimer", "cityID": 883230, "cityName": "Wellington" } }
+{ "create_at": datetime("2015-11-26T10:00:04.000Z"), "id": 669938687359102976, "text": "Mariah Carey so washed.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 45753127, "name": "$lo 20/3", "screen_name": "Slo_Motion", "lang": "en", "location": "Hyde Park, Chicago", "create_at": date("2009-06-08"), "description": "Don't make me fuck your mother, because I will. ♏", "followers_count": 1640, "friends_count": 978, "statues_count": 163730 }, "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-26T10:00:04.000Z"), "id": 669938687493275648, "text": "HAPY THANKSGIVING YOU JIVE TURKEY ASS NIGGAZ CELEBRATING THE DAY THAT THE EUROPEANS CAME AS GUESTS & STAYED AS CONQUER'S", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 297755596, "name": "Panthers 10-0", "screen_name": "Ohenry_", "lang": "en", "location": "Louisiana, USA", "create_at": date("2011-05-12"), "description": "University of Southern Mississippi C/o 2017. Pre-Med/ Biological Sciences Major! NOLA Hattiesburg.", "followers_count": 1194, "friends_count": 1023, "statues_count": 26621 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938687585746944, "text": "my grandpa just spent $40 on breakfast at McDonald's for four people lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3240498188, "name": "KiYA.", "screen_name": "princessrajhae", "lang": "en", "location": "null", "create_at": date("2015-06-08"), "description": "dope, blessed and wild✨", "followers_count": 420, "friends_count": 305, "statues_count": 13533 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938687723962368, "text": "ID BE EXTREMELY GRATEFUL IF SOMEONE GOT ME PANDA EXPRESS FOR BREAKFAST ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 544711049, "name": "G2", "screen_name": "water_fries", "lang": "en", "location": "null", "create_at": date("2012-04-03"), "description": "angels, disneyland, khs & taylor is a mini orange", "followers_count": 461, "friends_count": 627, "statues_count": 27364 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938687799631872, "text": "Best one yet������ https://t.co/T1rwbGPhMp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3740628917, "name": "Chris❌ Apple Sauce", "screen_name": "chrisdawwg21", "lang": "en", "location": "Wesley Chapel, FL", "create_at": date("2015-09-22"), "description": "WRHS C/O 2016 I just wanna do hoodrat stuff with my friends", "followers_count": 76, "friends_count": 82, "statues_count": 186 }, "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-26T10:00:04.000Z"), "id": 669938687870943236, "text": "https://t.co/quorOoYzu3 happy thanksgiving", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 446657628, "name": "Quinton Sykes", "screen_name": "CallMe_QSykes", "lang": "en", "location": "null", "create_at": date("2011-12-25"), "description": "being alone", "followers_count": 825, "friends_count": 922, "statues_count": 46307 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938688114094081, "text": "Afternoon thunderstorms/wind this afternoon, high 71 (22 C). Low 32 (0 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712333, "name": "LubbockWeather", "screen_name": "LubbockWeather", "lang": "en", "location": "Lubbock, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Lubbock, Texas", "followers_count": 1167, "friends_count": 93, "statues_count": 7924 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938688114229248, "text": "Happy thanksgiving everyone!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229376186, "name": "SAVANNAH✨", "screen_name": "OhSoRayy", "lang": "en", "location": "null", "create_at": date("2010-12-21"), "description": "•||namaste||•||uga17||•", "followers_count": 4525, "friends_count": 2261, "statues_count": 104462 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Griffin, GA", "id": "a846e2ce51e1233f", "name": "Griffin", "place_type": "city", "bounding_box": rectangle("-84.317652,33.202291 -84.228626,33.295471") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13255, "countyName": "Spalding", "cityID": 1335324, "cityName": "Griffin" } }
+{ "create_at": datetime("2015-11-26T10:00:04.000Z"), "id": 669938688197955585, "text": "it's a mother and son #reunion @ Village Of Lisburn, Pa https://t.co/1YbRj7rQKi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.90943325,40.16811832"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "reunion" }}, "user": { "id": 15249529, "name": "cara mia boyd", "screen_name": "caramiakelly", "lang": "en", "location": "null", "create_at": date("2008-06-26"), "description": "we like dancing and we look divine", "followers_count": 170, "friends_count": 346, "statues_count": 7742 }, "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": 42133, "countyName": "York" } }
+{ "create_at": datetime("2015-11-26T10:00:04.000Z"), "id": 669938688235806720, "text": "Still inlove with my school #chipride", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "chipride" }}, "user": { "id": 2198689893, "name": "Ervvvv", "screen_name": "erinnnnbabyyy", "lang": "en", "location": "chichester", "create_at": date("2013-11-27"), "description": "null", "followers_count": 1257, "friends_count": 1004, "statues_count": 16043 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boothwyn, PA", "id": "da9663d010c8036b", "name": "Boothwyn", "place_type": "city", "bounding_box": rectangle("-75.460232,39.822889 -75.428586,39.849375") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42045, "countyName": "Delaware", "cityID": 4207616, "cityName": "Boothwyn" } }
+{ "create_at": datetime("2015-11-26T10:00:04.000Z"), "id": 669938688294559748, "text": "take that turkey out of the oven and season it why do y'all refuse to properly season ?? the turkey basically alive https://t.co/AStLk0wbzu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 345451684, "name": "gran santo", "screen_name": "agudze_", "lang": "en", "location": "class of 2016 ✊", "create_at": date("2011-07-30"), "description": "i don't forget or forgive", "followers_count": 1283, "friends_count": 901, "statues_count": 69839 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938688458117120, "text": "I'm gonna go all out in the kitchen on Christmas", "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": 997, "friends_count": 747, "statues_count": 215293 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938688478973952, "text": "Thank you malia ! https://t.co/kO60vWEllS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 502465201, "name": "ryanne hig", "screen_name": "ryannehigbee", "lang": "en", "location": "bay area ", "create_at": date("2012-02-24"), "description": "lib 17' @stayblessed_n11", "followers_count": 257, "friends_count": 320, "statues_count": 1565 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938688533524480, "text": "Should I get the GBA Fire Emblem? I think it's Sword of Flame. Whichever one has Eliwood.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46904795, "name": "Patrick", "screen_name": "PhantomRoxas", "lang": "en", "location": "California", "create_at": date("2009-06-13"), "description": "Talkative nerd with questionable interests", "followers_count": 448, "friends_count": 519, "statues_count": 94230 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938688541917184, "text": "FREE: Child's Ikea Chalkboard (Jersey City Heights, NJ) @ https://t.co/vwLp68Cy9Q #trashnothing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0505752,40.7473758"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "trashnothing" }}, "user": { "id": 3820138154, "name": "NYC Freecycle", "screen_name": "NYCFreecycle", "lang": "en", "location": "New York, NY", "create_at": date("2015-10-07"), "description": "null", "followers_count": 0, "friends_count": 0, "statues_count": 329 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938688596402176, "text": "@Aylinl0pez looking like they like each other for once ����❤️", "in_reply_to_status": 669938448166334464, "in_reply_to_user": 339525417, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 339525417 }}, "user": { "id": 2322810247, "name": "ashley", "screen_name": "ashleyhunt33", "lang": "en", "location": "California", "create_at": date("2014-02-01"), "description": "I enjoy lemonade & back rubs", "followers_count": 475, "friends_count": 340, "statues_count": 3275 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938688713863168, "text": "Want to work at O'Reilly Auto Parts? We're #hiring in #LASVEGAS, NV! Click for details: https://t.co/qPPeEN9lTn #Transportation #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.0939977,36.1158082"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "LASVEGAS", "Transportation", "Job", "Jobs" }}, "user": { "id": 23502611, "name": "TMJ-LAS Transp. Jobs", "screen_name": "tmj_las_transp", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Transportation job tweets in Las Vegas, NV from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 329, "friends_count": 291, "statues_count": 65 }, "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("2015-11-26T10:00:04.000Z"), "id": 669938688902598656, "text": "@UltraGrobanite @SistersLoveD1D2 @BobLorrie @Nichols19Vicki Much better, finally!!!��", "in_reply_to_status": 669938402901512197, "in_reply_to_user": 1215978740, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1215978740, 2227712774, 3230349457, 3236865371 }}, "user": { "id": 3315724683, "name": "KATHLEEN ", "screen_name": "Kathlee67197879", "lang": "en", "location": "Old Pueblo, Az", "create_at": date("2015-06-09"), "description": "Family, animals, beautiful music and our big wide world are the greatest. Lots of laughs and giggles to keep us going. Love, love, love. Oh yeah-Josh Groban!!!", "followers_count": 435, "friends_count": 435, "statues_count": 5138 }, "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-26T10:00:04.000Z"), "id": 669938689003413504, "text": "@ryanseacrest tell me that x y and z are gay. By all means force them to love u weirdo cc: @thecourtneykerr", "in_reply_to_status": -1, "in_reply_to_user": 16190898, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16190898, 312457122 }}, "user": { "id": 629436153, "name": "Ester", "screen_name": "Ester822", "lang": "en", "location": "null", "create_at": date("2012-07-07"), "description": "null", "followers_count": 9, "friends_count": 1, "statues_count": 8528 }, "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("2015-11-26T10:00:05.000Z"), "id": 669938689204617216, "text": "@MerariM_ me too happy thanksgiving love u :) thank you��", "in_reply_to_status": 669936865546338304, "in_reply_to_user": 347687346, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 347687346 }}, "user": { "id": 207723039, "name": "Mari✌️", "screen_name": "doors_lovee", "lang": "en", "location": "null", "create_at": date("2010-10-25"), "description": "The Universe Works Mentally; Attending RHC, Future Occupational Therapist & Fitness.", "followers_count": 237, "friends_count": 211, "statues_count": 35874 }, "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("2015-11-26T10:00:05.000Z"), "id": 669938689347194880, "text": "Thankful for my dogs lol https://t.co/lzTYc7oLXf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3080352729, "name": "Haley ♥︎", "screen_name": "whateverhal", "lang": "en", "location": "Beaumont, CA", "create_at": date("2015-03-08"), "description": "cute", "followers_count": 159, "friends_count": 133, "statues_count": 5021 }, "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("2015-11-26T10:00:05.000Z"), "id": 669938689552683008, "text": "The one thanksgiving I'm gone my family decides to have a thanksgiving party ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2779640082, "name": "kody", "screen_name": "kodyjohnson58", "lang": "en", "location": "null", "create_at": date("2014-08-29"), "description": "null", "followers_count": 140, "friends_count": 305, "statues_count": 1489 }, "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("2015-11-26T10:00:05.000Z"), "id": 669938689724817408, "text": "#TESTIMONIALMUZIK\n#TEAMBRAYLON\n#TEAMKENZIE\n10% OF PROCEEDS GO TO BOTH #TEAMBRAYLON & #TEAMKENZIE\nhttps://t.co/pWMWo3aXp6 via youtube", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TESTIMONIALMUZIK", "TEAMBRAYLON", "TEAMKENZIE", "TEAMBRAYLON", "TEAMKENZIE" }}, "user": { "id": 1120870004, "name": "Kwesi Jones", "screen_name": "kwesijones20", "lang": "en", "location": "Rock Hill, SC", "create_at": date("2013-01-25"), "description": "just a guy who's trying to right the wrongs he's made. But with GOD instead of himself this time around.", "followers_count": 1047, "friends_count": 2099, "statues_count": 1420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rock Hill, SC", "id": "00696a56e1f9f443", "name": "Rock Hill", "place_type": "city", "bounding_box": rectangle("-81.086454,34.869151 -80.943098,34.961986") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45091, "countyName": "York", "cityID": 4561405, "cityName": "Rock Hill" } }
+{ "create_at": datetime("2015-11-26T10:00:05.000Z"), "id": 669938689758339074, "text": "All of a sudden thanksgiving is about roasting your relatives and not giving thanks. Thanks society.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2196840206, "name": "The Alpha", "screen_name": "Evan_7LTF", "lang": "en", "location": "Katy, TX", "create_at": date("2013-11-15"), "description": "Aussie born, Katy raised. 18. 3-year Varsity Track Athlete. SLHS '16", "followers_count": 216, "friends_count": 244, "statues_count": 3146 }, "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("2015-11-26T10:00:05.000Z"), "id": 669938690018398209, "text": "11 years ago in Zion, 10 years ago Venice Italy... The list is long & filled w/ great #thanksgiving… https://t.co/cGvmh4n8wk", "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": {{ "thanksgiving" }}, "user": { "id": 18945326, "name": "ChanceBoren", "screen_name": "ChanceBoren", "lang": "en", "location": "Here & Now!", "create_at": date("2009-01-13"), "description": "Husband ❤️ Father. Reader. LEGO Wizard. Professional Vagabond & Loud Mouth @ J!NX .com Love Lots. Make Comics Everyday. Never Grow Up", "followers_count": 232, "friends_count": 261, "statues_count": 8841 }, "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("2015-11-26T10:00:05.000Z"), "id": 669938690064429056, "text": "@jordanpage0 SAME", "in_reply_to_status": 669935563936411648, "in_reply_to_user": 2444928751, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2444928751 }}, "user": { "id": 2171011752, "name": "Austin Andreotti", "screen_name": "TinoAndreotti", "lang": "en", "location": "null", "create_at": date("2013-11-02"), "description": "Arizona State University '19", "followers_count": 127, "friends_count": 158, "statues_count": 285 }, "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("2015-11-26T10:00:05.000Z"), "id": 669938690077003777, "text": "@Simeonn_10 they bringing back the OG Cement 4s too��", "in_reply_to_status": 669938393577492480, "in_reply_to_user": 231898519, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 231898519 }}, "user": { "id": 165676083, "name": "Buloo", "screen_name": "AyyeRaay", "lang": "en", "location": "null", "create_at": date("2010-07-12"), "description": "Luke I:37", "followers_count": 698, "friends_count": 491, "statues_count": 14762 }, "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("2015-11-26T10:00:05.000Z"), "id": 669938690345451520, "text": "Still cute tho ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2214784609, "name": "nijonae", "screen_name": "janyamommy", "lang": "en", "location": "null", "create_at": date("2013-11-25"), "description": "null", "followers_count": 538, "friends_count": 626, "statues_count": 23705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hahnville, LA", "id": "633de12f5f2d4aac", "name": "Hahnville", "place_type": "city", "bounding_box": rectangle("-90.4326,29.942371 -90.388923,29.988476") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22089, "countyName": "St. Charles", "cityID": 2232510, "cityName": "Hahnville" } }
+{ "create_at": datetime("2015-11-26T10:00:05.000Z"), "id": 669938690366439424, "text": "@hughhefner voices from 90701", "in_reply_to_status": -1, "in_reply_to_user": 15830716, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15830716 }}, "user": { "id": 2657305226, "name": "Librado Banuelos", "screen_name": "11ifinda12", "lang": "en", "location": "Artesia, CA", "create_at": date("2014-07-18"), "description": "hi I am ifinda Endure us Choices world need know is love, sweet love, real love is kind. Yet He did so while being. Black hair brown eyes 5.6 hight weigh 170...", "followers_count": 44, "friends_count": 153, "statues_count": 785 }, "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("2015-11-26T10:00:05.000Z"), "id": 669938690576125952, "text": "https://t.co/90bebE5Vcd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 21704942, "name": "Pharoh JETINC72", "screen_name": "THEGREY1", "lang": "en", "location": "#Somewhereinamerica_ ", "create_at": date("2009-02-23"), "description": "Global Citizen, world traveler, Truth Seeker, HIP HOP ADDICT! -Hotep, Ase'♠️", "followers_count": 325, "friends_count": 2001, "statues_count": 3772 }, "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("2015-11-26T10:00:05.000Z"), "id": 669938690588844033, "text": "@_JimmyJJr https://t.co/4xquUMdCZi", "in_reply_to_status": 669937080642969600, "in_reply_to_user": 2586244010, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2586244010 }}, "user": { "id": 521969307, "name": "ⓉⒽⓄⓜⒶⓈ Ⓦ/ ⓉⒽⒺ ⒽⒶⒾⓇ", "screen_name": "Iplaytoomuch2k", "lang": "en", "location": "PCHS '16 ", "create_at": date("2012-03-11"), "description": "add me on snapchat: bucketsgonzalez // retired hooper, rapper, snapchat MVP, student and comedian", "followers_count": 537, "friends_count": 459, "statues_count": 14030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morrisville, NC", "id": "0122292c5bc9ff29", "name": "Morrisville", "place_type": "city", "bounding_box": rectangle("-78.881022,35.796927 -78.786799,35.870756") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3744520, "cityName": "Morrisville" } }
+{ "create_at": datetime("2015-11-26T10:00:05.000Z"), "id": 669938691171717120, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1172032224, "name": "Jose Medrano", "screen_name": "ChinoooMedrano", "lang": "en", "location": "• HTX • ", "create_at": date("2013-02-12"), "description": "null", "followers_count": 297, "friends_count": 252, "statues_count": 12387 }, "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("2015-11-26T10:00:05.000Z"), "id": 669938691414953984, "text": "I WANT IT https://t.co/JdKuicq5sU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2997733574, "name": "ΣΨΔ", "screen_name": "GomezSydni", "lang": "en", "location": "Alex is mine ((:", "create_at": date("2015-01-26"), "description": "love yourself and everyone else", "followers_count": 383, "friends_count": 512, "statues_count": 5051 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highlands, TX", "id": "da714a0bda9efd55", "name": "Highlands", "place_type": "city", "bounding_box": rectangle("-95.08943,29.792163 -95.02753,29.846673") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4833836, "cityName": "Highlands" } }
+{ "create_at": datetime("2015-11-26T10:00:05.000Z"), "id": 669938691557625856, "text": "Happy Thanksgiving to all, 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": 2268306180, "name": "£€MMį£Û", "screen_name": "ford_lem", "lang": "en", "location": "null", "create_at": date("2013-12-29"), "description": "I'm a Lazy person. Finna cry??", "followers_count": 337, "friends_count": 338, "statues_count": 1773 }, "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("2015-11-26T10:00:05.000Z"), "id": 669938691922464768, "text": "my home is open to anyone that doesn't have a place to celebrate thanksgiving :-)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 967273082, "name": "DEAUX.", "screen_name": "DeeIsaac_", "lang": "en", "location": "SATX", "create_at": date("2012-11-23"), "description": "what's normal anyway? @MARIEDESTINEEE is the queen.", "followers_count": 679, "friends_count": 193, "statues_count": 88902 }, "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-26T10:00:05.000Z"), "id": 669938692040089600, "text": "Take a damn seat https://t.co/AhX4L0F2Jq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3389194863, "name": "katie i.", "screen_name": "queenofmisandry", "lang": "en", "location": "☁️", "create_at": date("2015-07-23"), "description": "null", "followers_count": 57, "friends_count": 99, "statues_count": 304 }, "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-26T10:00:05.000Z"), "id": 669938692090302464, "text": "This #Hospitality #job might be a great fit for you: Shift Supervisor - https://t.co/1zM8QKEash #parttime #Naples, FL #Veterans #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.7995222,26.1739068"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "parttime", "Naples", "Veterans", "Hiring" }}, "user": { "id": 88076524, "name": "FL Hospitality Jobs", "screen_name": "tmj_fl_hrta", "lang": "en", "location": "Florida", "create_at": date("2009-11-06"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Florida Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 547, "friends_count": 337, "statues_count": 2019 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Naples, FL", "id": "a3f971b69fefb699", "name": "Naples", "place_type": "city", "bounding_box": rectangle("-81.817216,26.093538 -81.766738,26.211277") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12021, "countyName": "Collier", "cityID": 1247625, "cityName": "Naples" } }
+{ "create_at": datetime("2015-11-26T10:00:05.000Z"), "id": 669938692115468288, "text": "I wonder how long it would take for my feed to stop being 90% about kiss anime.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 204988689, "name": "Fernie (Fbot)", "screen_name": "Fbot_who", "lang": "en", "location": "Los Angeles", "create_at": date("2010-10-19"), "description": "I'm a 17yr old guy who doesn't know what to do with his life besides watching shit anime and playing shit video games. Ebina be my waifu.", "followers_count": 136, "friends_count": 223, "statues_count": 20049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:05.000Z"), "id": 669938692316930048, "text": "everyones shopping tonight https://t.co/UWHDhPvDrP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1932314768, "name": "cyberdragon", "screen_name": "cmvxox", "lang": "en", "location": "orlando #dp", "create_at": date("2013-10-03"), "description": "live by the codes of the ghetto", "followers_count": 469, "friends_count": 86, "statues_count": 5381 }, "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("2015-11-26T10:00:05.000Z"), "id": 669938692358856704, "text": "My contribution to Thanksgiving dinner: Awesome @RROysters! This might be a new tradition #HappyThanksgiving https://t.co/x4CnlS3AXw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HappyThanksgiving" }}, "user_mentions": {{ 83766632 }}, "user": { "id": 22627662, "name": "Kyle Bloom", "screen_name": "TheBloomBoys", "lang": "en", "location": "Norfolk, VA", "create_at": date("2009-03-03"), "description": "Director of @WAVY_News Today and The @HR_Show, Music Director @HR_Show, Music Contributor @hrScene, hop head, foodie, #AMNewser. My views are my own. @UCF Grad", "followers_count": 1109, "friends_count": 656, "statues_count": 7731 }, "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-26T10:00:05.000Z"), "id": 669938692560044032, "text": "COOL N DRE, THIS IS NO CEILINGS I DO THIS ALLDAY I GOT BARS NO RAILINGS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34789607, "name": "ɥdǝsoɾ", "screen_name": "underscoreJeaux", "lang": "en", "location": "houston new orleans", "create_at": date("2009-04-23"), "description": "twenty three. I dream of too much. coleworld", "followers_count": 373, "friends_count": 353, "statues_count": 34717 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:05.000Z"), "id": 669938692882960385, "text": "Thankful for this bitch ������ https://t.co/jH4BkfyWMt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1859550372, "name": "j", "screen_name": "aayyeee_", "lang": "en", "location": "CA✈️WA", "create_at": date("2013-09-12"), "description": "20. youre a wizard harry. tacos & tequila #sithlord #mileywhatsgood", "followers_count": 845, "friends_count": 569, "statues_count": 5747 }, "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("2015-11-26T10:00:05.000Z"), "id": 669938692903976961, "text": "I'm starving and hungover. Terrible combo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 122741751, "name": "Bryan", "screen_name": "SlowYourRoll79", "lang": "en", "location": "South River, NJ", "create_at": date("2010-03-13"), "description": "I chill. #Jets, #Knicks, #Devils, #Mets, and #Rutgers fan.", "followers_count": 671, "friends_count": 419, "statues_count": 57372 }, "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("2015-11-26T10:00:05.000Z"), "id": 669938692920705024, "text": "I'm v thankful for @yungnesssa even though she barely talks to me �� I love you �� https://t.co/g1EN0nxdNw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 725460565 }}, "user": { "id": 1607480544, "name": "Juicy Jaycee", "screen_name": "Jaayylmao_", "lang": "en", "location": "bay area", "create_at": date("2013-07-19"), "description": "tired", "followers_count": 603, "friends_count": 574, "statues_count": 18315 }, "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-26T10:00:05.000Z"), "id": 669938693017210880, "text": "Yuck... https://t.co/zgjbpq8eKa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 573102314, "name": "Kyle Masters✨☄", "screen_name": "KingCapitalKyle", "lang": "en", "location": "Off With Ya Hoe Laid Up...", "create_at": date("2012-05-06"), "description": "If Young ____ don't trust ya, I'm gon' shoot you. Ain't gon' even drive, I'ma Dougie to the party. Addicted to lightskin women with nice friends.", "followers_count": 1207, "friends_count": 639, "statues_count": 136368 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Okmulgee, OK", "id": "b8917d64f4d71c75", "name": "Okmulgee", "place_type": "city", "bounding_box": rectangle("-95.988707,35.595607 -95.890843,35.653606") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40111, "countyName": "Okmulgee", "cityID": 4055150, "cityName": "Okmulgee" } }
+{ "create_at": datetime("2015-11-26T10:00:05.000Z"), "id": 669938693017239552, "text": "Wind 0.0 mph ---. Barometer 1009.7 hPa, Falling. Temperature 87.6 °F. Rain today 0.17 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": 91, "friends_count": 264, "statues_count": 135562 }, "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("2015-11-26T10:00:05.000Z"), "id": 669938693038174208, "text": "@Spencer_Rayshon ��", "in_reply_to_status": 669938148772737024, "in_reply_to_user": 235770223, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 235770223 }}, "user": { "id": 2824978022, "name": "jessica", "screen_name": "litttleshitt", "lang": "en", "location": "San Marcos, TX", "create_at": date("2014-09-21"), "description": "fairy squad mother✨", "followers_count": 414, "friends_count": 291, "statues_count": 5571 }, "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("2015-11-26T10:00:05.000Z"), "id": 669938693126225920, "text": "What has 9 arms and stinks https://t.co/WsIr5OHwha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2188042480, "name": "RuinedJRuff", "screen_name": "Ruin2morrowJr", "lang": "en", "location": "Delaware Valley", "create_at": date("2013-11-19"), "description": "I love sarcasm as much as I love my sports teams. \nEvidently the worst sports fan ever", "followers_count": 1181, "friends_count": 802, "statues_count": 31473 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938693616963584, "text": "Thankful for this handsome man �� https://t.co/YsfaX4zCGB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 254658553, "name": "paige", "screen_name": "whittenpaige", "lang": "en", "location": "Washington", "create_at": date("2011-02-19"), "description": "The main way I educated myself, was just by living - Tony Alva", "followers_count": 661, "friends_count": 323, "statues_count": 43244 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orting, WA", "id": "2061d6e4f83d9ddf", "name": "Orting", "place_type": "city", "bounding_box": rectangle("-122.234941,47.063225 -122.15967,47.122039") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5352005, "cityName": "Orting" } }
+{ "create_at": datetime("2015-11-26T10:00:06.000Z"), "id": 669938693621350400, "text": "Happy thanksgiving ���� https://t.co/S8CimJLP39", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2710958532, "name": "AntisocialPessimist", "screen_name": "lexusdelaine", "lang": "en", "location": "Front Street", "create_at": date("2014-08-05"), "description": "Life.", "followers_count": 1001, "friends_count": 800, "statues_count": 21024 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938693680070656, "text": "@maarniefaaith how tan r u", "in_reply_to_status": -1, "in_reply_to_user": 2767544532, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2767544532 }}, "user": { "id": 1407752598, "name": "mr. logan", "screen_name": "SuperLogann", "lang": "en", "location": "Oswego, IL", "create_at": date("2013-05-06"), "description": "I like to heely my way through life. Hammock and chill? YL", "followers_count": 851, "friends_count": 682, "statues_count": 7832 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938693696651264, "text": "Wind 2.2 mph ESE. Barometer 30.01 in, Rising slowly. Temperature 54.7 °F. Rain today 0.00 in. Humidity 51%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.13472222,33.81666667"), "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": 20, "friends_count": 92, "statues_count": 12829 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938693705191426, "text": "Turkey Day selfie with the apple of my eye. Ain't she a beauty. :) @ Sabrina's House https://t.co/sB33wa2MWn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.69992422,37.12099388"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 183767188, "name": "Clint Tyler", "screen_name": "clinttyler12", "lang": "en", "location": "Benton, MO", "create_at": date("2010-08-27"), "description": "full time husband and dad. part time coach and stat guy. Sunday school teacher. sinner. Romans 8:28\n #TeamTyler #Rock_Chalk_Kelly_Hawk", "followers_count": 185, "friends_count": 323, "statues_count": 4323 }, "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": 29201, "countyName": "Scott" } }
+{ "create_at": datetime("2015-11-26T10:00:06.000Z"), "id": 669938693872812034, "text": "Afternoon thunderstorms this afternoon, high 70 (21 C). Low 41 (5 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712512, "name": "AbileneWeather", "screen_name": "AbileneWeather", "lang": "en", "location": "Abilene, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Abilene, Texas", "followers_count": 916, "friends_count": 93, "statues_count": 8022 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938694246277120, "text": "It's not a Happy Thanksgiving without a Bloody Mary breakfast!!! #happythanksgiving #Drunksgiving https://t.co/pGeXE6bDFn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "happythanksgiving", "Drunksgiving" }}, "user": { "id": 230853586, "name": "Nicole Smith", "screen_name": "snickers921", "lang": "en", "location": "null", "create_at": date("2010-12-26"), "description": "Part time writer, full time human, witty remarks, hipster flava, and the added deliciousness of a snickers bar!!!!", "followers_count": 68, "friends_count": 185, "statues_count": 88 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hunters Creek, FL", "id": "0a7eb306405f5129", "name": "Hunters Creek", "place_type": "city", "bounding_box": rectangle("-81.447294,28.347846 -81.404031,28.375674") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1232967, "cityName": "Hunters Creek" } }
+{ "create_at": datetime("2015-11-26T10:00:06.000Z"), "id": 669938694288117762, "text": "I'm grateful for everyone who handles all my bs ����❤", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3396119512, "name": "A.", "screen_name": "aaleeksees_", "lang": "en", "location": "null", "create_at": date("2015-07-30"), "description": "Alta Loma high", "followers_count": 227, "friends_count": 184, "statues_count": 3730 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938694309224448, "text": "@torbrwn literally nothing I hate myself", "in_reply_to_status": 669937884972060673, "in_reply_to_user": 1891066237, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1891066237 }}, "user": { "id": 475282867, "name": "allyson", "screen_name": "allysonheckard", "lang": "en", "location": "null", "create_at": date("2012-01-26"), "description": "delete my Twitter", "followers_count": 410, "friends_count": 297, "statues_count": 18283 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enola, PA", "id": "32fc5435112acd0e", "name": "Enola", "place_type": "city", "bounding_box": rectangle("-76.948898,40.261713 -76.923391,40.308957") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42041, "countyName": "Cumberland", "cityID": 4223744, "cityName": "Enola" } }
+{ "create_at": datetime("2015-11-26T10:00:06.000Z"), "id": 669938694351159296, "text": "Death on a stick out there, mate. https://t.co/rb0VSuw5Sk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 151148091, "name": "Cristina Ledra", "screen_name": "cledra", "lang": "en", "location": "Charlotte, NC", "create_at": date("2010-06-02"), "description": "Digital Content Producer @sportingnews. Toronto native, Florida woman. Helmet food enthusiast.", "followers_count": 597, "friends_count": 972, "statues_count": 9139 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938694388883457, "text": "�������� https://t.co/DjNz39z1S9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 938281597, "name": "Genesis..", "screen_name": "EdwinsBaked", "lang": "en", "location": "null", "create_at": date("2012-11-09"), "description": "Integrity & Tranquility | Sweet dreams, stuck in the 90's, 90's babies, its a matter of timing...", "followers_count": 311, "friends_count": 301, "statues_count": 5168 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938694476980229, "text": "https://t.co/Fiso6UoNl9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.1334034,25.7656012"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 23559416, "name": "Priscilla Michelle", "screen_name": "PriscillaPro", "lang": "en", "location": "Miami", "create_at": date("2009-03-09"), "description": "me!", "followers_count": 238, "friends_count": 272, "statues_count": 9588 }, "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-26T10:00:06.000Z"), "id": 669938694967721984, "text": "When days turned into months\nI begged her to explain....\nAnd this is what she sang...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 327690862, "name": "½ Justin X", "screen_name": "GrandFathrTiddy", "lang": "en", "location": "Davie, FL", "create_at": date("2011-07-01"), "description": "My goal is to be the hardest working man in any room im in.. Im gonna be influential in This country or Die tryin... • Talent Mgmt: GodspeedMgmt@outlook.com 20", "followers_count": 10604, "friends_count": 2036, "statues_count": 68824 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938695139528704, "text": "You can thank me later ���� https://t.co/mQOpxWYwjG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1428751057, "name": "Kimberly", "screen_name": "aggkimmy", "lang": "en", "location": "Houston ", "create_at": date("2013-05-14"), "description": "Fightin' Texas Aggie '03 | Classy & Sassy | I Bleed Maroon | #Bourbon Snob | I'm the 'Majestic Queen Awesome' | I'm the cool mom | #Cle | #OSU | With Turtle Man", "followers_count": 1235, "friends_count": 1989, "statues_count": 14327 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938695152144385, "text": "well happy thanksgiving everyone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3300895879, "name": "alec wendler", "screen_name": "AlecWendler", "lang": "en", "location": "null", "create_at": date("2015-07-29"), "description": "null", "followers_count": 28, "friends_count": 57, "statues_count": 1 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Taycheedah, WI", "id": "009a258f621f6cbc", "name": "Taycheedah", "place_type": "city", "bounding_box": rectangle("-88.401757,43.801523 -88.363325,43.83024") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55039, "countyName": "Fond du Lac", "cityID": 5579100, "cityName": "Taycheedah" } }
+{ "create_at": datetime("2015-11-26T10:00:06.000Z"), "id": 669938695223566336, "text": "@MailOnline lol!!!", "in_reply_to_status": 669908875294961664, "in_reply_to_user": 15438913, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 15438913 }}, "user": { "id": 3735801916, "name": "Maureen Carr", "screen_name": "Marr2rrCarr", "lang": "en", "location": "USA", "create_at": date("2015-09-21"), "description": "USA for USA Citizens, Pro-Life, Pro White , Proudly blocked by @CairNational Tired of whinny wussies like #mizzou backup @throwoutthebums", "followers_count": 738, "friends_count": 756, "statues_count": 5250 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938695282302976, "text": "Lucky I'm in love with my best friend (& lucky he has jerseys that I can wear as dresses) @ Windy… https://t.co/HhHrYMVA0L", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0288162,40.6280289"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 22301275, "name": "Valentina", "screen_name": "valcasamento", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2009-02-28"), "description": "Probably watching hockey", "followers_count": 604, "friends_count": 296, "statues_count": 45279 }, "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-26T10:00:06.000Z"), "id": 669938695626100736, "text": "@HVstation https://t.co/UYszySVeye", "in_reply_to_status": 669938241475321856, "in_reply_to_user": 3236571295, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3236571295 }}, "user": { "id": 3236571295, "name": "Happy Valley Station", "screen_name": "HVstation", "lang": "en", "location": "Happy Valley, OR", "create_at": date("2015-06-04"), "description": "null", "followers_count": 245, "friends_count": 363, "statues_count": 354 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938695701594112, "text": "Hello... �� and Happy Thanksgiving! ����Heeheehee ��: Dan Craig #adele #hello #hellothanksgiving… https://t.co/K5lG1KpTrn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.94162389,37.92071103"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "adele", "hello", "hellothanksgiving" }}, "user": { "id": 126459299, "name": "Camille Kelley", "screen_name": "sparkandfizzle", "lang": "en", "location": "Oakland, CA", "create_at": date("2010-03-25"), "description": "A twenty-something creative queerio living a life of odd adventures in the SF Bay Area.", "followers_count": 267, "friends_count": 1010, "statues_count": 2483 }, "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": 6013, "countyName": "Contra Costa", "cityID": 613882, "cityName": "Clayton" } }
+{ "create_at": datetime("2015-11-26T10:00:06.000Z"), "id": 669938695923892225, "text": "@savannahmcddd I love you ����������", "in_reply_to_status": 669938624721502208, "in_reply_to_user": 255180214, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 255180214 }}, "user": { "id": 1054222687, "name": "Skyler", "screen_name": "falkner_skyler", "lang": "en", "location": "Gardner, KS", "create_at": date("2013-01-01"), "description": "'19", "followers_count": 654, "friends_count": 492, "statues_count": 5995 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938695928057856, "text": "Check out our newsletter!...https://t.co/K5xQtlX8Rj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.769,45.431"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 243060056, "name": "LiveLaughLoveArt", "screen_name": "LLLoveGlass", "lang": "en", "location": "Tigard, Oregon", "create_at": date("2011-01-25"), "description": "Glass Blowing classes in Portland, OR", "followers_count": 83, "friends_count": 41, "statues_count": 951 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938695974227968, "text": "Can you recommend anyone for this #job? Pharmacy Technician - https://t.co/vPnxpvlBdq #FARMINGTONHILLS, MI #Healthcare #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.3771553,42.4853125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "FARMINGTONHILLS", "Healthcare", "Hiring", "CareerArc" }}, "user": { "id": 21723224, "name": "Detroit Healthcare", "screen_name": "tmj_dtw_health", "lang": "en", "location": "Detroit, MI", "create_at": date("2009-02-23"), "description": "Follow this account for geo-targeted Healthcare job tweets in Detroit, MI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 465, "friends_count": 305, "statues_count": 171 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Farmington Hills, MI", "id": "0b40afc99807b6ac", "name": "Farmington Hills", "place_type": "city", "bounding_box": rectangle("-83.437523,42.439001 -83.316839,42.529556") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2627440, "cityName": "Farmington Hills" } }
+{ "create_at": datetime("2015-11-26T10:00:06.000Z"), "id": 669938696104361985, "text": "Wind 0.0 mph E. Barometer 30.741 in, Falling. Temperature 61.5F. Rain today 0.00 in. Humidity 70% | 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": 564, "friends_count": 761, "statues_count": 34146 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938696221659136, "text": "final part�� @JeraminC https://t.co/mbNVKb2rqb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1141365091 }}, "user": { "id": 2628795597, "name": "trish", "screen_name": "_triiiisshh", "lang": "en", "location": "null", "create_at": date("2014-06-21"), "description": "'17 || ♋️", "followers_count": 190, "friends_count": 183, "statues_count": 5658 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938696422985732, "text": "Let's celebrate the genocide today by stuffing our faces with our families!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1466111870, "name": "Kurt", "screen_name": "adamjulio29", "lang": "en", "location": "W.B.C///DeadB⊗ys", "create_at": date("2013-05-28"), "description": "Fuck It Dawg Lifes A Risk.", "followers_count": 508, "friends_count": 376, "statues_count": 15203 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Springdale, AR", "id": "61ccab16a56229d3", "name": "Springdale", "place_type": "city", "bounding_box": rectangle("-94.223887,36.133324 -94.083891,36.23998") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5143, "countyName": "Washington", "cityID": 566080, "cityName": "Springdale" } }
+{ "create_at": datetime("2015-11-26T10:00:06.000Z"), "id": 669938696561385472, "text": "I'm missing a lot of people but I'm truly blessed with the friends and family I have,thank y'all for being here 4me�� https://t.co/0j7Jka4m5b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2690895968, "name": "BigSerg", "screen_name": "thetrill_24", "lang": "en", "location": "null", "create_at": date("2014-07-29"), "description": "Don't miss your opportunity", "followers_count": 488, "friends_count": 480, "statues_count": 5967 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938696582361088, "text": "Afternoon thunderstorms this afternoon, high 72 (22 C). Low 50 (10 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712530, "name": "MidlandWeather", "screen_name": "MidlandWeather", "lang": "en", "location": "Midland/Odessa, TX", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Midland/Odessa, TX", "followers_count": 864, "friends_count": 93, "statues_count": 8005 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938696670584833, "text": "❤️❤️❤️❤️❤️ https://t.co/Zu9LUCiqjW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1330432249, "name": "sydbabii", "screen_name": "syddythekiddy1", "lang": "en", "location": "w des❤️", "create_at": date("2013-04-05"), "description": "Justin is my Husband", "followers_count": 773, "friends_count": 778, "statues_count": 16792 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938696708186112, "text": "Eric's 10,000 miles away on this fine thanksgiving morning and I'm grateful that's the only thing I have to not be thankful for", "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": 741, "friends_count": 292, "statues_count": 33692 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938696817410048, "text": "17 seconds left to play Rumson has 1st and 10 at @ShoreAthletics 15", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 491599437 }}, "user": { "id": 338746486, "name": "The Link News", "screen_name": "TheLinkNews", "lang": "en", "location": "Long Branch, NJ", "create_at": date("2011-07-19"), "description": "You are what you read. \r\nYour link to what's going on in Long Branch, West Long Branch, Oceanport, Monmouth Beach, Ocean, Eatontown, and Sea Bright.", "followers_count": 1207, "friends_count": 179, "statues_count": 5740 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rumson, NJ", "id": "50eb4d67892e9d34", "name": "Rumson", "place_type": "city", "bounding_box": rectangle("-74.037661,40.339102 -73.976398,40.378911") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3465130, "cityName": "Rumson" } }
+{ "create_at": datetime("2015-11-26T10:00:06.000Z"), "id": 669938696846761984, "text": "@ReginaSchindel Happy Thanksgiving, cousin!", "in_reply_to_status": 669935005636907008, "in_reply_to_user": 461294702, "favorite_count": 0, "coordinate": point("-80.6079647,28.32535"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 461294702 }}, "user": { "id": 343546064, "name": "Tammy McCormack", "screen_name": "TammyAMcCormack", "lang": "en", "location": "N 42°50' 0'' / W 88°28' 0''", "create_at": date("2011-07-27"), "description": "VP of HR in Milwaukee, awesome mom, bibliophile, ASU undergrad alumna, UW-Milw grad school alumna, Chesapeake Bay Retriever owner & NASCAR/Brewers/Packers fan!", "followers_count": 183, "friends_count": 71, "statues_count": 3414 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cocoa Beach, FL", "id": "376e6de722d9a691", "name": "Cocoa Beach", "place_type": "city", "bounding_box": rectangle("-80.64362,28.286581 -80.601732,28.372031") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12009, "countyName": "Brevard", "cityID": 1213175, "cityName": "Cocoa Beach" } }
+{ "create_at": datetime("2015-11-26T10:00:06.000Z"), "id": 669938696909647872, "text": "mama was clacking��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2197278197, "name": "lololol stfu tooka:)", "screen_name": "staxemuprae", "lang": "en", "location": "null", "create_at": date("2013-11-26"), "description": "lol @ life", "followers_count": 606, "friends_count": 537, "statues_count": 20995 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938697014370304, "text": "How to save us all from the outcome you should be holding yourself accountable for..?\n#America how do we go about voting without", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "America" }}, "user": { "id": 1198982090, "name": "7639235149", "screen_name": "7639235149", "lang": "en", "location": "null", "create_at": date("2013-02-19"), "description": "#cNN America pre news news man", "followers_count": 32, "friends_count": 6, "statues_count": 90537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn Center, MN", "id": "70e030d7c5f03769", "name": "Brooklyn Center", "place_type": "city", "bounding_box": rectangle("-93.351357,45.039888 -93.281148,45.087566") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2707948, "cityName": "Brooklyn Center" } }
+{ "create_at": datetime("2015-11-26T10:00:06.000Z"), "id": 669938697069076480, "text": "Oh how the tables have turned https://t.co/F05sY2D7qA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2891571370, "name": "KING OF CHEVRON", "screen_name": "elijahwilliamsx", "lang": "en", "location": "Fl", "create_at": date("2014-11-24"), "description": "conversation starter", "followers_count": 516, "friends_count": 220, "statues_count": 7940 }, "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("2015-11-26T10:00:06.000Z"), "id": 669938697190543360, "text": "They watching porno for the dick these days smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 58236002, "name": "FreeMind619", "screen_name": "FreeMind619", "lang": "en", "location": "San Diego, CA", "create_at": date("2009-07-19"), "description": "for contact or booking:\nfreemind619@gmail.com", "followers_count": 1082, "friends_count": 374, "statues_count": 23017 }, "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-26T10:00:07.000Z"), "id": 669938697664466944, "text": "@_myBEAUTYspeakx dyke bitch", "in_reply_to_status": 669938612545417217, "in_reply_to_user": 156055808, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 156055808 }}, "user": { "id": 577601533, "name": "jada", "screen_name": "HaHaHa_NotFunny", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2012-05-11"), "description": "....just me", "followers_count": 989, "friends_count": 192, "statues_count": 41115 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:07.000Z"), "id": 669938697790328832, "text": "Can you recommend anyone for this #job? Carpet Cleaners - https://t.co/HjSkc0QlTC #Denver, CO #Transportation #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.0002242,39.7541032"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Denver", "Transportation", "Hiring" }}, "user": { "id": 23496679, "name": "TMJ-DEN Transp. Jobs", "screen_name": "tmj_den_transp", "lang": "en", "location": "Denver, CO", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Transportation job tweets in Denver, CO. Need help? Tweet us at @CareerArc!", "followers_count": 349, "friends_count": 292, "statues_count": 275 }, "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-26T10:00:07.000Z"), "id": 669938697807114241, "text": "This is also true https://t.co/cefaEYYuvd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 612269369, "name": "Temporary®", "screen_name": "Ardietwotimes", "lang": "en", "location": "Not in Jail", "create_at": date("2012-06-18"), "description": "Loading... 18| 214Tx|'17| Family first| R.I.P Kam| Wook| Newton Ananti 5/26| 2x, Be Great, Be Great!", "followers_count": 1324, "friends_count": 949, "statues_count": 54922 }, "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("2015-11-26T10:00:07.000Z"), "id": 669938697819717633, "text": "@Isaacja5 Thanks back", "in_reply_to_status": 669911906304438272, "in_reply_to_user": 4184528794, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4184528794 }}, "user": { "id": 724508209, "name": "Laura Nestor", "screen_name": "LauraNestor4", "lang": "en", "location": "L.A.Ca / Las Vegas, Nv USA", "create_at": date("2012-07-29"), "description": "60's child, Flower power Peace Love & enjoy many music genre,retired USPS Pls don't send porn to my DM", "followers_count": 8089, "friends_count": 7959, "statues_count": 64172 }, "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("2015-11-26T10:00:07.000Z"), "id": 669938697828044801, "text": "Join the Department of Veterans Affairs team! See our latest #Nursing #job opening here: https://t.co/QwzfcdRYY5 #Hines, IL #Veterans", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.8395035,41.8536425"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "Hines", "Veterans" }}, "user": { "id": 22874173, "name": "TMJ-IL Nursing Jobs", "screen_name": "tmj_il_nursing", "lang": "en", "location": "Illinois", "create_at": date("2009-03-04"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Illinois Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 379, "friends_count": 302, "statues_count": 595 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Broadview, IL", "id": "422cf154312cd404", "name": "Broadview", "place_type": "city", "bounding_box": rectangle("-87.883955,41.834043 -87.812541,41.871475") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1708446, "cityName": "Broadview" } }
+{ "create_at": datetime("2015-11-26T10:00:07.000Z"), "id": 669938698243301377, "text": "Yall don't know how happy I am to be off today. I haven't had a Thanksgiving in 5 years!�� this 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": 45653405, "name": "Briana", "screen_name": "AndaChola", "lang": "en", "location": "San Antonio, TX", "create_at": date("2009-06-08"), "description": "Nursing Major. Business Minor. Fit Life. Travel the World!", "followers_count": 782, "friends_count": 382, "statues_count": 67555 }, "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("2015-11-26T10:00:07.000Z"), "id": 669938698339921920, "text": "Wind 5.0 mph S. Barometer 30.424 in, Falling. Temperature 58.7 °F. Rain today 0.00 in. Humidity 62%", "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": 363, "friends_count": 286, "statues_count": 6260 }, "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("2015-11-26T10:00:07.000Z"), "id": 669938698503503872, "text": "Stop fouls and turnovers and it's a ball game", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 58814916, "name": "Taylor Mooney™", "screen_name": "Mooneypie13", "lang": "en", "location": "Mobile/ Tuscaloosa", "create_at": date("2009-07-21"), "description": "I am a washed up high school athlete Intern at @wvua23", "followers_count": 1337, "friends_count": 1233, "statues_count": 38060 }, "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("2015-11-26T10:00:07.000Z"), "id": 669938698570481664, "text": "On this day of thanks, I am thankful for many things. But most of all I am thankful for my future. #happythanksgiving #RisingStar", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "happythanksgiving", "RisingStar" }}, "user": { "id": 2778356031, "name": "Josh Weeks", "screen_name": "joshweeks9207", "lang": "en", "location": "null", "create_at": date("2014-09-20"), "description": "Athletic Operations and Marketing Professional", "followers_count": 74, "friends_count": 203, "statues_count": 453 }, "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("2015-11-26T10:00:07.000Z"), "id": 669938698754981890, "text": "@Honeyy_Tayy Happy Thanksgiving! ��", "in_reply_to_status": 669938293413314561, "in_reply_to_user": 318297356, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 318297356 }}, "user": { "id": 298379331, "name": "lë meilleur✨", "screen_name": "shambriaxo", "lang": "en", "location": "null", "create_at": date("2011-05-13"), "description": "17.| @ahstonlaflare ❤️", "followers_count": 1420, "friends_count": 525, "statues_count": 43556 }, "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("2015-11-26T10:00:07.000Z"), "id": 669938699329630208, "text": "@Nourberam94 I don't think so ��", "in_reply_to_status": 669934903996375042, "in_reply_to_user": 4123268093, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4123268093 }}, "user": { "id": 2694618027, "name": "ɯᴉɥɐɹqI", "screen_name": "cEngineer80", "lang": "en", "location": "Texas, USA ", "create_at": date("2014-07-10"), "description": "قَالَ رَبِّ إِنِّي وَهَنَ الْعَظْمُ مِنِّي وَاشْتَعَلَ الرَّأْسُ شَيْباً وَلَمْ أَكُن بِدُعَائِكَ رَبِّ شَقِيّاً", "followers_count": 388, "friends_count": 83, "statues_count": 12359 }, "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("2015-11-26T10:00:07.000Z"), "id": 669938699396759553, "text": "Cloudy this afternoon, high 78 (26 C). Low 69 (21 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712593, "name": "CorpusChristiWX", "screen_name": "CorpusChristiWX", "lang": "en", "location": "Corpus Christi, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Corpus Christi, Texas", "followers_count": 1004, "friends_count": 93, "statues_count": 7997 }, "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("2015-11-26T10:00:07.000Z"), "id": 669938699514175489, "text": "Trying to hang out with @Threetwoo_juan , since I'm in California.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 853934893 }}, "user": { "id": 2964893154, "name": "Allison", "screen_name": "_meganrdz", "lang": "en", "location": "null", "create_at": date("2015-01-07"), "description": "•|||||||• | livin' up my 20's ✌️ .", "followers_count": 77, "friends_count": 154, "statues_count": 451 }, "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-26T10:00:07.000Z"), "id": 669938699581288448, "text": "Temp: 51.7°F - Dew Point: 51.7° - Wind: 11.0 mph - Gust: 15.2 - Rain Today: 0.33in. - Pressure: 30.16in, - 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": 9701 }, "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("2015-11-26T10:00:07.000Z"), "id": 669938699782721536, "text": "#Round1 https://t.co/0TEl6SYRZa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Round1" }}, "user": { "id": 18098839, "name": "Brad Bialy", "screen_name": "bradbialy", "lang": "en", "location": "Buffalo, NY", "create_at": date("2008-12-13"), "description": "#SocialMedia Strategist and Content #Marketing Specialist advising 50+ firms in the #Staffing Industry at @HaleyMarketing. Acoustic musician. Founder @716Golf", "followers_count": 1059, "friends_count": 469, "statues_count": 5451 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hamburg, NY", "id": "0032b71e19053652", "name": "Hamburg", "place_type": "city", "bounding_box": rectangle("-78.916774,42.696345 -78.783331,42.789724") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3631643, "cityName": "Hamburg" } }
+{ "create_at": datetime("2015-11-26T10:00:07.000Z"), "id": 669938699807891457, "text": "@TWINbutnotTWINS we with y'all in spirit", "in_reply_to_status": 669930802684239874, "in_reply_to_user": 3492816082, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3492816082 }}, "user": { "id": 231809282, "name": "Ron Artest", "screen_name": "_fknight", "lang": "en", "location": "912", "create_at": date("2010-12-29"), "description": "Philippians 4:13 \n #ssu", "followers_count": 628, "friends_count": 717, "statues_count": 28028 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgetown, GA", "id": "002fe3572a510aea", "name": "Georgetown", "place_type": "city", "bounding_box": rectangle("-81.273589,31.960337 -81.183041,32.00743") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13051, "countyName": "Chatham", "cityID": 1332482, "cityName": "Georgetown" } }
+{ "create_at": datetime("2015-11-26T10:00:07.000Z"), "id": 669938699967184896, "text": "@k3osorio true", "in_reply_to_status": 669938374430474241, "in_reply_to_user": 183791327, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 183791327 }}, "user": { "id": 2346187092, "name": "Swægg Mônęÿ", "screen_name": "GonzoG_23", "lang": "en", "location": "Space Time Continuum ", "create_at": date("2014-02-15"), "description": "SGHS '15 ➡️ UCLA '19 Living life by the rules of my own philosophy. Know Your Worth. IG: gonzo_wolfgang", "followers_count": 706, "friends_count": 671, "statues_count": 40217 }, "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("2015-11-26T10:00:07.000Z"), "id": 669938700214759424, "text": "Unfortunately I have to work today.. But I don't go in till 3. Everyone have a great Thanksgiving", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 105954070, "name": "Brett Shearer™", "screen_name": "shearerboy", "lang": "en", "location": "Lancaster, KY", "create_at": date("2010-01-17"), "description": "Avid Kentucky Fan. #BBN #WeAreUK Garrard County District 1 Fire Department. Greater love has no one than this: to lay down ones life for ones friends John 15:13", "followers_count": 762, "friends_count": 547, "statues_count": 38148 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, KY", "id": "95b6844ded20a45b", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-84.602316,37.591909 -84.54868,37.631249") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21079, "countyName": "Garrard", "cityID": 2143840, "cityName": "Lancaster" } }
+{ "create_at": datetime("2015-11-26T10:00:07.000Z"), "id": 669938700407709697, "text": "LMAO https://t.co/Xuu7u8ysZq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 50031182, "name": "guzman", "screen_name": "tiffyMontana", "lang": "en", "location": "sangster intl", "create_at": date("2009-06-23"), "description": "west indian, rum & soca, lakers.", "followers_count": 2393, "friends_count": 729, "statues_count": 196222 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palm Springs, FL", "id": "018c4463229feb96", "name": "Palm Springs", "place_type": "city", "bounding_box": rectangle("-80.113171,26.619434 -80.068784,26.66254") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1254450, "cityName": "Palm Springs" } }
+{ "create_at": datetime("2015-11-26T10:00:07.000Z"), "id": 669938700441153536, "text": "This #Retail #job might be a great fit for you: Retail Store Shift Supervisor - https://t.co/6qpUgKpEWL #BREMEN, IN #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.148058,41.446436"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "BREMEN", "Hiring", "CareerArc" }}, "user": { "id": 59921955, "name": "TMJ-IN Retail Jobs", "screen_name": "tmj_in_retail", "lang": "en", "location": "Indiana", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in Indiana Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 452, "friends_count": 300, "statues_count": 2478 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bremen, IN", "id": "87bf34279eb19ff2", "name": "Bremen", "place_type": "city", "bounding_box": rectangle("-86.174628,41.435515 -86.129779,41.461389") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18099, "countyName": "Marshall", "cityID": 1807318, "cityName": "Bremen" } }
+{ "create_at": datetime("2015-11-26T10:00:07.000Z"), "id": 669938700495777792, "text": "hey that's me https://t.co/cnYujuZkE9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2575391099, "name": "h@y", "screen_name": "alohahay", "lang": "en", "location": "Philadelphia, Atlanta, L.A.", "create_at": date("2014-06-01"), "description": "bff/bf @AndrewCorneliu4", "followers_count": 550, "friends_count": 882, "statues_count": 6814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beechwood Trails, OH", "id": "465281ea63fbf96a", "name": "Beechwood Trails", "place_type": "city", "bounding_box": rectangle("-82.659676,40.008151 -82.623566,40.025966") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39089, "countyName": "Licking", "cityID": 3905013, "cityName": "Beechwood Trails" } }
+{ "create_at": datetime("2015-11-26T10:00:07.000Z"), "id": 669938700550332416, "text": "How imma stop being crazy ! How ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 348218443, "name": "5/18", "screen_name": "SoThatsKera", "lang": "en", "location": "null", "create_at": date("2011-08-03"), "description": "V-XVIII-XV", "followers_count": 608, "friends_count": 563, "statues_count": 41981 }, "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-26T10:00:07.000Z"), "id": 669938700588097537, "text": "I hope he don't think I'm dirty ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229146483, "name": "Positive Vibe.", "screen_name": "_Tonester", "lang": "en", "location": "null", "create_at": date("2010-12-21"), "description": "Jam❤️ ...UToledo18", "followers_count": 1010, "friends_count": 1014, "statues_count": 17006 }, "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-26T10:00:07.000Z"), "id": 669938701062025220, "text": "That voice message I left tho ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3020339958, "name": "THE REAL SMOOVE", "screen_name": "ricosmoovebaby1", "lang": "en", "location": "null", "create_at": date("2015-02-14"), "description": "FREE: SJ GANG\nR.I.P TERRY ️", "followers_count": 622, "friends_count": 852, "statues_count": 2730 }, "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-26T10:00:07.000Z"), "id": 669938701137416193, "text": "Can I eat pumpkin pie before dinner?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 389049013, "name": ".kassady.", "screen_name": "kassadyy", "lang": "en", "location": "null", "create_at": date("2011-10-11"), "description": "life is a journey, so dont worry", "followers_count": 898, "friends_count": 646, "statues_count": 26895 }, "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.604641") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4833146, "cityName": "Helotes" } }
+{ "create_at": datetime("2015-11-26T10:00:07.000Z"), "id": 669938701225562113, "text": "@KordasAlexia ily❤️☺️", "in_reply_to_status": 669938481662173185, "in_reply_to_user": 2263807560, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2263807560 }}, "user": { "id": 2187962536, "name": "Alyssa Casatelli", "screen_name": "CasatelliAlyssa", "lang": "en", "location": "null", "create_at": date("2013-11-19"), "description": "success is hard work and nobody's gunna do it for you", "followers_count": 742, "friends_count": 1024, "statues_count": 2525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Greenbush, NY", "id": "3326f9919ce45ed3", "name": "East Greenbush", "place_type": "city", "bounding_box": rectangle("-73.731523,42.580322 -73.682536,42.629289") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36083, "countyName": "Rensselaer", "cityID": 3622106, "cityName": "East Greenbush" } }
+{ "create_at": datetime("2015-11-26T10:00:07.000Z"), "id": 669938701321924608, "text": "Have a wonderful holiday and remember our friends and family working patrol today! https://t.co/NcyYLSilAK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3021512887, "name": "SBPD Reserve Unit", "screen_name": "SBPDReserveUnit", "lang": "en", "location": "San Bernardino, Ca.", "create_at": date("2015-02-15"), "description": "San Bernardino PD's reserve officer corps is chartered by the City of San Bernardino to provide volunteer uniformed law enforcement services to our citizens.", "followers_count": 112, "friends_count": 72, "statues_count": 131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crestline, CA", "id": "d436a8eab9002a8b", "name": "Crestline", "place_type": "city", "bounding_box": rectangle("-117.333719,34.222906 -117.242876,34.271189") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 617162, "cityName": "Crestline" } }
+{ "create_at": datetime("2015-11-26T10:00:07.000Z"), "id": 669938701359779841, "text": "Idol @meghangeiger https://t.co/HBIBJDh7yJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 770268644 }}, "user": { "id": 733194943, "name": "cailey leahy", "screen_name": "caybays", "lang": "en", "location": "null", "create_at": date("2012-08-02"), "description": "|-/ SJA", "followers_count": 484, "friends_count": 600, "statues_count": 6923 }, "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("2015-11-26T10:00:07.000Z"), "id": 669938701510836225, "text": "Successful high school reunion with the homies. #DidntBlackOutThisTime… https://t.co/h6UCOhjrkh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.095192,38.9823303"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DidntBlackOutThisTime" }}, "user": { "id": 17006996, "name": "quyen", "screen_name": "QuyenTran", "lang": "en", "location": "Hit the link below ", "create_at": date("2008-10-27"), "description": "Tax Accountant by day, and still doing taxes at night...", "followers_count": 1057, "friends_count": 961, "statues_count": 42272 }, "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.022437") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2407125, "cityName": "Bethesda" } }
+{ "create_at": datetime("2015-11-26T10:00:07.000Z"), "id": 669938701645037569, "text": "It's been a long time. @ Freed-Hardeman University https://t.co/DWS9V9HfQQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.6373713,35.43853293"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 389783318, "name": "BIGG SARGE", "screen_name": "biggsarge69", "lang": "en", "location": "GULFPORT MS", "create_at": date("2011-10-12"), "description": "An up and coming actor out of SIMPLE CITY, TN.\nUS ARMY SOLDIER with 23 years of Service.\n\nCEO of SARGE GOLD INTERNATIONAL", "followers_count": 922, "friends_count": 1556, "statues_count": 10429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, TN", "id": "f7a4a0076130c659", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-88.680513,35.410915 -88.611073,35.46227") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47023, "countyName": "Chester", "cityID": 4733260, "cityName": "Henderson" } }
+{ "create_at": datetime("2015-11-26T10:00:08.000Z"), "id": 669938701787467776, "text": "@NeilaGoddess hell yeah. ian wearing nun but basketball shorts ��", "in_reply_to_status": 669938424128929792, "in_reply_to_user": 3426689920, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3426689920 }}, "user": { "id": 463461124, "name": "ㅤㅤㅤㅤㅤㅤ", "screen_name": "nlmbtav", "lang": "en", "location": "Oak♿️liff, TX ", "create_at": date("2012-01-13"), "description": "#KitaWorld", "followers_count": 9086, "friends_count": 6844, "statues_count": 43259 }, "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-26T10:00:08.000Z"), "id": 669938702047698944, "text": "@jackiedevitoo love u sm papi������", "in_reply_to_status": 669927391985291264, "in_reply_to_user": 1520705796, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1520705796 }}, "user": { "id": 1486346718, "name": "allie", "screen_name": "alliieescolaroo", "lang": "en", "location": "null", "create_at": date("2013-06-05"), "description": "snap: alliescolaro", "followers_count": 491, "friends_count": 346, "statues_count": 5751 }, "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("2015-11-26T10:00:08.000Z"), "id": 669938702051708928, "text": "Heavy rain this afternoon, high 67 (19 C). Low 39 (4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712706, "name": "TulsaWX", "screen_name": "TulsaWX", "lang": "en", "location": "Tulsa, Oklahoma", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Tulsa, Oklahoma", "followers_count": 511, "friends_count": 93, "statues_count": 8059 }, "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("2015-11-26T10:00:08.000Z"), "id": 669938702240514048, "text": "Thankful for everything. The highs and the lows.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1656039978, "name": "Joss", "screen_name": "lynlynn__", "lang": "en", "location": "HTX", "create_at": date("2013-08-08"), "description": "◡̈⃝◡̈⃝◡̈⃝◡̈⃝◡̈⃝", "followers_count": 833, "friends_count": 433, "statues_count": 10499 }, "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("2015-11-26T10:00:08.000Z"), "id": 669938702269997059, "text": "Snap chat me dat me �� Kanishaaa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "nl", "is_retweet": false, "user": { "id": 95552992, "name": "⚓️Naj.", "screen_name": "_Kanishaa", "lang": "en", "location": "null", "create_at": date("2009-12-08"), "description": "null", "followers_count": 1967, "friends_count": 867, "statues_count": 65318 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delmar, MD", "id": "019f39d07a2d3507", "name": "Delmar", "place_type": "city", "bounding_box": rectangle("-75.603236,38.409095 -75.553685,38.46881") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24045, "countyName": "Wicomico", "cityID": 2422600, "cityName": "Delmar" } }
+{ "create_at": datetime("2015-11-26T10:00:08.000Z"), "id": 669938702504714240, "text": "The RAW IMPACT Unleashed Episode #02 S06 https://t.co/XJVSt7uoXh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 83498576, "name": "Kevin Kendrick", "screen_name": "wcbpodcastkk1", "lang": "en", "location": "Bay Area", "create_at": date("2009-10-18"), "description": "West Coast Bias.... Sports.... With a west coast feel.", "followers_count": 689, "friends_count": 1601, "statues_count": 12591 }, "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("2015-11-26T10:00:08.000Z"), "id": 669938702983012352, "text": "Aw ur cute taking pictures with unlit cigarettes in ur mouth :) steal them from your mom ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 84905641, "name": "angel", "screen_name": "frickailee", "lang": "en", "location": "KEMMA", "create_at": date("2009-10-24"), "description": "Always been daddy's little angel", "followers_count": 1080, "friends_count": 198, "statues_count": 49646 }, "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("2015-11-26T10:00:08.000Z"), "id": 669938703058362368, "text": "so many instagram thankful posts �� I love reading how people are thankful for others lol ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3021315950, "name": "Mami", "screen_name": "FaithAlxx", "lang": "en", "location": "null", "create_at": date("2015-02-15"), "description": "who me? :)", "followers_count": 149, "friends_count": 119, "statues_count": 4832 }, "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-26T10:00:08.000Z"), "id": 669938703196946433, "text": "Ripley SW Limestone Co. Temp: 69.8°F Wind:9.2mph Pressure: 1011.9mb 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": 37, "friends_count": 33, "statues_count": 44736 }, "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("2015-11-26T10:00:08.000Z"), "id": 669938703654125569, "text": "@valvs_jared55 some things u just gotta do, at least it wasn't a picture of ur bum", "in_reply_to_status": 669938607378026496, "in_reply_to_user": 2768519393, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2768519393 }}, "user": { "id": 870863671, "name": "Jake Unger", "screen_name": "ungs_jake5", "lang": "en", "location": "Buffalo NY", "create_at": date("2012-10-09"), "description": "Class of '17", "followers_count": 845, "friends_count": 491, "statues_count": 14677 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Seneca, NY", "id": "8e7819a4c1852d78", "name": "West Seneca", "place_type": "city", "bounding_box": rectangle("-78.801819,42.803843 -78.696854,42.874972") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3680907, "cityName": "West Seneca" } }
+{ "create_at": datetime("2015-11-26T10:00:08.000Z"), "id": 669938703796584448, "text": "@MikaylaNetser ❤️❤️❤️❤️", "in_reply_to_status": 669934034089193472, "in_reply_to_user": 775072302, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 775072302 }}, "user": { "id": 615693293, "name": "libbs", "screen_name": "lliibbbyy", "lang": "en", "location": "Flagstaff, AZ", "create_at": date("2012-06-22"), "description": "nothing will work unless you do ••• NAU '19", "followers_count": 892, "friends_count": 590, "statues_count": 15547 }, "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("2015-11-26T10:00:08.000Z"), "id": 669938703863848960, "text": "@drewdelucci @OnlyHipHopFacts @mymixtapez @LilReese300 that shit definitely ain't droppin today ����✌��️", "in_reply_to_status": 669927972049182720, "in_reply_to_user": 47124954, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 47124954, 834095274, 354347533, 303595809 }}, "user": { "id": 1603591334, "name": "LamsOffThaShitz", "screen_name": "LamsTheGreat", "lang": "en", "location": "Islip, NY", "create_at": date("2013-07-18"), "description": "Liam Lennon | 18 Year Old Producer | New York State Of Mind | Email (Business Only) : lamsthegreat@gmail.com | IG: lamsoffthashitz | snapchat: loudpacklams |", "followers_count": 564, "friends_count": 354, "statues_count": 13830 }, "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("2015-11-26T10:00:08.000Z"), "id": 669938704467759104, "text": "shoutout to the girls touching the snow they use to keep the fish cold that's laying on the side of the street", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 454481379, "name": "zoë eisenhaure", "screen_name": "eisennugget", "lang": "en", "location": "the tiny state", "create_at": date("2012-01-03"), "description": "Bears. Beets. Battlestar Galactica.", "followers_count": 214, "friends_count": 369, "statues_count": 362 }, "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("2015-11-26T10:00:08.000Z"), "id": 669938704526393344, "text": "need my hair to get a bit longer �� https://t.co/dRNDrQ6r8P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 546532110, "name": "me", "screen_name": "newnewxo", "lang": "en", "location": "HTX", "create_at": date("2012-04-05"), "description": "21 years. Kennedy Madisyn 11/29 & Mr. Houston ❤️", "followers_count": 1077, "friends_count": 473, "statues_count": 47325 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:08.000Z"), "id": 669938704555749376, "text": "im hungover #thankful", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thankful" }}, "user": { "id": 2403749177, "name": "baby mërë", "screen_name": "ttttalley", "lang": "en", "location": "Virginia Beach, VA", "create_at": date("2014-03-11"), "description": "eat my shorts", "followers_count": 899, "friends_count": 467, "statues_count": 25499 }, "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("2015-11-26T10:00:08.000Z"), "id": 669938704677376000, "text": "latiranie brother so sweet (': ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1393507326, "name": "كيمبرلي نيكول", "screen_name": "kimbxrlytruuu", "lang": "en", "location": "ronnie banks bed ", "create_at": date("2013-04-30"), "description": "in love with my 6 pc of chicken | lit till the day I die", "followers_count": 833, "friends_count": 257, "statues_count": 39186 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:08.000Z"), "id": 669938704727707648, "text": "Rain/wind this afternoon, high 64 (18 C). Low 34 (1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712679, "name": "OKCWX", "screen_name": "OKCWX", "lang": "en", "location": "Oklahoma City, Oklahoma", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Oklahoma City, Oklahoma", "followers_count": 175, "friends_count": 93, "statues_count": 8086 }, "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-26T10:00:08.000Z"), "id": 669938704836919296, "text": "A 3 mile pre-�� hike? No problem ���� @ Mead's Quarry https://t.co/CvXm2UQNvb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.86591584,35.95178352"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 301135035, "name": "Jordan Conway", "screen_name": "jayconway01", "lang": "en", "location": "Kansas City, MO", "create_at": date("2011-05-18"), "description": "Grad Student at UMKC. FSA & Community Service Grad Assistant. Kappa Kappa Gamma Alumnae. Lover of all things queso.", "followers_count": 702, "friends_count": 938, "statues_count": 6491 }, "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("2015-11-26T10:00:08.000Z"), "id": 669938704853544960, "text": "@real_moiklmowse @judyannemcc @tbo51972 that's not a word", "in_reply_to_status": 669938280549457920, "in_reply_to_user": 3179927847, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3179927847, 19360394, 2670425995 }}, "user": { "id": 2518715191, "name": "Jill", "screen_name": "seejillrun1", "lang": "en", "location": "Portland, Or", "create_at": date("2014-05-23"), "description": "Wanderer + Photographer + Opera Singer + Runner + Portlander + Anglophile who finally got The Twitter!", "followers_count": 121, "friends_count": 188, "statues_count": 5322 }, "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("2015-11-26T10:00:08.000Z"), "id": 669938704862048258, "text": "Happy Thanksgiving from Minnesota! @ Macalester College https://t.co/2BozPpSzIP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.1681061,44.9399567"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19807961, "name": "Mary Warren Dickens", "screen_name": "dickensmw", "lang": "en", "location": "St Paul, MN / Eastern NC", "create_at": date("2009-01-30"), "description": "Providing you with mostly useless thoughts of 140 characters or less since 2009. My favorite seasons are election season and awards season.", "followers_count": 516, "friends_count": 1675, "statues_count": 10071 }, "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.89076 -93.003514,44.992279") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27123, "countyName": "Ramsey", "cityID": 2758000, "cityName": "St. Paul" } }
+{ "create_at": datetime("2015-11-26T10:00:08.000Z"), "id": 669938704866234368, "text": "@zoiekins Yes ������", "in_reply_to_status": 669938508950319104, "in_reply_to_user": 3057123810, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3057123810 }}, "user": { "id": 2765527004, "name": "Jourdan", "screen_name": "Jxvrdyn", "lang": "en", "location": "null", "create_at": date("2014-08-24"), "description": "null", "followers_count": 559, "friends_count": 436, "statues_count": 1926 }, "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-26T10:00:08.000Z"), "id": 669938705000488961, "text": "@Nolapane dc are more complex books", "in_reply_to_status": 669937625114017793, "in_reply_to_user": 53309635, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 53309635 }}, "user": { "id": 313407727, "name": "Cris", "screen_name": "From_1991", "lang": "en", "location": "NYFLCA", "create_at": date("2011-06-08"), "description": "I'm here because I was told there would be doughnuts.", "followers_count": 43, "friends_count": 140, "statues_count": 1762 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oviedo, FL", "id": "fea68b7c38002608", "name": "Oviedo", "place_type": "city", "bounding_box": rectangle("-81.241886,28.622785 -81.13398,28.692232") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12117, "countyName": "Seminole", "cityID": 1253575, "cityName": "Oviedo" } }
+{ "create_at": datetime("2015-11-26T10:00:08.000Z"), "id": 669938705021460484, "text": "wus good�� https://t.co/5ISelB7oDc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 224371249, "name": "thomas costello", "screen_name": "_thomascostello", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-12-08"), "description": "instagram/snapchat : thomasjcostello", "followers_count": 37967, "friends_count": 1107, "statues_count": 4555 }, "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("2015-11-26T10:00:08.000Z"), "id": 669938705184894978, "text": "@TheBennGinigeme I probably did lowkey but who's name first��", "in_reply_to_status": 669937364848914432, "in_reply_to_user": 280395631, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 280395631 }}, "user": { "id": 417473806, "name": "BĪGGG WoË", "screen_name": "Westoospacely", "lang": "en", "location": "RIP Eloka, KAM & WOOK", "create_at": date("2011-11-20"), "description": "Lost my brudda in 2015..... So I gotta put on By myself", "followers_count": 1285, "friends_count": 721, "statues_count": 60693 }, "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("2015-11-26T10:00:08.000Z"), "id": 669938705193414657, "text": "#ImThankfulFor being able to continue my education & the reality check I was given that made me get my shit together in school.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ImThankfulFor" }}, "user": { "id": 375917305, "name": "ANTI-TREA$URE", "screen_name": "thedrunktruth", "lang": "en", "location": "Montserrat, WI", "create_at": date("2011-09-18"), "description": "treasure. 20. bisexual. music enthusiast. RI ♋️ @rihanna @NICKIMINAJ @Drake @JColeNC", "followers_count": 1245, "friends_count": 987, "statues_count": 104911 }, "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("2015-11-26T10:00:08.000Z"), "id": 669938705331838978, "text": "Yeah do i celebrate these BS killing holiday's.No but i will eat some bird and all that good stuff fam or myself making.I will all can eat.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43050075, "name": "IZZYBK", "screen_name": "Izzyizo", "lang": "en", "location": "Brooklyn NY", "create_at": date("2009-05-27"), "description": "#TEAM40BARRS,#TEAMQB Born Brooklyn.Raised n BK&the South BX NY. I work even before i had working papers.I paid the cost to be Boss!Lovethereal", "followers_count": 1427, "friends_count": 2080, "statues_count": 62963 }, "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-26T10:00:08.000Z"), "id": 669938705424105472, "text": "@queeencrys thanks boo��", "in_reply_to_status": 669936471806210048, "in_reply_to_user": 4005558616, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4005558616 }}, "user": { "id": 379754446, "name": "ally esposito", "screen_name": "Crazyesposito21", "lang": "en", "location": "null", "create_at": date("2011-09-25"), "description": "no matter where life takes me you'll find me with a smile ... #Frostburg19", "followers_count": 1931, "friends_count": 1106, "statues_count": 45550 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Odenton, MD", "id": "3795cbef20412c50", "name": "Odenton", "place_type": "city", "bounding_box": rectangle("-76.734394,39.019327 -76.643899,39.103254") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24003, "countyName": "Anne Arundel", "cityID": 2458300, "cityName": "Odenton" } }
+{ "create_at": datetime("2015-11-26T10:00:08.000Z"), "id": 669938705612668928, "text": "Just posted a photo @ Bannockburn, Illinois https://t.co/GRVY0tsPhj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.8631,42.1915"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 247634521, "name": "Evan Romoff", "screen_name": "evanromoff", "lang": "en", "location": "Santa Monica, CA", "create_at": date("2011-02-04"), "description": "Commercial Director, Est. 1985 email for inquiries: contact@evanromoff.com", "followers_count": 7871, "friends_count": 205, "statues_count": 2778 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bannockburn, IL", "id": "4bd35b7b55e854f9", "name": "Bannockburn", "place_type": "city", "bounding_box": rectangle("-87.889627,42.178197 -87.85187,42.203769") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1703610, "cityName": "Bannockburn" } }
+{ "create_at": datetime("2015-11-26T10:00:08.000Z"), "id": 669938705612718080, "text": "Happy Thanksgiving! ♥ I'm thankful for my family & for everyone who has put 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": 313460620, "name": "Jazzy Nicole ❀", "screen_name": "HeyItsJazlynne", "lang": "en", "location": "SD, CA", "create_at": date("2011-06-08"), "description": "Stay Happy & Smile More (๑˙╰╯˙๑)♡", "followers_count": 470, "friends_count": 2013, "statues_count": 71478 }, "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-26T10:00:08.000Z"), "id": 669938705612804096, "text": "@Almightyyyyy possibly omw over there", "in_reply_to_status": 669938630308331520, "in_reply_to_user": 2403570785, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2403570785 }}, "user": { "id": 2733026722, "name": "FSUoverUF", "screen_name": "TaylorJr_4", "lang": "en", "location": "null", "create_at": date("2014-08-03"), "description": "813✈✈✈251\nBishop State Student Athlete⚾\nFlorida", "followers_count": 409, "friends_count": 329, "statues_count": 9707 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938706019516416, "text": "Happy thanksgiving to all & I hope everyone has a great day of eating & joy with their families ❤️��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1116070795, "name": "Matria", "screen_name": "matriuhhh", "lang": "en", "location": "Murrieta,Ca", "create_at": date("2013-01-23"), "description": "just let me know wus good", "followers_count": 1538, "friends_count": 1534, "statues_count": 16772 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938706221015040, "text": "For my dhmc family ���� https://t.co/gn52XjUhiV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2304060434, "name": "Aya Khanafer", "screen_name": "ayak300", "lang": "en", "location": "where the wild things are ", "create_at": date("2014-01-21"), "description": "Dchs '19", "followers_count": 149, "friends_count": 113, "statues_count": 3496 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938706464174081, "text": "I'm thankful for music, the office and my dogs.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 445726934, "name": "Mariah.™", "screen_name": "MariahLuna2", "lang": "en", "location": "null", "create_at": date("2011-12-24"), "description": "null", "followers_count": 227, "friends_count": 228, "statues_count": 9771 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938706481061888, "text": "can't wait till it's time eat https://t.co/8vuKIetw7c", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 516182529, "name": "nelz", "screen_name": "__chanelly", "lang": "en", "location": "subistagang", "create_at": date("2012-03-05"), "description": "dance ✨", "followers_count": 686, "friends_count": 636, "statues_count": 16220 }, "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-26T10:00:09.000Z"), "id": 669938706514583552, "text": "Thankful for C cubed ����‍��‍���� @ Battis Field https://t.co/VALnNNTBWx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.90527073,41.89270593"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 373511892, "name": "Chris Kenney", "screen_name": "ChrisKenney13", "lang": "en", "location": "@kellyslason", "create_at": date("2011-09-14"), "description": "massart :~) boston :~)", "followers_count": 876, "friends_count": 478, "statues_count": 24162 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Middleborough Center, MA", "id": "00d8beb558f3e0da", "name": "Middleborough Center", "place_type": "city", "bounding_box": rectangle("-70.977086,41.868615 -70.874632,41.955934") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25023, "countyName": "Plymouth", "cityID": 2540885, "cityName": "Middleborough Center" } }
+{ "create_at": datetime("2015-11-26T10:00:09.000Z"), "id": 669938706686472192, "text": "I never received the memo about everyone showing up at 12. My family is all here & I'm not even dressed yet....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 365787227, "name": "JH|✿", "screen_name": "JessicaTX__", "lang": "en", "location": "•TX❥", "create_at": date("2011-08-31"), "description": "null", "followers_count": 641, "friends_count": 592, "statues_count": 15142 }, "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-26T10:00:09.000Z"), "id": 669938706749325312, "text": "@JoseCruz_4 bro I mean I'm down to chill but not Finna give free rides", "in_reply_to_status": 669938581029306370, "in_reply_to_user": 2545324872, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2545324872 }}, "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": 1359, "friends_count": 913, "statues_count": 23947 }, "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-26T10:00:09.000Z"), "id": 669938706824871937, "text": "the way he shows her off man:// https://t.co/n1zVtpxDfU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 734886686, "name": "urfavforeign✨", "screen_name": "x_lisanicole", "lang": "en", "location": "dtx", "create_at": date("2012-08-03"), "description": "battle's", "followers_count": 1959, "friends_count": 982, "statues_count": 6321 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938706967490560, "text": "@BrandieeAlenaaa @BrianBelich @KTVieira1 @HannahKraft1 @alex_kraft7 okay", "in_reply_to_status": 669938206888951808, "in_reply_to_user": 269983513, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 269983513, 4266126748, 357195834, 350991169, 2507354374 }}, "user": { "id": 453385985, "name": "Kay", "screen_name": "KayleighK143", "lang": "en", "location": "null", "create_at": date("2012-01-02"), "description": "A beautiful little fool", "followers_count": 304, "friends_count": 112, "statues_count": 8010 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forked River, NJ", "id": "00d88eb470b73bd2", "name": "Forked River", "place_type": "city", "bounding_box": rectangle("-74.223558,39.809802 -74.143,39.879125") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean", "cityID": 3424180, "cityName": "Forked River" } }
+{ "create_at": datetime("2015-11-26T10:00:09.000Z"), "id": 669938706971803649, "text": "this family time is really cutting into my Tindering time ...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 258024469, "name": "Chloe", "screen_name": "krank_daddy", "lang": "en", "location": "Skeeps", "create_at": date("2011-02-26"), "description": "*screams internally*", "followers_count": 229, "friends_count": 237, "statues_count": 8523 }, "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-26T10:00:09.000Z"), "id": 669938707009421313, "text": "@DAnarche I am Native American and find it extremely offensive and would never celebrate the murder of my people. To each is own, though.", "in_reply_to_status": 669938286635290624, "in_reply_to_user": 1478230033, "favorite_count": 0, "coordinate": point("-121.9352227,37.6957035"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1478230033 }}, "user": { "id": 32772956, "name": "kotaaa wintt t t t t", "screen_name": "dakotawint", "lang": "en", "location": "ॐcurrently on a human journeyॐ", "create_at": date("2009-04-17"), "description": "(a nervous wreck that's faking fine) - dakotaswint@gmail.com-- alien / earth activist / writer http://instagram.com/dakotawint http://youtube.com/dakotawint", "followers_count": 157125, "friends_count": 56880, "statues_count": 30679 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938707068243973, "text": "@DarrenKavinoky thx for the follow", "in_reply_to_status": -1, "in_reply_to_user": 53210455, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 53210455 }}, "user": { "id": 2914023112, "name": "LJ Johnson", "screen_name": "lorrainej1227", "lang": "en", "location": "New Jersey not born & raised", "create_at": date("2014-12-10"), "description": "Knowing too much & knowing too little can be determential.\n\n50, yikes, that age did not happen.", "followers_count": 625, "friends_count": 1456, "statues_count": 17122 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938707189776384, "text": "@Yung_Yamaka �� aw na, Dev caught all the Angela Simmons in Juicy track suits & Rev tweeting live from the tub! Lol those were the days huh?", "in_reply_to_status": 669933215818186752, "in_reply_to_user": 1324935030, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1324935030 }}, "user": { "id": 83458285, "name": "☀️SOLEIL B☀️", "screen_name": "SoleilBeez", "lang": "en", "location": "Calif. / Bawwwston.", "create_at": date("2009-10-18"), "description": "california native. lil'journalist & editor, a social sunbeam - or ♡ just a girl that likes boys who like the NewYorkTimes. soleilbarros@gmail.com if u need me", "followers_count": 706, "friends_count": 270, "statues_count": 19544 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938707193925633, "text": "We ran the whole thing without walking ������ #turkeytrot #happythanksgiving @ Manito Park https://t.co/1BsDsUKjYQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.41202956,47.6357396"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "turkeytrot", "happythanksgiving" }}, "user": { "id": 2724718452, "name": "Jenitals", "screen_name": "JennaJensen05", "lang": "en", "location": "null", "create_at": date("2014-08-11"), "description": "don't", "followers_count": 718, "friends_count": 211, "statues_count": 12108 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938707412029440, "text": "Isolated thunderstorms this afternoon, high 77 (25 C). Low 70 (21 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712488, "name": "SanAntonioWX", "screen_name": "SanAntonioWX", "lang": "en", "location": "San Antonio, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for San Antonio, Texas", "followers_count": 245, "friends_count": 93, "statues_count": 8076 }, "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-26T10:00:09.000Z"), "id": 669938707726667776, "text": "#ImThankfulFor @violetcordova @thatkidangel @_Auhum_ and everyone else who doesn't have twitter ���������� https://t.co/t2z703KlQg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ImThankfulFor" }}, "user_mentions": {{ 3189452228, 85733164, 621098913 }}, "user": { "id": 332055038, "name": "Ronnee", "screen_name": "urfaev", "lang": "en", "location": "GVHS", "create_at": date("2011-07-08"), "description": "gvhs key club is the best // ig: ronneefae // I highly recommend listening to @TheFilharmonic :)", "followers_count": 120, "friends_count": 162, "statues_count": 4130 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938707751800832, "text": "Aunt: we're gonna use Mary's(my cousin wife) M&C casserole recipe\nMe: is there going to be regular macaroni\nAunt: no https://t.co/iEyHz2WKll", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 321461357, "name": "Joseph Uh•Knee•Kwe", "screen_name": "JosephAniekwe", "lang": "en", "location": "Denton, Tx", "create_at": date("2011-06-21"), "description": "UNT Live|ASO|Collegiate 100|UNT| Austin ✈ Denton", "followers_count": 986, "friends_count": 976, "statues_count": 24698 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938707772915713, "text": "@jaydiegaines like 3 or 4 or maybe 5 years ago I'm pretty sure I went to thanksgiving with you at your familys house so thats funny", "in_reply_to_status": -1, "in_reply_to_user": 48085792, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 48085792 }}, "user": { "id": 415245757, "name": "abby sikes", "screen_name": "arsikes13", "lang": "en", "location": "cullman, al", "create_at": date("2011-11-17"), "description": "instagram @abbyrsikes // sc @arsikes13", "followers_count": 817, "friends_count": 427, "statues_count": 3139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cullman, AL", "id": "6ebac9d8febba273", "name": "Cullman", "place_type": "city", "bounding_box": rectangle("-86.895519,34.125239 -86.795043,34.238319") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1043, "countyName": "Cullman", "cityID": 118976, "cityName": "Cullman" } }
+{ "create_at": datetime("2015-11-26T10:00:09.000Z"), "id": 669938708003618816, "text": "@IM_SIMPLYMadE_ KJ finna eat so good today ������", "in_reply_to_status": 669714925548236800, "in_reply_to_user": 273655950, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 273655950 }}, "user": { "id": 3188209117, "name": "Kay-Kay✨", "screen_name": "GoldenTrillest", "lang": "en", "location": "null", "create_at": date("2015-05-07"), "description": "KJ & Ty", "followers_count": 242, "friends_count": 300, "statues_count": 3874 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938708293001216, "text": "Thankful 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": 3029513950, "name": "Qua.Glizzy™", "screen_name": "qliggans15", "lang": "en", "location": "Michigan state university 10-1", "create_at": date("2015-02-10"), "description": "Class of 19'", "followers_count": 70, "friends_count": 49, "statues_count": 137 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938708498374657, "text": "Thanksgiving BINGO #blessed https://t.co/xpTsA5gDBl", "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": 25168425, "name": "Amy Kane Williams", "screen_name": "amybo1", "lang": "en", "location": "Phoenix Metro", "create_at": date("2009-03-18"), "description": "Realtor specializing in the southeast Phoenix metro area.", "followers_count": 358, "friends_count": 801, "statues_count": 1277 }, "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-26T10:00:09.000Z"), "id": 669938708506767360, "text": "First Turkey Day �� https://t.co/l75VwLb12V", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70427197, "name": "moneymakinmani", "screen_name": "_thelovelym", "lang": "en", "location": "finding forever", "create_at": date("2009-08-31"), "description": "half poison, half god | black by popular demand. sc: queeenmani", "followers_count": 1105, "friends_count": 818, "statues_count": 66458 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:09.000Z"), "id": 669938708548747264, "text": "Wind 5.2 mph E. Barometer 30.154 in, Falling slowly. Temperature 75.0 °F. Rain today 0.02in. Humidity 84%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.11555556,29.6725"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25210383, "name": "New Braunfels WX", "screen_name": "jeffreycentexwx", "lang": "en", "location": "New Braunfels, TX", "create_at": date("2009-03-18"), "description": "Weather Station located on County Line Road in New Braunfels, TX.\r\n\r\nRealtime Weather @ http://www.nbtx-wx.com\r\nFollow me @ http://twitter.com/jeffreycentex", "followers_count": 370, "friends_count": 22, "statues_count": 156306 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938708599013376, "text": "This picture says a lot. https://t.co/eQh39qmknu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2207328618, "name": "Jake Neumann", "screen_name": "realjakeneumann", "lang": "en", "location": "null", "create_at": date("2013-11-21"), "description": "M3 AGAINST TH3 WORLD \nFL·MN", "followers_count": 153, "friends_count": 110, "statues_count": 262 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938708771160065, "text": "turkey time courtesy of @frontierchicago (@ Frontier in Chicago, IL) https://t.co/ikpMl0CfEC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.66366387,41.90127095"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 182048258 }}, "user": { "id": 15649203, "name": "narni", "screen_name": "narni", "lang": "en", "location": "chicago", "create_at": date("2008-07-29"), "description": "a digitally- technology- driven @chicago native and president of the @UofCAlumChicago club", "followers_count": 460, "friends_count": 401, "statues_count": 1511 }, "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-26T10:00:09.000Z"), "id": 669938708842311680, "text": "@lannnnii love you ��", "in_reply_to_status": 669934143224999937, "in_reply_to_user": 1373301720, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1373301720 }}, "user": { "id": 2591725608, "name": "kenn", "screen_name": "kennr0se", "lang": "en", "location": "qc", "create_at": date("2014-06-27"), "description": "kg", "followers_count": 596, "friends_count": 448, "statues_count": 3551 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Globe, AZ", "id": "4fb788c6d5b6b5ec", "name": "Globe", "place_type": "city", "bounding_box": rectangle("-110.833315,33.378346 -110.748379,33.432198") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4007, "countyName": "Gila", "cityID": 428030, "cityName": "Globe" } }
+{ "create_at": datetime("2015-11-26T10:00:09.000Z"), "id": 669938709005860864, "text": "I love you @DollieDee__ . I'm so thankful for you. https://t.co/ai88marq3B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 283374990 }}, "user": { "id": 633320356, "name": "B", "screen_name": "Biankaabeyta", "lang": "en", "location": "Albuquerque, NM", "create_at": date("2012-07-11"), "description": "fit lil prince$$.... FÈRINA", "followers_count": 1931, "friends_count": 947, "statues_count": 40117 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938709073104896, "text": "@TylerGonzales23 we did good considering our previous track record still not impressive haha", "in_reply_to_status": 669938368554401792, "in_reply_to_user": 345592867, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 345592867 }}, "user": { "id": 412590671, "name": "Sara Barker", "screen_name": "_Saroc_", "lang": "en", "location": "null", "create_at": date("2011-11-14"), "description": "Scraft af", "followers_count": 1085, "friends_count": 749, "statues_count": 45838 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938709165268992, "text": "To be honest my family doesn't really celebrate today since where not close and everyone has their own families. So I'm not getting ready .��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1025243185, "name": "ॐ", "screen_name": "_CristianAmadeo", "lang": "en", "location": "#peacemob♡ॐ☮", "create_at": date("2012-12-20"), "description": "My only goal is to be happy ;)", "followers_count": 1122, "friends_count": 897, "statues_count": 24096 }, "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-26T10:00:09.000Z"), "id": 669938709270261760, "text": "Always Go Back To The Hood For Holidays", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 332462155, "name": "MillySantana⭐️", "screen_name": "milikebennett", "lang": "en", "location": "Mobile, AL", "create_at": date("2011-07-09"), "description": "Somewhere In The Trenches .......... it ✌️✋.", "followers_count": 2511, "friends_count": 2145, "statues_count": 28304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prichard, AL", "id": "149f31daeaf575d8", "name": "Prichard", "place_type": "city", "bounding_box": rectangle("-88.180728,30.719003 -88.061948,30.803025") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile", "cityID": 162496, "cityName": "Prichard" } }
+{ "create_at": datetime("2015-11-26T10:00:09.000Z"), "id": 669938709303795712, "text": "so thankful for my best friends, I love y'all so much @FigleyLauren @maya_jade3 https://t.co/eawKcUMznl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1963432753, 2922989278 }}, "user": { "id": 568626519, "name": "Kimber", "screen_name": "kimscarbrough_", "lang": "en", "location": "TX // NC", "create_at": date("2012-05-01"), "description": "remember my chains", "followers_count": 600, "friends_count": 419, "statues_count": 20798 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Piney Green, NC", "id": "7ae09b8022667372", "name": "Piney Green", "place_type": "city", "bounding_box": rectangle("-77.362653,34.712551 -77.285071,34.784616") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37133, "countyName": "Onslow", "cityID": 3752260, "cityName": "Piney Green" } }
+{ "create_at": datetime("2015-11-26T10:00:09.000Z"), "id": 669938709391740929, "text": "Appreciation post for deez nuts ������ https://t.co/AQjpYTPYvq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 320399459, "name": "han", "screen_name": "hannahwicki", "lang": "en", "location": "RO '16", "create_at": date("2011-06-19"), "description": "don't count the days, make the days count // sc: hannahwicki", "followers_count": 404, "friends_count": 311, "statues_count": 6744 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkley, MI", "id": "5cbea7748a197682", "name": "Berkley", "place_type": "city", "bounding_box": rectangle("-83.203431,42.488362 -83.158788,42.511326") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2607660, "cityName": "Berkley" } }
+{ "create_at": datetime("2015-11-26T10:00:09.000Z"), "id": 669938709433716736, "text": "Happy thanksgiving to everyone.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 968150688, "name": "bryan", "screen_name": "steezgodbryan", "lang": "en", "location": "null", "create_at": date("2012-11-24"), "description": "rbryan.info@gmail.com for all inquires.", "followers_count": 981, "friends_count": 287, "statues_count": 18688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:09.000Z"), "id": 669938709484060672, "text": "U.S. pulls spies from China after hack September 30 - CNNMoney https://t.co/nAdDs6ZBfg https://t.co/4iLaNBzBAU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.276067,30.433762"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3853367722, "name": "James Giddeon", "screen_name": "jammie1169", "lang": "en", "location": "null", "create_at": date("2015-10-03"), "description": "null", "followers_count": 867, "friends_count": 2653, "statues_count": 70 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walgreens", "id": "07d9eccedbc82000", "name": "Walgreens", "place_type": "poi", "bounding_box": rectangle("-87.27606709999999,30.4337619 -87.276067,30.433762") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12033, "countyName": "Escambia", "cityID": 1247550, "cityName": "Myrtle Grove" } }
+{ "create_at": datetime("2015-11-26T10:00:09.000Z"), "id": 669938709547130882, "text": "@imnotsavana thanks for being you��", "in_reply_to_status": 669938349176655872, "in_reply_to_user": 1576270436, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1576270436 }}, "user": { "id": 1923721610, "name": "lil santa", "screen_name": "love50463", "lang": "en", "location": "null", "create_at": date("2013-10-01"), "description": "Idk man you do you ☃", "followers_count": 173, "friends_count": 60, "statues_count": 329 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938709794562049, "text": "Biggest thanks for the man up above who keeps watching over me&given me the ability to fight cancer 2x &win those battles❤️����#overlygrateful", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "overlygrateful" }}, "user": { "id": 921739418, "name": "Bri", "screen_name": "BriannaBove", "lang": "en", "location": "Staten Island, NY", "create_at": date("2012-11-02"), "description": "itt iss what itt iss #fuckcancer", "followers_count": 780, "friends_count": 679, "statues_count": 48629 }, "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-26T10:00:09.000Z"), "id": 669938709895077888, "text": "@LivingByChances Hope you're having a tremendous day. Will you have a Thanksgiving gathering and all the love.❤️ https://t.co/GSB7s4APGm", "in_reply_to_status": -1, "in_reply_to_user": 2831047673, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2831047673 }}, "user": { "id": 1616114203, "name": "KathrynCookHernandez", "screen_name": "rynher", "lang": "en", "location": "Midwest, USA", "create_at": date("2013-07-23"), "description": "Wandering the pathways of the mind in image, tale and song, KATHRYN I AM", "followers_count": 248, "friends_count": 258, "statues_count": 6334 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938709895114752, "text": "Thunderstorms this afternoon, high 71 (22 C). Low 57 (14 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712452, "name": "FortWorthWX", "screen_name": "FortWorthWX", "lang": "en", "location": "Fort Worth, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Fort Worth, Texas", "followers_count": 208, "friends_count": 93, "statues_count": 8063 }, "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("2015-11-26T10:00:09.000Z"), "id": 669938709945565184, "text": "YRVW Greg.. TY for all the @swarm-ins.. Feeling grateful Happy Thanksgiving.. @gregweir @GaryLoper @RT2EAT", "in_reply_to_status": 669925224729677824, "in_reply_to_user": 113413353, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 117197667, 113413353, 16175706, 574672888 }}, "user": { "id": 861548858, "name": "@cindycapo", "screen_name": "cindycapo", "lang": "en", "location": "IL,USA", "create_at": date("2012-10-04"), "description": "Social Media Enthusiast! #Twitter #FaceBook #G+ #EmpireAve, Social Media, Friends, Opportunities, #TeamRT2EAT @RT2EAT #TeamTwitWhizz NO MGWV NO PORN NO DMs", "followers_count": 24271, "friends_count": 8333, "statues_count": 152212 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grayslake, IL", "id": "26d88ba39cce8290", "name": "Grayslake", "place_type": "city", "bounding_box": rectangle("-88.067992,42.305548 -87.992956,42.388167") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1731121, "cityName": "Grayslake" } }
+{ "create_at": datetime("2015-11-26T10:00:09.000Z"), "id": 669938709978968064, "text": "I'm thankful for Netflix and chipotle", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2720210208, "name": "xanax", "screen_name": "ana_avocado", "lang": "en", "location": "null", "create_at": date("2014-08-09"), "description": "sunsets & no regrets CHS XC/T&F\r\nSenior", "followers_count": 274, "friends_count": 438, "statues_count": 2110 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chesterton, IN", "id": "bcbf83640d70852e", "name": "Chesterton", "place_type": "city", "bounding_box": rectangle("-87.124381,41.568289 -87.008687,41.640063") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18127, "countyName": "Porter", "cityID": 1812412, "cityName": "Chesterton" } }
+{ "create_at": datetime("2015-11-26T10:00:10.000Z"), "id": 669938710142578688, "text": "The spam song, substitute yam.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24789484, "name": "Nicholas Mitchell", "screen_name": "puxxled", "lang": "en", "location": "Santa Cruz Mountains ♭♩♬♪♫♩ ", "create_at": date("2009-03-16"), "description": "Once & future host of [cancelled] #MusicaDellaSera (@mdsKUSP) a #classical show on @KUSP #SantaCruz, co-hosted by @MeeraHyphenated. Tweets éclectique.", "followers_count": 2939, "friends_count": 2173, "statues_count": 82141 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boulder Creek, CA", "id": "1745c8216f56bf42", "name": "Boulder Creek", "place_type": "city", "bounding_box": rectangle("-122.148283,37.112351 -122.099741,37.162952") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 607652, "cityName": "Boulder Creek" } }
+{ "create_at": datetime("2015-11-26T10:00:10.000Z"), "id": 669938710176100353, "text": "i have so much more but im running out of data, will add more when i get wifi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3063418001, "name": "linden ramsey", "screen_name": "cubgender", "lang": "en", "location": "null", "create_at": date("2015-02-25"), "description": "he/it/xe", "followers_count": 103, "friends_count": 210, "statues_count": 266 }, "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("2015-11-26T10:00:10.000Z"), "id": 669938710411001856, "text": "Thanksgiving 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": { "id": 1236739292, "name": "Farrah Cross", "screen_name": "CrossFarrah", "lang": "en", "location": "null", "create_at": date("2013-03-02"), "description": "If you always do what makes you happy at least one person is pleased", "followers_count": 98, "friends_count": 226, "statues_count": 837 }, "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("2015-11-26T10:00:10.000Z"), "id": 669938710444568576, "text": "Chief Keef x Un Un", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 421277199, "name": "PAT 10:3 & PJ 11:13", "screen_name": "IFinesseThePlug", "lang": "en", "location": "C\\O 2017", "create_at": date("2011-11-25"), "description": "@itstaty__ is my blood #PatsWorld #freemark #freeant #freedeewee #freejt #freejunior #freefoole #freemyuncle #freekeyon #freedesto K.I.P PJ 11:13", "followers_count": 2227, "friends_count": 1632, "statues_count": 68590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:10.000Z"), "id": 669938710687805440, "text": "@sagethewhales pumpkin better win ������", "in_reply_to_status": 669911228437786624, "in_reply_to_user": 437883530, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 437883530 }}, "user": { "id": 437883530, "name": "ⓢⓐⓖⓔ", "screen_name": "sagethewhales", "lang": "en", "location": "KU ΑΣΤ", "create_at": date("2011-12-15"), "description": "vegan whales painting hashbrowns", "followers_count": 853, "friends_count": 638, "statues_count": 8403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Schnecksville, PA", "id": "01ad30c96eb32132", "name": "Schnecksville", "place_type": "city", "bounding_box": rectangle("-75.636567,40.648322 -75.590882,40.683918") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42077, "countyName": "Lehigh", "cityID": 4268192, "cityName": "Schnecksville" } }
+{ "create_at": datetime("2015-11-26T10:00:10.000Z"), "id": 669938710708908036, "text": "Story of my lifeeeee https://t.co/CtrczHnzdP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2452266217, "name": "Negus .", "screen_name": "blurry__viii", "lang": "en", "location": "null", "create_at": date("2014-04-18"), "description": "US Marine Corp. Soccer ❤️⚽️ SUNY OW", "followers_count": 471, "friends_count": 348, "statues_count": 11514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elmont, NY", "id": "bf83670d90b7f0bc", "name": "Elmont", "place_type": "city", "bounding_box": rectangle("-73.729189,40.686184 -73.688289,40.720895") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3624273, "cityName": "Elmont" } }
+{ "create_at": datetime("2015-11-26T10:00:10.000Z"), "id": 669938710897491968, "text": "@TheTideNate is Drew farting again������", "in_reply_to_status": 669938484145090560, "in_reply_to_user": 3057666629, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3057666629 }}, "user": { "id": 611295897, "name": "Megan Simpson❤️", "screen_name": "violet77789", "lang": "en", "location": "4/5 the tide 3/5 the vamps", "create_at": date("2012-06-17"), "description": "The Tide RTx4/favx2| Nate replied x1 RTx10/favx14| Drew repliedx3 RTx3/favx6| Con favx1| Tris replied fav/RTx1|Austin Mahone/Kendall Schmidt follow", "followers_count": 1047, "friends_count": 514, "statues_count": 21036 }, "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("2015-11-26T10:00:10.000Z"), "id": 669938710927048704, "text": "Today I'm spending Thanksgiving with a Jewish European family.\nI've spent it with black, white, arab and latino.\nMy track record so diverse.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 233998756, "name": "Sergeant Copper Lynx", "screen_name": "hottubjhonny", "lang": "en", "location": "Based World, NYC.", "create_at": date("2011-01-04"), "description": "Journalist. WHCS Radio Talk Show Host and News Anchor. Mixed Martial Artist. Comedian. TaskForce Member. Colombian Immigrant. The Daily Ramen", "followers_count": 1425, "friends_count": 1150, "statues_count": 58202 }, "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("2015-11-26T10:00:10.000Z"), "id": 669938710989946882, "text": "The Ezekiel Elliott E:60 gave me chills", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 316223284, "name": "Devan Henley™", "screen_name": "therealdevante", "lang": "en", "location": "null", "create_at": date("2011-06-12"), "description": "DeSalian -φΚΤ", "followers_count": 613, "friends_count": 714, "statues_count": 4583 }, "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-26T10:00:10.000Z"), "id": 669938711065329664, "text": "Oh gosh! Thanks Andrew:)) https://t.co/Dkh95z5lAN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2934128426, "name": "Delaney Bird", "screen_name": "delaney_bird", "lang": "en", "location": "Kansas, USA", "create_at": date("2014-12-17"), "description": "TA Track & XC", "followers_count": 211, "friends_count": 282, "statues_count": 397 }, "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("2015-11-26T10:00:10.000Z"), "id": 669938711925166080, "text": "@TobyKeithMusic fans check out our music & follow us @ryebrothers free music https://t.co/gmnZoHD34I #powercountry https://t.co/ocRS2U7XTx", "in_reply_to_status": -1, "in_reply_to_user": 25395727, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "powercountry" }}, "user_mentions": {{ 25395727, 3313151947 }}, "user": { "id": 3313151947, "name": "ryebrothers", "screen_name": "ryebrothers", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-08-11"), "description": "null", "followers_count": 392, "friends_count": 251, "statues_count": 2104 }, "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("2015-11-26T10:00:10.000Z"), "id": 669938711933513728, "text": "Happy thanksgiving to everyone ���� #happythanksgiving", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "happythanksgiving" }}, "user": { "id": 3909271153, "name": "Beth Vescio", "screen_name": "beth_vescio", "lang": "en", "location": "null", "create_at": date("2015-10-15"), "description": "I'm on a one way ticket to flavor town", "followers_count": 19, "friends_count": 81, "statues_count": 6 }, "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("2015-11-26T10:00:10.000Z"), "id": 669938711996559360, "text": "Happy thanksgiving from me and Ray!����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3245588103, "name": "⚾️Chase Townsend⚾️", "screen_name": "chaset_20", "lang": "en", "location": "Pike Co.⚾️", "create_at": date("2015-05-10"), "description": "•Phil 4:13 •Baseball was, is and always will be to me the best game in the world. -Babe Ruth", "followers_count": 257, "friends_count": 315, "statues_count": 332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jesup, GA", "id": "014ab1a57a92c51a", "name": "Jesup", "place_type": "city", "bounding_box": rectangle("-81.932367,31.556757 -81.859429,31.692481") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13305, "countyName": "Wayne", "cityID": 1342268, "cityName": "Jesup" } }
+{ "create_at": datetime("2015-11-26T10:00:10.000Z"), "id": 669938712378150913, "text": "I admire the God father", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 959764243, "name": "Jrod", "screen_name": "JarydSerna", "lang": "en", "location": "null", "create_at": date("2012-11-19"), "description": "null", "followers_count": 183, "friends_count": 305, "statues_count": 4413 }, "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-26T10:00:10.000Z"), "id": 669938712491372544, "text": "“If I ever run for office, I totally would use this logo.” - anonymous student\n\nCompliments feel good. #posterdesign https://t.co/wJKZpho3Ue", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "posterdesign" }}, "user": { "id": 182017315, "name": "Flex McCool", "screen_name": "DesignerFlex", "lang": "en", "location": "Manitowoc, WI", "create_at": date("2010-08-23"), "description": "Freelance #GraphicDesigner for the past 5 years, UWGB design arts graduate, #coffee lover, & aspiring nostalgic. Currently tackling #indiegame design.", "followers_count": 1023, "friends_count": 1125, "statues_count": 9149 }, "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("2015-11-26T10:00:10.000Z"), "id": 669938712663359488, "text": "���������������� @thehulkdavis why I see cast doing something like this ������ https://t.co/6EdWgqfx3X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2838251762 }}, "user": { "id": 1695714386, "name": "SpideyLew", "screen_name": "saivon_88v", "lang": "en", "location": "null", "create_at": date("2013-08-23"), "description": "1738 #ThrowuptheX", "followers_count": 553, "friends_count": 494, "statues_count": 6971 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merrydale, LA", "id": "a299620c865b2ba9", "name": "Merrydale", "place_type": "city", "bounding_box": rectangle("-91.13054,30.480556 -91.085408,30.518986") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2249940, "cityName": "Merrydale" } }
+{ "create_at": datetime("2015-11-26T10:00:10.000Z"), "id": 669938712822816768, "text": "@mnfortune16 what a shitty person..", "in_reply_to_status": 669922156642443264, "in_reply_to_user": 521680633, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 521680633 }}, "user": { "id": 98707668, "name": "hannabis", "screen_name": "_hanflo", "lang": "en", "location": "dirty thirty", "create_at": date("2009-12-22"), "description": "live free •scf•mnf•jkk•jrh• never forget •fc•jr•g•jv•", "followers_count": 840, "friends_count": 542, "statues_count": 16241 }, "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-26T10:00:10.000Z"), "id": 669938712990609408, "text": "ily ���� https://t.co/wyfrOq8gc4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1537774430, "name": "pony princess", "screen_name": "hannahestes_", "lang": "en", "location": "mercedes has a penis", "create_at": date("2013-06-21"), "description": "spoiled bitch who loves ponies // Jonah ❤️", "followers_count": 635, "friends_count": 413, "statues_count": 28023 }, "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-26T10:00:10.000Z"), "id": 669938713112129536, "text": "#Finance #Job in #Riverview, FL: Commercial Account Processing... at Bright House Networks https://t.co/yiES39bylK https://t.co/zkmMsKFCV2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.3144403,27.8539796"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Finance", "Job", "Riverview" }}, "user": { "id": 871791890, "name": "Bright House Careers", "screen_name": "JobsBrightHouse", "lang": "en", "location": "http://bit.ly/BHNJobMap", "create_at": date("2012-10-10"), "description": "Join the conversation and let us know what you are looking for. By joining our Talent Community, you can stay abreast of news about us and future job openings.", "followers_count": 896, "friends_count": 453, "statues_count": 206 }, "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("2015-11-26T10:00:10.000Z"), "id": 669938713120612352, "text": "@CiganyChris great idea!", "in_reply_to_status": 669938454390833152, "in_reply_to_user": 844411098, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 844411098 }}, "user": { "id": 2337481460, "name": "emotional shawty", "screen_name": "Nikkkkkkkkkkole", "lang": "en", "location": "North Ridgeville, OH", "create_at": date("2014-02-10"), "description": "Let's get together and feel alright. Capricorn ☯", "followers_count": 691, "friends_count": 463, "statues_count": 21549 }, "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("2015-11-26T10:00:10.000Z"), "id": 669938713338712064, "text": "I like it so bad it's very #Barbie.#fashion #fashionista #love #followme #followers #follow #moda… https://t.co/CdwSKjNIQK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0064,40.7142"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Barbie", "fashion", "fashionista", "love", "followme", "followers", "follow", "moda" }}, "user": { "id": 2974856664, "name": "iamsaveen", "screen_name": "Saveen99C", "lang": "en", "location": "Iraq", "create_at": date("2015-01-11"), "description": "singer | songwriter | dancer| fashion blogger | artistic| actor I'm against racism. proud to be g.a.y. alien. been bullied in school. this is how I born.", "followers_count": 80, "friends_count": 147, "statues_count": 1161 }, "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("2015-11-26T10:00:10.000Z"), "id": 669938713422598144, "text": "#happythanksgiving ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "happythanksgiving" }}, "user": { "id": 1031453863, "name": "❁ ya girl a ❁", "screen_name": "annuuhm_", "lang": "en", "location": "miami, fl", "create_at": date("2012-12-23"), "description": "// GTFG// ♡ 2fab4u ♡", "followers_count": 328, "friends_count": 287, "statues_count": 20381 }, "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("2015-11-26T10:00:10.000Z"), "id": 669938713447694336, "text": "Apparently I missed the memo where EVERYONE parties on Thanksgiving Eve.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 613006940, "name": "Ashley Lamb", "screen_name": "AshleyLLamb", "lang": "en", "location": "null", "create_at": date("2012-06-19"), "description": "Ashley Lynn. 21. Jeep life.", "followers_count": 165, "friends_count": 207, "statues_count": 7564 }, "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("2015-11-26T10:00:10.000Z"), "id": 669938713602846721, "text": "thankful for the text backs I get����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2823433650, "name": "laTrenty️", "screen_name": "iam_tjohnson21", "lang": "en", "location": "prolly at ya girl's house", "create_at": date("2014-09-20"), "description": "THS 21 || SC: iam_tjohnson21", "followers_count": 443, "friends_count": 353, "statues_count": 11523 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houma, LA", "id": "2a4c12f527197dab", "name": "Houma", "place_type": "city", "bounding_box": rectangle("-90.779993,29.544349 -90.649261,29.61995") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22109, "countyName": "Terrebonne", "cityID": 2236255, "cityName": "Houma" } }
+{ "create_at": datetime("2015-11-26T10:00:11.000Z"), "id": 669938714508816386, "text": "I wish i could just give you a hug and you tell you everything's gonna be alright princess", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2978075434, "name": "Roy Stetser", "screen_name": "StetserRoy", "lang": "en", "location": "null", "create_at": date("2015-01-14"), "description": "61215 ❤️ Jayla Marie Jaggard ❤️", "followers_count": 88, "friends_count": 122, "statues_count": 1782 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938714756251648, "text": "@ThatKetzyKid Oh no, I woke up late and started watching it too! lol", "in_reply_to_status": 669938611433836546, "in_reply_to_user": 248445955, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 248445955 }}, "user": { "id": 31577646, "name": "Genesis", "screen_name": "genmichellee", "lang": "en", "location": "Los Angeles", "create_at": date("2009-04-15"), "description": "I'm just as confused as you are", "followers_count": 1699, "friends_count": 1937, "statues_count": 60253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:11.000Z"), "id": 669938714869567488, "text": "my moms staring at the yams boiling and she goes \"they look like manatees\" #happythanksgiving", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "happythanksgiving" }}, "user": { "id": 389945387, "name": "Jenn", "screen_name": "jennalexisgrace", "lang": "en", "location": "Lafayette, CA", "create_at": date("2011-10-13"), "description": "don't be fooled by the rocks that i got, i'm still jenny from da block", "followers_count": 82, "friends_count": 184, "statues_count": 4387 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938714907451392, "text": "Happy Thanksgiving�� I'm thankful for all of yall❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 816206942, "name": "JaimieSTATESBelch ☀", "screen_name": "Beachgirrr127", "lang": "en", "location": "null", "create_at": date("2012-09-10"), "description": "WBHS Varsity cheerleader. Class of '16.", "followers_count": 684, "friends_count": 552, "statues_count": 6631 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938714953453569, "text": "alllllright. @ Inn at Riverbend https://t.co/ploJAZz5l6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.686142,37.3196106"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 224124467, "name": "Lauren Young", "screen_name": "lalalaurenyoung", "lang": "en", "location": "Nashville, TN", "create_at": date("2010-12-07"), "description": "{{I am a delightful minister of reconciliation, joyfully displaying redemption that others may find new life in Christ.}}", "followers_count": 680, "friends_count": 718, "statues_count": 4393 }, "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": 51071, "countyName": "Giles" } }
+{ "create_at": datetime("2015-11-26T10:00:11.000Z"), "id": 669938715003875328, "text": "This Thanksgiving I'm also grateful to Kevin Spacey for finally letting us know he's an unironic shill. https://t.co/Gz3AaG8Jb4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4808581, "name": "Alex Roy 144", "screen_name": "AlexRoy144", "lang": "en", "location": "New York, USA", "create_at": date("2007-04-15"), "description": "Driver. Author: The Driver. Host: DRIVE. CEO: Europe By Car. Columnist: Jalopnik. Veteran: The Moth. Morgan 3W, Citroen SM & 928 owner. NY-LA 31:04.", "followers_count": 5285, "friends_count": 1323, "statues_count": 2848 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938715016364032, "text": "All my feelings are tucked away inside a box labeled \"don't give af\" ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 374764713, "name": "good girl..", "screen_name": "_PrincessLI", "lang": "en", "location": "MKE✈️TX ", "create_at": date("2011-09-16"), "description": "pretty bird...", "followers_count": 825, "friends_count": 1869, "statues_count": 63965 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938715205238788, "text": "I been holding in another Twitter rant lol idk how much longer I can keep it in", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 98766484, "name": "✨Safiah✨", "screen_name": "Safiah11", "lang": "en", "location": "temporarily in the dunya", "create_at": date("2009-12-22"), "description": "Even a small lighter can burn a bridge | Instagram: Safiah11 | Vanderbilt University | ΣΛΓ | 20 | Singer | #Ethiopian | #AICP | صفية", "followers_count": 1431, "friends_count": 984, "statues_count": 12624 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938715213500420, "text": "Thankful for everybody I got. Happy Thanksgiving ������❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3278873210, "name": "Karah Wiseman", "screen_name": "KarahWiseman", "lang": "en", "location": "null", "create_at": date("2015-07-13"), "description": "RCHS Senior | CSUB ⚾️", "followers_count": 184, "friends_count": 235, "statues_count": 152 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938715284930561, "text": "come home for a break and catch nothing but extra drama.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 345954517, "name": "partygetsmewetter", "screen_name": "dvnihana_", "lang": "en", "location": "free squeak free curt", "create_at": date("2011-07-31"), "description": "keep em all on the need to know. #UMES19 all about loco & rashi ❤️", "followers_count": 5911, "friends_count": 6328, "statues_count": 130880 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938715360292864, "text": "@itsnayohmeee �������� #NewNew https://t.co/oaoSGjpAmT", "in_reply_to_status": -1, "in_reply_to_user": 2783562740, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "NewNew" }}, "user_mentions": {{ 2783562740 }}, "user": { "id": 1199961378, "name": "E m", "screen_name": "Tweet_Em_dead", "lang": "en", "location": "null", "create_at": date("2013-02-19"), "description": "nice watch. run it.", "followers_count": 269, "friends_count": 234, "statues_count": 10917 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938715448471553, "text": "https://t.co/31PK2IJGNm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2329151727, "name": "val", "screen_name": "_vrizzle", "lang": "en", "location": "TPA • ORL ", "create_at": date("2014-02-06"), "description": "nineteen", "followers_count": 513, "friends_count": 221, "statues_count": 6693 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938715553366017, "text": "I have purple hair", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 346326747, "name": "Lorin Migaiolo", "screen_name": "lorinleigh", "lang": "en", "location": "null", "create_at": date("2011-07-31"), "description": "How wild it was, to let it be.", "followers_count": 454, "friends_count": 242, "statues_count": 2488 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938715624652801, "text": "@vielmetti I do! I'm making some sort of vegan loaf", "in_reply_to_status": 669936689867943936, "in_reply_to_user": 9272, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 9272 }}, "user": { "id": 39909868, "name": "Brandon", "screen_name": "brandonzwa", "lang": "en", "location": "Ridgewood, Queens, NYC", "create_at": date("2009-05-13"), "description": "null", "followers_count": 1285, "friends_count": 1802, "statues_count": 34069 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938715662356480, "text": "Thankful for my baby Yung Poc❤ https://t.co/WPYh21V1t8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2740359328, "name": "Heidi Grossnickle", "screen_name": "heidi_louise3", "lang": "en", "location": "Virginia, USA", "create_at": date("2014-08-11"), "description": "LCB #3 Conference Champs 13 & 15 PreNursing EMU volleyball '20", "followers_count": 357, "friends_count": 323, "statues_count": 8576 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nags Head, NC", "id": "00a9dd516d11c0b7", "name": "Nags Head", "place_type": "city", "bounding_box": rectangle("-75.670276,35.84598 -75.562843,35.995022") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37055, "countyName": "Dare", "cityID": 3745880, "cityName": "Nags Head" } }
+{ "create_at": datetime("2015-11-26T10:00:11.000Z"), "id": 669938715691606016, "text": "today im thankful i have a car to drive . i would be even more thankful if Sky Harbor Airport Pakring would pick me TF UP so i can drive it��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 162102905, "name": "Trill Tiff", "screen_name": "mrstiffanyanne", "lang": "en", "location": "INSTAGRAM: @mrstiffanyanne", "create_at": date("2010-07-02"), "description": "❤️✈️", "followers_count": 4670, "friends_count": 2650, "statues_count": 156417 }, "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-26T10:00:11.000Z"), "id": 669938715813355521, "text": "@arvongordon4 @spitta_jay @KendallSoFly @TC_Doeeee @XxTahManxx @Jj_algarin05 I'm thankful for y'all too nbs", "in_reply_to_status": 669938262216212480, "in_reply_to_user": 1280819394, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1280819394, 383471797, 406731967, 2975256354, 161516619, 630672442 }}, "user": { "id": 3132016103, "name": "Bhvrlez Bvpone ⚠️", "screen_name": "Charlezboy21", "lang": "en", "location": "null", "create_at": date("2015-04-03"), "description": "Take no game for granted cause you never know when the ball will stop bouncing for you #MikeVazStrong #StateChampInTheMaking #21 #VarsityBasketball", "followers_count": 305, "friends_count": 284, "statues_count": 4485 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938715834339328, "text": "Wow that was a Stafford window smh and Pettigrew drops that hoe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 236532495, "name": "jorge esmieu", "screen_name": "JorgeEsmieu", "lang": "en", "location": "Damascus", "create_at": date("2011-01-10"), "description": "class of '17/ P.M. ❤️", "followers_count": 589, "friends_count": 601, "statues_count": 24060 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Damascus, MD", "id": "45a1bbc1771f8ca3", "name": "Damascus", "place_type": "city", "bounding_box": rectangle("-77.25614,39.229003 -77.149609,39.315707") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2421475, "cityName": "Damascus" } }
+{ "create_at": datetime("2015-11-26T10:00:11.000Z"), "id": 669938715884716033, "text": "PLEASE �������� https://t.co/zRSqFhxU36", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2728746545, "name": "Nini Chester", "screen_name": "_asvpxenniaa", "lang": "en", "location": "null", "create_at": date("2014-07-29"), "description": "Confidence is key", "followers_count": 366, "friends_count": 343, "statues_count": 1250 }, "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-26T10:00:11.000Z"), "id": 669938716127924225, "text": "This world is disgusting", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 230791566, "name": "The Real Randi Rozay", "screen_name": "randiROOOZAY", "lang": "en", "location": "PHILLY", "create_at": date("2010-12-26"), "description": "null", "followers_count": 3177, "friends_count": 914, "statues_count": 31865 }, "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-26T10:00:11.000Z"), "id": 669938716480245760, "text": "Ilyt❤️ https://t.co/R8EvLZ1YBm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "hu", "is_retweet": false, "user": { "id": 2733192680, "name": "Abbie", "screen_name": "Abbie1024", "lang": "en", "location": "null", "create_at": date("2014-08-14"), "description": "null", "followers_count": 132, "friends_count": 134, "statues_count": 2287 }, "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.957039") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1214895, "cityName": "Country Club" } }
+{ "create_at": datetime("2015-11-26T10:00:11.000Z"), "id": 669938716564168704, "text": "No Ceilings 2 better be good.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 987738534, "name": "Miles", "screen_name": "blessedvirgo16", "lang": "en", "location": "CT ", "create_at": date("2012-12-03"), "description": "Future psychologist living life to the fullest. IG: blessedvirgo16 Psn: real_milesking", "followers_count": 702, "friends_count": 650, "statues_count": 22866 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wethersfield, CT", "id": "0e97d4804c3d8363", "name": "Wethersfield", "place_type": "city", "bounding_box": rectangle("-72.708224,41.66877 -72.617316,41.729879") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 984970, "cityName": "Wethersfield" } }
+{ "create_at": datetime("2015-11-26T10:00:11.000Z"), "id": 669938716949934080, "text": "thankful for all of these lovely people & all of you lovely people https://t.co/Sk4XHXS6AP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 384661709, "name": "miki ratsula", "screen_name": "mikiratsula", "lang": "en", "location": "North Tustin, CA", "create_at": date("2011-10-03"), "description": "''We all have the ability to inspire, @mikiratsula! Your music is powerful, beautiful, and inspiring as well. Thank you!'' - @hopesolo | 4.24.15", "followers_count": 716, "friends_count": 538, "statues_count": 10004 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938717067382784, "text": "@Mateeeen https://t.co/0CG3jkqqdT", "in_reply_to_status": -1, "in_reply_to_user": 36524003, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 36524003 }}, "user": { "id": 69189603, "name": "Ali Shamim", "screen_name": "alishmee", "lang": "en", "location": "Los Angeles", "create_at": date("2009-08-26"), "description": "Hopefully this 5am update will tire me and I can fall back asleep", "followers_count": 135, "friends_count": 226, "statues_count": 4497 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938717189099524, "text": "90 percent of guys look like that but hey boo cushion for pushin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 420698210, "name": "Bryan Shimeld", "screen_name": "Bryanmyhero", "lang": "en", "location": "Ct", "create_at": date("2011-11-24"), "description": "Enfield, Ct Aerospace Repair Technician for UTC Aerospace Systems. \nI work out as a hobby for the satisfaction for myself..", "followers_count": 27, "friends_count": 90, "statues_count": 805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hazardville, CT", "id": "123157afadcf9b33", "name": "Hazardville", "place_type": "city", "bounding_box": rectangle("-72.571279,41.977215 -72.499397,42.006796") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 937770, "cityName": "Hazardville" } }
+{ "create_at": datetime("2015-11-26T10:00:11.000Z"), "id": 669938717302370304, "text": "@saramoran22 https://t.co/T8Nl2AV68G", "in_reply_to_status": -1, "in_reply_to_user": 1535682432, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1535682432 }}, "user": { "id": 2157971292, "name": "kearra:)))", "screen_name": "kearra_nicole", "lang": "en", "location": "TN", "create_at": date("2013-10-26"), "description": "don't try & save her she don't want to be saved.// hebrews 6:19", "followers_count": 456, "friends_count": 763, "statues_count": 6054 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938717331714048, "text": "#GeneralDisturbance at 101-298 Lynbrook Dr, Azalea Park, FL 32807. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2966396,28.5421474"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GeneralDisturbance", "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": 189, "friends_count": 1, "statues_count": 32238 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938717361090560, "text": "My mom cooked everything yo ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3387919181, "name": "Only1Noonie", "screen_name": "edd_noon", "lang": "en", "location": "null", "create_at": date("2015-07-22"), "description": "LiveLoveLil;", "followers_count": 171, "friends_count": 120, "statues_count": 4442 }, "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-26T10:00:11.000Z"), "id": 669938717780504576, "text": "@PirepC", "in_reply_to_status": 663838755397079040, "in_reply_to_user": 3213429889, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3255028895 }}, "user": { "id": 3213429889, "name": "Huge gamer AP✌", "screen_name": "99725659e0e546d", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2015-05-17"), "description": "Sort of active", "followers_count": 168, "friends_count": 352, "statues_count": 138 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938718116065280, "text": "My uncle started all lions players today on his fantasy team����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 730863174, "name": "_AMG", "screen_name": "angelogrillo7", "lang": "en", "location": "#17⚽️", "create_at": date("2012-08-01"), "description": "DLS Senior, Nova U18s", "followers_count": 776, "friends_count": 561, "statues_count": 16086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shelby, MI", "id": "5c8c81a4fef94364", "name": "Shelby", "place_type": "city", "bounding_box": rectangle("-83.095428,42.623997 -82.973298,42.715777") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26127, "countyName": "Oceana", "cityID": 2672840, "cityName": "Shelby" } }
+{ "create_at": datetime("2015-11-26T10:00:11.000Z"), "id": 669938718267023360, "text": "dona da comedia é assim mesmo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 1335059035, "name": "fernando", "screen_name": "heartedkaty", "lang": "pt", "location": "Windermere, FL", "create_at": date("2013-04-07"), "description": "DELET IT FAT | PWT 6/30/14 https://twitter.com/heartedkaty/status/584889311146958848", "followers_count": 3161, "friends_count": 578, "statues_count": 158664 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938718350839808, "text": "Let's go Detroit!��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 730739862, "name": "Jordannn.", "screen_name": "jWreck10", "lang": "en", "location": "null", "create_at": date("2012-08-01"), "description": "|OTF||SnapFitnessAssociate| |GymIsLyfe| |IHaveTheGreatestDogEver| |SC:Jordann_19|", "followers_count": 1735, "friends_count": 591, "statues_count": 25971 }, "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("2015-11-26T10:00:11.000Z"), "id": 669938718359158784, "text": "Afternoon thunderstorms this afternoon, high 74 (23 C). Low 68 (20 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712472, "name": "AustinWX", "screen_name": "AustinWX", "lang": "en", "location": "Austin, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Austin, Texas", "followers_count": 1010, "friends_count": 1367, "statues_count": 7901 }, "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-26T10:00:11.000Z"), "id": 669938718380281856, "text": "#CheckWellBeing at 7701-7798 Prato Ave. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.447171,28.4527481"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CheckWellBeing", "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": 264, "friends_count": 1, "statues_count": 65026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tangelo Park, FL", "id": "55a0058817d827cd", "name": "Tangelo Park", "place_type": "city", "bounding_box": rectangle("-81.450684,28.451994 -81.441998,28.461362") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1271075, "cityName": "Tangelo Park" } }
+{ "create_at": datetime("2015-11-26T10:00:11.000Z"), "id": 669938718434705408, "text": "Thankful for yall ������ @brunkreygan https://t.co/u8DUeXZl3r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2654950711 }}, "user": { "id": 1939735843, "name": "Alyssa", "screen_name": "aleighblackburn", "lang": "en", "location": "Iowa, USA", "create_at": date("2013-10-05"), "description": "justin bieber", "followers_count": 388, "friends_count": 361, "statues_count": 11355 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Urbandale, IA", "id": "85ec017be8926e26", "name": "Urbandale", "place_type": "city", "bounding_box": rectangle("-93.814689,41.614643 -93.697511,41.658733") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1979950, "cityName": "Urbandale" } }
+{ "create_at": datetime("2015-11-26T10:00:12.000Z"), "id": 669938718476779521, "text": "@DougCruces @KaboomKrusader Wow.", "in_reply_to_status": 669938620900491266, "in_reply_to_user": 438176707, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 438176707, 21727217 }}, "user": { "id": 15962154, "name": "Tanooki Kuribo", "screen_name": "TanookiKuribo", "lang": "en", "location": "Brooklyn/Staten Island", "create_at": date("2008-08-23"), "description": "null", "followers_count": 812, "friends_count": 197, "statues_count": 41725 }, "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-26T10:00:12.000Z"), "id": 669938718787026945, "text": "#happythanksgiving #clients #friends #family #bjorkstudio @ Bjork Studio https://t.co/NnLwtjiFXn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.4254532,33.7999992"), "retweet_count": 0, "lang": "no", "is_retweet": false, "hashtags": {{ "happythanksgiving", "clients", "friends", "family", "bjorkstudio" }}, "user": { "id": 21921516, "name": "BJORK STUDIO", "screen_name": "bjorkstudio", "lang": "en", "location": "Atlanta, GA, USA", "create_at": date("2009-02-25"), "description": "Fine bench-made furniture, custom upholstery atelier - along with vintage Scandinavian furnishings. A marriage of culture and style made in Atlanta, GA, USA!", "followers_count": 949, "friends_count": 1304, "statues_count": 5908 }, "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("2015-11-26T10:00:12.000Z"), "id": 669938719021907972, "text": "Me: \"Dave are you still drunk?\" Uncle Dave: \"I probably am\"��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 377110434, "name": "Kelli Bradshaw", "screen_name": "KelliBradshaw1", "lang": "en", "location": "Carlisle, Iowa", "create_at": date("2011-09-20"), "description": "Life is short. Make it great. Let nothing take you down. ~Sophia Bush", "followers_count": 510, "friends_count": 445, "statues_count": 9387 }, "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("2015-11-26T10:00:12.000Z"), "id": 669938719030251521, "text": "In the dirty South, thankful I got this strap 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": 599367536, "name": "ㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤ", "screen_name": "Davon_Caldwell7", "lang": "en", "location": "null", "create_at": date("2012-06-04"), "description": "I DO ALPHA-MALE SHIT | Don't call it impossible, if you really want that! #KING #777", "followers_count": 1130, "friends_count": 922, "statues_count": 49672 }, "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("2015-11-26T10:00:12.000Z"), "id": 669938719072219138, "text": "We're picture perfect in a broken frame", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3158872045, "name": "Alanah Laupua", "screen_name": "LaupuaAlanah", "lang": "en", "location": "SC: Alanahmarie23 ", "create_at": date("2015-04-15"), "description": "CSU Stanislaus softball", "followers_count": 270, "friends_count": 270, "statues_count": 929 }, "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-26T10:00:12.000Z"), "id": 669938719097536512, "text": "@bonnieux they're the #shithole on the #Dick? Blehg! Rather eat #McDonalds. #JerseyBlood", "in_reply_to_status": 669937730630098944, "in_reply_to_user": 55727745, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "shithole", "Dick", "McDonalds", "JerseyBlood" }}, "user_mentions": {{ 55727745 }}, "user": { "id": 23790825, "name": "Mike Law", "screen_name": "radioday", "lang": "en", "location": "New York", "create_at": date("2009-03-11"), "description": "I'll have my music, and listen too. http://www.last.fm/user/radioday", "followers_count": 408, "friends_count": 1991, "statues_count": 19678 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Depew, NY", "id": "956fd3d6e2d45da8", "name": "Depew", "place_type": "city", "bounding_box": rectangle("-78.739947,42.892376 -78.676988,42.929405") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3620313, "cityName": "Depew" } }
+{ "create_at": datetime("2015-11-26T10:00:12.000Z"), "id": 669938719126896640, "text": "@23_Suave mockingjay part 2?", "in_reply_to_status": 669938527422062592, "in_reply_to_user": 76431711, "favorite_count": 0, "retweet_count": 0, "lang": "hu", "is_retweet": false, "user_mentions": {{ 76431711 }}, "user": { "id": 406673417, "name": "#Sovereign", "screen_name": "GlooSR", "lang": "en", "location": "Nashay&Jr", "create_at": date("2011-11-06"), "description": "faith is the ability not to panic .", "followers_count": 1410, "friends_count": 589, "statues_count": 108167 }, "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("2015-11-26T10:00:12.000Z"), "id": 669938719265165312, "text": "@thewizardofRJ haha u in Austin ����", "in_reply_to_status": 669934770625753088, "in_reply_to_user": 2604594628, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 2604594628 }}, "user": { "id": 1279087501, "name": "Simply just..B✨❤️", "screen_name": "Brazilianbb", "lang": "en", "location": "Texas, USA", "create_at": date("2013-03-18"), "description": "Taurus♉ just remeber who kept it trill✨✊TEXAS RUNNER✊✨", "followers_count": 1171, "friends_count": 2009, "statues_count": 15331 }, "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("2015-11-26T10:00:12.000Z"), "id": 669938719303049216, "text": "@JuiceDodd_32 @savvvy_t don't be jealous", "in_reply_to_status": 669938632547893248, "in_reply_to_user": 1222831338, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1222831338, 277837557 }}, "user": { "id": 2248064629, "name": "Hannah Marie-Rose", "screen_name": "hanhutch101", "lang": "en", "location": "Holland, MI", "create_at": date("2013-12-15"), "description": "Grand Valley State University ⚓️ '19. Bio / PA", "followers_count": 1155, "friends_count": 843, "statues_count": 6132 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Holland, MI", "id": "ee258f4bb56d1711", "name": "Holland", "place_type": "city", "bounding_box": rectangle("-86.197409,42.728541 -86.022493,42.802658") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26139, "countyName": "Ottawa", "cityID": 2638640, "cityName": "Holland" } }
+{ "create_at": datetime("2015-11-26T10:00:12.000Z"), "id": 669938719340797952, "text": "80 mil https://t.co/c3Vp1XanMA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 67819141, "name": "Murphy Two Times", "screen_name": "Murphy2x", "lang": "en", "location": "Pensacola, FL", "create_at": date("2009-08-21"), "description": "Basquiat out now. Troy University. I'm not ignorant, you're just stupid. Logical thinker. #TroyU19", "followers_count": 4637, "friends_count": 1059, "statues_count": 65212 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gulf Shores, AL", "id": "dd4fb0ab4e6cd837", "name": "Gulf Shores", "place_type": "city", "bounding_box": rectangle("-87.753372,30.239427 -87.659759,30.334228") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1003, "countyName": "Baldwin", "cityID": 132272, "cityName": "Gulf Shores" } }
+{ "create_at": datetime("2015-11-26T10:00:12.000Z"), "id": 669938719567187969, "text": "@BigBrother_PM ��������������������������������������������������������������������������������������������������", "in_reply_to_status": 669937495728001025, "in_reply_to_user": 3074431417, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3074431417 }}, "user": { "id": 113558477, "name": "Valle", "screen_name": "Coccoth", "lang": "es", "location": "Arizona", "create_at": date("2010-02-11"), "description": "I luv my f@ckin ENGLISH y Mi Excelente ESPAÑOL!!! BTW I'm happy everyday ✌", "followers_count": 43, "friends_count": 43, "statues_count": 570 }, "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-26T10:00:12.000Z"), "id": 669938719625838592, "text": "@_RuthlessBeauty hmmmm", "in_reply_to_status": 669938545935564800, "in_reply_to_user": 614870497, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 614870497 }}, "user": { "id": 249664931, "name": "Ed Williams", "screen_name": "24toocold", "lang": "en", "location": "Dallas, TX", "create_at": date("2011-02-09"), "description": "|#24 Eastfield Hoops | KG | RIP Arlena Johnson | 22 | NLMB | Humble Beast", "followers_count": 3230, "friends_count": 2624, "statues_count": 197691 }, "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("2015-11-26T10:00:12.000Z"), "id": 669938719680393216, "text": "@AWall925 I think they do", "in_reply_to_status": 669191920871043072, "in_reply_to_user": 1593659732, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1593659732 }}, "user": { "id": 411296842, "name": "Austin", "screen_name": "austinjckson", "lang": "en", "location": "Houston, Texas", "create_at": date("2011-11-12"), "description": "Student developer and designer. ✞\nContact: austinjckson@gmail.com", "followers_count": 291, "friends_count": 323, "statues_count": 11427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:12.000Z"), "id": 669938719785381888, "text": "Byron Maxwell's pathetic attempt at ripping the football away after the play has been dead for 5 seconds it's starting to get real annoying.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36582529, "name": "Michael DiMaggio", "screen_name": "M_DiMaggio30", "lang": "en", "location": "Philadelphia", "create_at": date("2009-04-29"), "description": "Minor setbacks make for Major comebacks... Nobodies built like you, you design yourself. #TempleUniversity #570 - #215", "followers_count": 537, "friends_count": 1160, "statues_count": 13198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inkerman, PA", "id": "006827a714c0bbb4", "name": "Inkerman", "place_type": "city", "bounding_box": rectangle("-75.829731,41.288994 -75.806049,41.303593") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42079, "countyName": "Luzerne", "cityID": 4237008, "cityName": "Inkerman" } }
+{ "create_at": datetime("2015-11-26T10:00:12.000Z"), "id": 669938719932059648, "text": "This is how it's done! https://t.co/WYItD0v9Rk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 10104242, "name": "Tim Krajcar", "screen_name": "TimKrajcar", "lang": "en", "location": "Portland, OR", "create_at": date("2007-11-09"), "description": "Ruby agent manager at @newrelic, co-founder of two early-stage daughters, fan of flightsims and multiplayer text-based games, probably should go outside more.", "followers_count": 524, "friends_count": 815, "statues_count": 4156 }, "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("2015-11-26T10:00:12.000Z"), "id": 669938719994961920, "text": "Best part of thanksgiving with my moms side is all the babies!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 548471800, "name": "Carl Coster", "screen_name": "LennyCoster", "lang": "en", "location": "null", "create_at": date("2012-04-08"), "description": "null", "followers_count": 206, "friends_count": 226, "statues_count": 5742 }, "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("2015-11-26T10:00:12.000Z"), "id": 669938720154501120, "text": "@citizensrock @kasinca @hillaryclinton was a Goldwater girl. https://t.co/zoIeR2zF2u", "in_reply_to_status": 669938259233890305, "in_reply_to_user": 271030180, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 271030180, 24053369, 1339835893 }}, "user": { "id": 22666333, "name": "Shizzle the Land", "screen_name": "Shizzletheland", "lang": "en", "location": "somewhere 'round #CLE", "create_at": date("2009-03-03"), "description": "Doing what I was born to do.", "followers_count": 1089, "friends_count": 1865, "statues_count": 14425 }, "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("2015-11-26T10:00:12.000Z"), "id": 669938720406007808, "text": "@sarah_vonn I knooow ����", "in_reply_to_status": 669780429038194688, "in_reply_to_user": 317570522, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 317570522 }}, "user": { "id": 1247744252, "name": "ૐ❂ ▴", "screen_name": "__sammerz", "lang": "en", "location": "null", "create_at": date("2013-03-06"), "description": "18 | arc softball⚾️ | zh❤️", "followers_count": 1281, "friends_count": 846, "statues_count": 27447 }, "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.702268 -121.30897,38.728927") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 602210, "cityName": "Antelope" } }
+{ "create_at": datetime("2015-11-26T10:00:12.000Z"), "id": 669938720477319172, "text": "Happy thanks giving... ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2331045902, "name": "Angelito D. Chua III", "screen_name": "HomicideHank44", "lang": "en", "location": "Malabon City", "create_at": date("2014-02-06"), "description": "Boxing fanatic, Shoe lover, Underground Hip-hop, Philippine represent.", "followers_count": 127, "friends_count": 446, "statues_count": 134 }, "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("2015-11-26T10:00:12.000Z"), "id": 669938720628277248, "text": "Guau bb veo q ya estas lista para meterte toda mi bella verguita asta el fondo de tu bella panochita y en tu culito https://t.co/uFHUN5qMAu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 3195665928, "name": "gabino rey", "screen_name": "ReyGabino", "lang": "en", "location": "null", "create_at": date("2015-05-14"), "description": "null", "followers_count": 391, "friends_count": 1360, "statues_count": 23065 }, "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("2015-11-26T10:00:12.000Z"), "id": 669938721249095681, "text": "@BudweiserQueen_ come visit me at work", "in_reply_to_status": 669938018078359552, "in_reply_to_user": 2848995766, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2848995766 }}, "user": { "id": 2266637359, "name": "Crazy Cat Lady", "screen_name": "crzycatldyy", "lang": "en", "location": "Texas ❤", "create_at": date("2013-12-28"), "description": "I tweet about my boyfriend and 3 fur babies a lot. warr;or, still fighting.", "followers_count": 1403, "friends_count": 1044, "statues_count": 32429 }, "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("2015-11-26T10:00:12.000Z"), "id": 669938721299435520, "text": "im thankful for the ones that are always here for me , thankful for the ones that never turned their backs on me ��. #thankfulforyall", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thankfulforyall" }}, "user": { "id": 4310281272, "name": "mml.bitch", "screen_name": "mylalopez_x", "lang": "en", "location": "null", "create_at": date("2015-11-21"), "description": "null", "followers_count": 7, "friends_count": 2, "statues_count": 6 }, "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("2015-11-26T10:00:12.000Z"), "id": 669938721362341888, "text": "This thanksgiving is so hard to not have my brother home with us", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 359158318, "name": "tano", "screen_name": "abbiemontano6", "lang": "en", "location": "Tooele, UT", "create_at": date("2011-08-20"), "description": "THS | volleyball | softball | @Casey_roberts22", "followers_count": 520, "friends_count": 361, "statues_count": 4894 }, "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("2015-11-26T10:00:12.000Z"), "id": 669938721878224896, "text": "@detox_holla", "in_reply_to_status": 669938539711197184, "in_reply_to_user": 541420481, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 434692517 }}, "user": { "id": 541420481, "name": "B", "screen_name": "bowennmckenzie", "lang": "en", "location": "null", "create_at": date("2012-03-30"), "description": "whooooo disss", "followers_count": 1179, "friends_count": 627, "statues_count": 23496 }, "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("2015-11-26T10:00:12.000Z"), "id": 669938722104860672, "text": "@penutbutterkid @TheSmokeSpots @PeytonMercier", "in_reply_to_status": 669938067759730688, "in_reply_to_user": 3256182960, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3256182960, 2764153647, 499990476 }}, "user": { "id": 2869931532, "name": "Dylan Gibbons", "screen_name": "heyitsdyl", "lang": "en", "location": "null", "create_at": date("2014-10-21"), "description": "i'm kind of a big deal.", "followers_count": 173, "friends_count": 263, "statues_count": 332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Bellmore, NY", "id": "ea82a33637cabb81", "name": "North Bellmore", "place_type": "city", "bounding_box": rectangle("-73.560023,40.67418 -73.52109,40.706377") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3651517, "cityName": "North Bellmore" } }
+{ "create_at": datetime("2015-11-26T10:00:12.000Z"), "id": 669938722121609216, "text": "@Uptomyknees Birdman is one of my favorite movies of last year but it was divisive, so I'm not gonna fight people who didn't like it.", "in_reply_to_status": 669937668629749760, "in_reply_to_user": 83112392, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 83112392 }}, "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": 835, "friends_count": 987, "statues_count": 42654 }, "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("2015-11-26T10:00:12.000Z"), "id": 669938722146619392, "text": "#happythanksgiving #livelaughlove https://t.co/E0nBLzIQkE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "happythanksgiving", "livelaughlove" }}, "user": { "id": 3331753239, "name": "Barbara Gustafson", "screen_name": "Stee7er_gurl", "lang": "en", "location": "Fountain Hills, AZ", "create_at": date("2015-06-17"), "description": "I'm not just any girl. I'm a STEELERS GIRL", "followers_count": 45, "friends_count": 108, "statues_count": 466 }, "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("2015-11-26T10:00:12.000Z"), "id": 669938722310397953, "text": "@RVDICALmusic ok :) don't have to be rude", "in_reply_to_status": 669937888755367936, "in_reply_to_user": 491480409, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 491480409 }}, "user": { "id": 491690289, "name": "em ❉", "screen_name": "alyse_emily", "lang": "en", "location": "somewhere with pugs ", "create_at": date("2012-02-13"), "description": "19.☽☼☾ (^^^) sc: emily_alyse ❃ j ❥❥", "followers_count": 1142, "friends_count": 814, "statues_count": 27610 }, "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("2015-11-26T10:00:12.000Z"), "id": 669938722436022272, "text": "I really don't know what's wrong with me lol ..I'm in my feelings hard but idk why", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 364399798, "name": "mia.", "screen_name": "laaaalaa_", "lang": "en", "location": "null", "create_at": date("2011-08-29"), "description": "null", "followers_count": 459, "friends_count": 354, "statues_count": 23415 }, "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("2015-11-26T10:00:12.000Z"), "id": 669938722545111040, "text": "#happythanksgiving @ Edgewater Hotel Casino https://t.co/cfZI9feU2f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-114.57189376,35.16095293"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "happythanksgiving" }}, "user": { "id": 1442064451, "name": "AMommyOnTheMove", "screen_name": "christina_hauff", "lang": "en", "location": "null", "create_at": date("2013-05-19"), "description": "Traveler, Explorer, Student, Teacher, MOM", "followers_count": 23, "friends_count": 49, "statues_count": 658 }, "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": 3241000, "cityName": "Laughlin" } }
+{ "create_at": datetime("2015-11-26T10:00:12.000Z"), "id": 669938722549456897, "text": "Ayo https://t.co/q0esl6qfLK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 32704484, "name": "nov.20 ♥", "screen_name": "nyalauryn", "lang": "en", "location": "Philly", "create_at": date("2009-04-17"), "description": "Guyanese❤️Bestfriend.@i_killfuckboys", "followers_count": 2807, "friends_count": 2796, "statues_count": 14846 }, "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-26T10:00:12.000Z"), "id": 669938722641571840, "text": "������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 468011010, "name": "Μιχαήλ Katsos", "screen_name": "mike_katsos1", "lang": "en", "location": "Boston, MA•CO'16-LEHS ", "create_at": date("2012-01-18"), "description": "Sweeeds' going to sleep for a while | 05 Expedition ~ SNAP: mike_katsos1 #FlyHighAshley❤️", "followers_count": 666, "friends_count": 1112, "statues_count": 18675 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938723354714112, "text": "crazy to think the holidays next year will be a lot different", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 580453806, "name": "Jennifer♡", "screen_name": "_Jen_Dube", "lang": "en", "location": "null", "create_at": date("2012-05-14"), "description": "null", "followers_count": 705, "friends_count": 386, "statues_count": 8765 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938723442655232, "text": "@snbradywriter *rereads tweet*\n\nI type well. I type very well.", "in_reply_to_status": 669938237994045440, "in_reply_to_user": 2975163840, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2975163840 }}, "user": { "id": 104387738, "name": "Thom", "screen_name": "thomicks", "lang": "en", "location": "Modesto, CA", "create_at": date("2010-01-12"), "description": "I run a guild. I'm gonna teach maths. I do #BlizzardKeets and made that #FriendshipMoose thing. Banner by @Whammo_", "followers_count": 1451, "friends_count": 708, "statues_count": 20896 }, "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-26T10:00:13.000Z"), "id": 669938723610472449, "text": "Afternoon thunderstorms this afternoon, high 72 (22 C). Low 56 (13 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575081, "name": "San Angelo Weather", "screen_name": "SanAngeloWX", "lang": "en", "location": "San Angelo, TX", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for San Angelo, TX", "followers_count": 1918, "friends_count": 92, "statues_count": 7800 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938723652440064, "text": "Might be in Canada during the Christmas break with my fam either Toronto or Montreal ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 102524976, "name": "Keñley", "screen_name": "KenleyPurp", "lang": "en", "location": "HTX.181 ✈️✈️ Kingsville ", "create_at": date("2010-01-06"), "description": "TAMUK'18 Track & Field . . . SC: Kenley4", "followers_count": 1425, "friends_count": 1326, "statues_count": 40149 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:13.000Z"), "id": 669938723828703232, "text": "When u give ur uncle the nod so you dont have to talk to relatives 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": 778545175, "name": "cole lepkowski", "screen_name": "colelep25", "lang": "en", "location": "null", "create_at": date("2012-08-24"), "description": "null", "followers_count": 361, "friends_count": 426, "statues_count": 904 }, "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-26T10:00:13.000Z"), "id": 669938723832774656, "text": "I'm v thankful for Mariah (@izaiahjoaquin's little sister) bc she's my favorite person ever ���� https://t.co/P4z6TbB3EA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1665985608 }}, "user": { "id": 1607480544, "name": "Juicy Jaycee", "screen_name": "Jaayylmao_", "lang": "en", "location": "bay area", "create_at": date("2013-07-19"), "description": "tired", "followers_count": 603, "friends_count": 574, "statues_count": 18316 }, "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-26T10:00:13.000Z"), "id": 669938723899858944, "text": "my appetite honestly doesn't change on Thanksgiving", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 520097833, "name": "eb", "screen_name": "ebonynforbes", "lang": "en", "location": "null", "create_at": date("2012-03-09"), "description": "cen. tx #teammarcus", "followers_count": 419, "friends_count": 321, "statues_count": 3764 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938723979530240, "text": "See our latest #Kahului, HI #job and click to apply: Load Puller - https://t.co/kQGzQbJKWF #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-156.47,20.8947222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Kahului", "job", "Hiring" }}, "user": { "id": 38082277, "name": "TMJ-MAU Jobs", "screen_name": "tmj_mau_jobs", "lang": "en", "location": "Maui, HI", "create_at": date("2009-05-05"), "description": "Follow this account for geo-targeted Other job tweets in Maui, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 545, "friends_count": 535, "statues_count": 16 }, "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", "cityID": 1522700, "cityName": "Kahului" } }
+{ "create_at": datetime("2015-11-26T10:00:13.000Z"), "id": 669938724109549568, "text": "Wind 2.0 mph NNW. Barometer 30.150 in, Rising. Temperature 41.0 °F. Rain today 0.01 in. Humidity 59%", "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": 192, "friends_count": 58, "statues_count": 229082 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938724185206784, "text": "@Always_DFTC love you suga ��", "in_reply_to_status": 669938640286519300, "in_reply_to_user": 24971995, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24971995 }}, "user": { "id": 377980158, "name": "hilldog.", "screen_name": "aside0fbac0n", "lang": "en", "location": "Bacon Island, MI", "create_at": date("2011-09-22"), "description": "pitbull mom | good vibe tribe | kd fighter | twentytwo |", "followers_count": 1257, "friends_count": 1764, "statues_count": 19637 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durand, MI", "id": "5c14e13491807c2c", "name": "Durand", "place_type": "city", "bounding_box": rectangle("-84.011607,42.900403 -83.972297,42.928967") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26155, "countyName": "Shiawassee", "cityID": 2623500, "cityName": "Durand" } }
+{ "create_at": datetime("2015-11-26T10:00:13.000Z"), "id": 669938724231372800, "text": "@_nicolelol love u��", "in_reply_to_status": 669938628156653568, "in_reply_to_user": 107621326, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 107621326 }}, "user": { "id": 120769030, "name": "legs", "screen_name": "jasmeanieeee", "lang": "en", "location": "Miami, FL", "create_at": date("2010-03-07"), "description": "my own meal ticket", "followers_count": 1523, "friends_count": 792, "statues_count": 42216 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938724252221440, "text": "everybody gone be everywhere tryna do everything", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209299550, "name": "Stevie Lakes", "screen_name": "_missrivers", "lang": "en", "location": "11:11", "create_at": date("2010-10-28"), "description": "| work harder than who works the HARDEST | love God . st louis", "followers_count": 1133, "friends_count": 878, "statues_count": 35483 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bel-Ridge, MO", "id": "7c663cbe46f4a8b2", "name": "Bel-Ridge", "place_type": "city", "bounding_box": rectangle("-90.336687,38.705022 -90.310458,38.722249") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2904366, "cityName": "Bel-Ridge" } }
+{ "create_at": datetime("2015-11-26T10:00:13.000Z"), "id": 669938724441088000, "text": "I can't wait to eat ten pounds of mashed potatoes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 542166966, "name": "chels ☯", "screen_name": "chelssgraham", "lang": "en", "location": "null", "create_at": date("2012-03-31"), "description": "it's not that bad", "followers_count": 658, "friends_count": 420, "statues_count": 25758 }, "place": { "country": "United States", "country_code": "United States", "full_name": "White Horse, NJ", "id": "001f79723d2f975e", "name": "White Horse", "place_type": "city", "bounding_box": rectangle("-74.749703,40.148224 -74.660651,40.220009") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34021, "countyName": "Mercer", "cityID": 3480630, "cityName": "White Horse" } }
+{ "create_at": datetime("2015-11-26T10:00:13.000Z"), "id": 669938724491300864, "text": "Me in the year 2115 https://t.co/qYR9qG75cD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1452187981, "name": "Haythunn~", "screen_name": "Naythunnn", "lang": "en", "location": "Avengers HQ", "create_at": date("2013-05-23"), "description": "Best Roy on Earth. Jeseille is my butt buddy~ #TeamCap \n@Hikariouss is my beloved senpai~", "followers_count": 519, "friends_count": 397, "statues_count": 29328 }, "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-26T10:00:13.000Z"), "id": 669938724587888640, "text": "THE NATIONAL DOG SHOW IS MY LIIIIIFEEE #happythanksgiving", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "happythanksgiving" }}, "user": { "id": 100880591, "name": "Sarah Fasulka", "screen_name": "SarahMelanieF", "lang": "en", "location": "null", "create_at": date("2009-12-31"), "description": "rev up those fryers // WU '19", "followers_count": 241, "friends_count": 241, "statues_count": 1182 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shavertown, PA", "id": "0b12c2212d92a2d5", "name": "Shavertown", "place_type": "city", "bounding_box": rectangle("-76.025396,41.278082 -75.905418,41.384029") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42079, "countyName": "Luzerne", "cityID": 4269832, "cityName": "Shavertown" } }
+{ "create_at": datetime("2015-11-26T10:00:13.000Z"), "id": 669938725128802305, "text": "https://t.co/4BahURC2Bd #pilotseason #tvshow #actor #la #hollywood #manager #agent #film #movie #sagaftra #cop https://t.co/tYEUvJFspQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "pilotseason", "tvshow", "actor", "la", "hollywood", "manager", "agent", "film", "movie", "sagaftra", "cop" }}, "user": { "id": 576144643, "name": "Tiago Ferreira", "screen_name": "tiagomsferreira", "lang": "pt", "location": "Los Angeles, CA", "create_at": date("2012-05-10"), "description": "#model,#actor , #voiceover artist and certified smartass! make sure tou follow me on this LONG journey ❤️ #Instagram - tiagomsferreira #TugasInLA", "followers_count": 688, "friends_count": 317, "statues_count": 6974 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938725347008513, "text": "I'm Rolling I'm Rolling I'm Rolling...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3976300595, "name": "〽️att $av #23", "screen_name": "DoubleR_MattSav", "lang": "en", "location": "#G4T", "create_at": date("2015-10-16"), "description": "$outh$ide I Ball Undersized But Unguardable I Can't Wait To Turn My Dreams Into Reality..#NLMB #060Da Fam #Savage Squadd #TrillFamilia", "followers_count": 3778, "friends_count": 3750, "statues_count": 11308 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Danville, VA", "id": "a58a872123811d63", "name": "Danville", "place_type": "city", "bounding_box": rectangle("-79.519308,36.540813 -79.317136,36.673579") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51590, "countyName": "Danville", "cityID": 5121344, "cityName": "Danville" } }
+{ "create_at": datetime("2015-11-26T10:00:13.000Z"), "id": 669938725405761538, "text": "@Steph504gurl We didn't even stop there we stopped at the Mobil up the street that line was to long.", "in_reply_to_status": 669921132594728960, "in_reply_to_user": 22141520, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 22141520 }}, "user": { "id": 2793198762, "name": "Shay ", "screen_name": "Sheabutta1", "lang": "en", "location": "null", "create_at": date("2014-09-05"), "description": "null", "followers_count": 32, "friends_count": 175, "statues_count": 153 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938725460176896, "text": "Losing their fuckn minds������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 399701266, "name": "120597", "screen_name": "Shay__YuLoveMe", "lang": "en", "location": "w/Lamar ", "create_at": date("2011-10-27"), "description": "#CheerIsMYLife\nI live for my baby", "followers_count": 453, "friends_count": 450, "statues_count": 46167 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938725506277376, "text": "So ready for this training Tuesday. �� excited.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 594689618, "name": "Worshiper", "screen_name": "AltonGatling", "lang": "en", "location": "west palm beach, FL", "create_at": date("2012-05-30"), "description": "I'm just a follower of Christ and i am so in love with Him.", "followers_count": 1112, "friends_count": 768, "statues_count": 39947 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938725514801152, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1608520746, "name": "tayyy", "screen_name": "tayyyhere", "lang": "en", "location": "null", "create_at": date("2013-07-20"), "description": ":-)", "followers_count": 309, "friends_count": 238, "statues_count": 5975 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whitehall, MI", "id": "00aa93aa000b2efa", "name": "Whitehall", "place_type": "city", "bounding_box": rectangle("-86.379959,43.370952 -86.276661,43.414203") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26121, "countyName": "Muskegon", "cityID": 2686780, "cityName": "Whitehall" } }
+{ "create_at": datetime("2015-11-26T10:00:13.000Z"), "id": 669938725535801345, "text": "@MijazzL happy thanksgiving baby ��❤️ love you", "in_reply_to_status": -1, "in_reply_to_user": 2799341165, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2799341165 }}, "user": { "id": 1222231272, "name": "mw †", "screen_name": "Callmeinnky", "lang": "en", "location": "Columbus, OH", "create_at": date("2013-02-26"), "description": "Snapchat:@Callmeeinky Ig:@Callmeinnky", "followers_count": 809, "friends_count": 364, "statues_count": 11711 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938725577605121, "text": "Happy thanksgiving everyone ❤️ Spend today with your loved ones ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3410374514, "name": "Ruvy Hinojosa", "screen_name": "itsmeRuvyy", "lang": "en", "location": "Omaha, NE", "create_at": date("2015-08-31"), "description": "I'm my own team", "followers_count": 22, "friends_count": 89, "statues_count": 235 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938725594402816, "text": "@ariannnaluna well u too sugar plum", "in_reply_to_status": 669936680745508864, "in_reply_to_user": 737809818, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 737809818 }}, "user": { "id": 724610551, "name": "Caroline", "screen_name": "carolinemccahon", "lang": "en", "location": "null", "create_at": date("2012-07-29"), "description": "MU '19", "followers_count": 449, "friends_count": 267, "statues_count": 6773 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tinton Falls, NJ", "id": "0654550d44724bd1", "name": "Tinton Falls", "place_type": "city", "bounding_box": rectangle("-74.120076,40.215689 -74.050362,40.337286") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3473020, "cityName": "Tinton Falls" } }
+{ "create_at": datetime("2015-11-26T10:00:13.000Z"), "id": 669938725653123072, "text": "Amen! https://t.co/ZuepKdH2ml", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 236904426, "name": "Jackson Hearn", "screen_name": "jackson_hearn", "lang": "en", "location": "Houston/Kingwood, Texas", "create_at": date("2011-01-11"), "description": "I'm a classical musician working @GoodShepherdKW Episcopal Church & Lonestar College in Kingwood TX, a suburb of Houston. Live in the Heights", "followers_count": 233, "friends_count": 152, "statues_count": 3235 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:13.000Z"), "id": 669938725653250048, "text": "start Eli or Romo this week? Help people.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 257564625, "name": "Rob Marks", "screen_name": "RobMarks06", "lang": "en", "location": "North End, Boston", "create_at": date("2011-02-25"), "description": "I can do all things through Christ who strengthens me-Philippians 4:13", "followers_count": 391, "friends_count": 357, "statues_count": 3356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Revere, MA", "id": "407b2c699286aab5", "name": "Revere", "place_type": "city", "bounding_box": rectangle("-71.033007,42.388503 -70.959648,42.4499") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2556585, "cityName": "Revere" } }
+{ "create_at": datetime("2015-11-26T10:00:13.000Z"), "id": 669938725850185728, "text": "@frangipvris love you ❤️����", "in_reply_to_status": 669938545071685632, "in_reply_to_user": 3033137905, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3033137905 }}, "user": { "id": 297719436, "name": "isabel", "screen_name": "fagstralians", "lang": "en", "location": "california", "create_at": date("2011-05-12"), "description": "⠀⠀⠀⠀i've found happiness", "followers_count": 2393, "friends_count": 351, "statues_count": 26403 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938726043172864, "text": "ATTENTION ALL FAMILIES: keep the \"how's school\"-type questions to a minimum", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 167668890, "name": "L!T", "screen_name": "dartheazy", "lang": "en", "location": "Arlington,TX --- PVTX", "create_at": date("2010-07-16"), "description": "#dccfe IG: og.eazy SC: CocaineChef #PVAMU #ArchitectMajor", "followers_count": 1890, "friends_count": 1577, "statues_count": 102683 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938726068486145, "text": "Who wanna get a lil football game going at Ivan green @3 with me and bro @cj_turner1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 287253599, 1486521870 }}, "user": { "id": 3060430051, "name": "DubDub©", "screen_name": "OvOThrone", "lang": "en", "location": "null", "create_at": date("2015-03-03"), "description": "SvpremexWave", "followers_count": 184, "friends_count": 221, "statues_count": 3727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irondequoit, NY", "id": "31bfc1f1d5544e1b", "name": "Irondequoit", "place_type": "city", "bounding_box": rectangle("-77.623126,43.166743 -77.521784,43.256791") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3637737, "cityName": "Irondequoit" } }
+{ "create_at": datetime("2015-11-26T10:00:13.000Z"), "id": 669938726102040578, "text": "#HappyThanksgiving #Thankful https://t.co/LuD1UN7o8d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "HappyThanksgiving", "Thankful" }}, "user": { "id": 199108958, "name": "Brandon Autry", "screen_name": "iambrandonautry", "lang": "en", "location": "34.8203° N, 87.6628° W", "create_at": date("2010-10-05"), "description": "Everything will be alright in the end. So if it's not alright, it's not yet the end.", "followers_count": 442, "friends_count": 55, "statues_count": 13534 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938726106103808, "text": "Cloudy this afternoon, high 74 (23 C). Low 63 (17 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575147, "name": "Baton Rouge Weather", "screen_name": "BatonRougeWX", "lang": "en", "location": "Baton Rouge, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Baton Rouge, LA", "followers_count": 2859, "friends_count": 92, "statues_count": 7753 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938726219489280, "text": "This new @PrismoMusic song is fuego as usual, dont sleep on this dude!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1107067296 }}, "user": { "id": 1121354678, "name": "mitchell", "screen_name": "Mitchkute", "lang": "en", "location": "null", "create_at": date("2013-01-26"), "description": "i get people turnt. contact me at mitchellkute@gmail.com", "followers_count": 233, "friends_count": 254, "statues_count": 11737 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938726278070273, "text": "I don't care for ceremonies and rituals. #ThanksgivingDayParade #HappyTurkeyDay #gobblegobble #GobbleGobbleGobble #gobblefuckinggobble", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ThanksgivingDayParade", "HappyTurkeyDay", "gobblegobble", "GobbleGobbleGobble", "gobblefuckinggobble" }}, "user": { "id": 2279670090, "name": "Mike Bivins", "screen_name": "itsmikebivins", "lang": "en", "location": "Portland", "create_at": date("2014-01-06"), "description": "@therearguard's online editor | News tips go to online.rearguard@gmail.com", "followers_count": 1336, "friends_count": 399, "statues_count": 28960 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938726403837952, "text": "@ChelseaRenee87 scrooge :)", "in_reply_to_status": 669938609760239616, "in_reply_to_user": 26169965, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 26169965 }}, "user": { "id": 2710600128, "name": "ZenguinOnSaMattical", "screen_name": "theamazinmattyb", "lang": "en", "location": "null", "create_at": date("2014-08-05"), "description": "(Former) Blogger guy for @sbnGrizzlies @pensblog Co host of @GBBLive... Never trust a big butt and smile", "followers_count": 567, "friends_count": 1305, "statues_count": 14135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arab, AL", "id": "bea4c362d07ab720", "name": "Arab", "place_type": "city", "bounding_box": rectangle("-86.545742,34.302982 -86.461981,34.3657") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1095, "countyName": "Marshall", "cityID": 102116, "cityName": "Arab" } }
+{ "create_at": datetime("2015-11-26T10:00:13.000Z"), "id": 669938726420807680, "text": "HAPPY THANKSGIVING.. WE WILL BE OPEN 10-2 TODAY!! SON OF A VET THRIFT SHOP IS NOW ACCEPTING EBT… https://t.co/lJYY9qlmlb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.177681,34.0838318"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1274129971, "name": "SON OF A VET THRIFT", "screen_name": "sonofavetthrift", "lang": "en", "location": "3310 N. Eastern Ave. L.A. , CA", "create_at": date("2013-03-16"), "description": "WE SPECIALIZE IN ENTIRE ESTATE CLEAN-OUTS, BUY-OUTS, & ESTATE LIQUIDATION SALES", "followers_count": 330, "friends_count": 96, "statues_count": 2538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:13.000Z"), "id": 669938726479400961, "text": "Just got done watching one of my favorite viners and YouTubers @TheGabbieShow love what she does and is amazing and makes me laugh a lot!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 168778996 }}, "user": { "id": 3151329666, "name": "Charlie Thompson", "screen_name": "Charlie_T27", "lang": "en", "location": "null", "create_at": date("2015-04-12"), "description": "My names Charlie! Im one of the guitarist for Before We Collapse check us out on Facebook give us a like!!! https://www.facebook.com/beforewecollapse", "followers_count": 188, "friends_count": 706, "statues_count": 60 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938726504648704, "text": "FUCKING PETTIGREW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 617562920, "name": "NEW YORK GIANTS 5-5", "screen_name": "MIGHTYTRAPLORD_", "lang": "en", "location": "HOMICIDE HARTFORD ", "create_at": date("2012-06-24"), "description": "FUCK FEMINISTS FUCK TWITTER HOES FUCK ANYONE WHO PARTICIPATES IN #FREETHENIPPLE FUCK WHOEVER GETS OFFENDED BY MY BIO DM IF YOU TRYNA FUCK HOP OFF MY DICK", "followers_count": 1533, "friends_count": 319, "statues_count": 61983 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938726538117120, "text": "@taylorswift13 TWEET SOMETHING", "in_reply_to_status": -1, "in_reply_to_user": 17919972, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17919972 }}, "user": { "id": 566913465, "name": "jacob✨✨", "screen_name": "OhMySwift1213", "lang": "en", "location": "null", "create_at": date("2012-04-29"), "description": "I'm always gonna wanna go the extra mile for them because I cannot believe the extra thousand miles they've gone for me", "followers_count": 2409, "friends_count": 875, "statues_count": 61015 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938726621937664, "text": "We're #hiring! Click to apply: Foodservice Specialist - https://t.co/TwH7UWnOQL #Hospitality #StMatthews, KY #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.6357789,38.2435659"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Hospitality", "StMatthews", "Job", "Jobs" }}, "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": 93, "friends_count": 7, "statues_count": 2960 }, "place": { "country": "United States", "country_code": "United States", "full_name": "St Matthews, KY", "id": "412fdc6320c5197f", "name": "St Matthews", "place_type": "city", "bounding_box": rectangle("-85.668625,38.225963 -85.60599,38.272105") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2148006, "cityName": "Louisville/Jefferson County metro government (balance)" } }
+{ "create_at": datetime("2015-11-26T10:00:13.000Z"), "id": 669938726747811840, "text": "@flynnkelsiee I have a grammar error, but I love you so much Kels", "in_reply_to_status": 669938398329741312, "in_reply_to_user": 1514271283, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1514271283 }}, "user": { "id": 451803350, "name": "makayela shepperd", "screen_name": "MakayelaS35", "lang": "en", "location": "gabriel richard sophomore #HAP", "create_at": date("2011-12-31"), "description": "WAZA FC", "followers_count": 959, "friends_count": 1758, "statues_count": 26242 }, "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("2015-11-26T10:00:13.000Z"), "id": 669938726777151489, "text": "Me in school https://t.co/r2ObT8KdeM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3111514379, "name": "Amanda Johnson ♓️", "screen_name": "amanda_rose_00", "lang": "en", "location": "null", "create_at": date("2015-03-24"), "description": "15 | single | VHS 18' | Drag racing ❤️ |", "followers_count": 187, "friends_count": 570, "statues_count": 2873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vineland, NJ", "id": "ecc2e1285c7d074f", "name": "Vineland", "place_type": "city", "bounding_box": rectangle("-75.076284,39.401507 -74.945245,39.568715") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34011, "countyName": "Cumberland", "cityID": 3476070, "cityName": "Vineland" } }
+{ "create_at": datetime("2015-11-26T10:00:14.000Z"), "id": 669938727087566848, "text": "K-Doubles Take: Alberto Del Rio has returned to WWE!!... https://t.co/96rAZSqypa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 83498576, "name": "Kevin Kendrick", "screen_name": "wcbpodcastkk1", "lang": "en", "location": "Bay Area", "create_at": date("2009-10-18"), "description": "West Coast Bias.... Sports.... With a west coast feel.", "followers_count": 689, "friends_count": 1601, "statues_count": 12592 }, "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("2015-11-26T10:00:14.000Z"), "id": 669938727095918592, "text": "@reasilv06 love you lots ��", "in_reply_to_status": 669938390050037760, "in_reply_to_user": 3666562454, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3666562454 }}, "user": { "id": 1877927540, "name": "Julia Betti", "screen_name": "juuuliabetti", "lang": "en", "location": "null", "create_at": date("2013-09-17"), "description": "3214 // snapchat : juliabetti", "followers_count": 1007, "friends_count": 268, "statues_count": 835 }, "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("2015-11-26T10:00:14.000Z"), "id": 669938727154642944, "text": "Thankful for my family. Teaching me what I need to know. Loving me unconditionally. Supporting me. And having my back 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": 3041970170, "name": "Rodrigo Perez", "screen_name": "Rod_God1999", "lang": "en", "location": "null", "create_at": date("2015-02-25"), "description": "CVHS", "followers_count": 310, "friends_count": 293, "statues_count": 3283 }, "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("2015-11-26T10:00:14.000Z"), "id": 669938727737671681, "text": "Thankful 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": 1938908354, "name": "STEELERS 6-4", "screen_name": "gonzalezluis61", "lang": "en", "location": "your heart", "create_at": date("2013-10-05"), "description": "talking to you makes me feel better✈", "followers_count": 491, "friends_count": 587, "statues_count": 22332 }, "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("2015-11-26T10:00:14.000Z"), "id": 669938727884619777, "text": "Princeton Unive is hiring! Talent Acquisit #jobs in PRINCETON Apply today https://t.co/ZebHsiSefC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.662206,40.378246"), "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": 166, "friends_count": 42, "statues_count": 68839 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Princeton North, NJ", "id": "012c2b170b251774", "name": "Princeton North", "place_type": "city", "bounding_box": rectangle("-74.697861,40.352614 -74.620837,40.393136") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34021, "countyName": "Mercer", "cityID": 3460900, "cityName": "Princeton" } }
+{ "create_at": datetime("2015-11-26T10:00:14.000Z"), "id": 669938728140337152, "text": "@_Munchkin0 you really just abandon us.", "in_reply_to_status": 669938591007571969, "in_reply_to_user": 3198693152, "favorite_count": 0, "coordinate": point("-96.76107639,32.68969685"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3198693152 }}, "user": { "id": 336256499, "name": "broke.", "screen_name": "KazzYoubettaaa", "lang": "en", "location": "null", "create_at": date("2011-07-15"), "description": "null", "followers_count": 1475, "friends_count": 903, "statues_count": 55798 }, "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-26T10:00:14.000Z"), "id": 669938728685715456, "text": "My Family Never Have Turkey And Ham �� On Thanksgiving \n\nRibs, Chicken , Steak And Stuff Like That You Know , Nigga Ish", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2964671907, "name": "davonte dowdell", "screen_name": "gl0tae_", "lang": "en", "location": "null", "create_at": date("2015-01-06"), "description": "Stay Focused , Stay Motivated", "followers_count": 181, "friends_count": 112, "statues_count": 3466 }, "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("2015-11-26T10:00:14.000Z"), "id": 669938728962408449, "text": "Cloudy this afternoon, high 70 (21 C). Low 64 (18 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575204, "name": "New Orleans Weather", "screen_name": "NewOrleansWX", "lang": "en", "location": "New Orleans, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for New Orleans, LA", "followers_count": 2897, "friends_count": 92, "statues_count": 7771 }, "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("2015-11-26T10:00:14.000Z"), "id": 669938729579118596, "text": "@Lady_Infamous30 @ReemaDaBeast everyone would have to be on board for that", "in_reply_to_status": 669937527042605056, "in_reply_to_user": 3314414004, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3314414004, 66154168 }}, "user": { "id": 2479101762, "name": "(#4HG)Gstacks Gaming", "screen_name": "kingstacks88", "lang": "en", "location": "Maryland, USA", "create_at": date("2014-05-05"), "description": "A true gamer who takes his gaming serious subscribe to my channel http://t.co/k0M6GtXQFb gaming #HORSEMENNATION #4hg #LADYHORSEMEN we are a streaming family", "followers_count": 521, "friends_count": 664, "statues_count": 4544 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salisbury, MD", "id": "00caf39d503a84e9", "name": "Salisbury", "place_type": "city", "bounding_box": rectangle("-75.71412,38.28923 -75.487032,38.431613") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24045, "countyName": "Wicomico", "cityID": 2469925, "cityName": "Salisbury" } }
+{ "create_at": datetime("2015-11-26T10:00:14.000Z"), "id": 669938729868353536, "text": "Ready to chow down today & feel guilty about it tomorrow #HappyThanksgiving ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HappyThanksgiving" }}, "user": { "id": 317818870, "name": "Abby Psomas", "screen_name": "AbbyPsomas", "lang": "en", "location": "FTG", "create_at": date("2011-06-15"), "description": "null", "followers_count": 638, "friends_count": 551, "statues_count": 6255 }, "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("2015-11-26T10:00:14.000Z"), "id": 669938730023677952, "text": "Union County Co is hiring! Application and #jobs in CRANFORD Apply today https://t.co/6j3PfHQgov", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.303959,40.656317"), "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": 166, "friends_count": 42, "statues_count": 68841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cranford, NJ", "id": "de7b13455f6b23f3", "name": "Cranford", "place_type": "city", "bounding_box": rectangle("-74.327484,40.633736 -74.28071,40.67891") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union" } }
+{ "create_at": datetime("2015-11-26T10:00:14.000Z"), "id": 669938730325544960, "text": "Interested in a #Healthcare #job near #Baltimore, MD? This could be a great fit: https://t.co/HUCUE4AAcd #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.6121893,39.2903848"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "Baltimore", "Hiring", "CareerArc" }}, "user": { "id": 21221744, "name": "Baltimore Health Job", "screen_name": "tmj_bal_health", "lang": "en", "location": "Baltimore, MD", "create_at": date("2009-02-18"), "description": "Follow this account for geo-targeted Healthcare job tweets in Baltimore, MD. Need help? Tweet us at @CareerArc!", "followers_count": 573, "friends_count": 305, "statues_count": 415 }, "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-26T10:00:14.000Z"), "id": 669938730648666112, "text": "I'm thankful for my beautiful friends ❤️ Happy Thanksgiving everyone! #rhodeisland #thanksgiving… https://t.co/S6MPOdtFyv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.32086853,41.74783988"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "rhodeisland", "thanksgiving" }}, "user": { "id": 471645053, "name": "Ana Marie Calise", "screen_name": "AnaMarieCalise", "lang": "en", "location": "New York, New York", "create_at": date("2012-01-22"), "description": "NYU Tisch- Lee Strasberg Institute and The New Studio || Rhode Island & New York", "followers_count": 268, "friends_count": 389, "statues_count": 2477 }, "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.776049") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4422960, "cityName": "East Providence" } }
+{ "create_at": datetime("2015-11-26T10:00:14.000Z"), "id": 669938730916954112, "text": "The weather forecast for tonight is rain with a low of 34°F. #HappyThanksgiving #NationalCakeDay #Gobblegobble", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.5333,41.5167"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HappyThanksgiving", "NationalCakeDay", "Gobblegobble" }}, "user": { "id": 42666285, "name": "Quad Cities Weather", "screen_name": "qcweather", "lang": "en", "location": "Quad Cities", "create_at": date("2009-05-26"), "description": "Is it going to be warm, wet or cold in the Quad Cities today? You can also find us on Facebook.", "followers_count": 735, "friends_count": 0, "statues_count": 10450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rock Island Arsenal, IL", "id": "8f966c3da836a196", "name": "Rock Island Arsenal", "place_type": "city", "bounding_box": rectangle("-90.570223,41.509992 -90.511303,41.523635") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17161, "countyName": "Rock Island", "cityID": 1765104, "cityName": "Rock Island Arsenal" } }
+{ "create_at": datetime("2015-11-26T10:00:14.000Z"), "id": 669938730917056512, "text": "@vmurray101 we have each other fo dat❤️❤️", "in_reply_to_status": 669919911167336448, "in_reply_to_user": 493618801, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 493618801 }}, "user": { "id": 408763673, "name": "kayla marie ✨", "screen_name": "kaaykay_xoxo", "lang": "en", "location": "Cincinnati, Ohio", "create_at": date("2011-11-09"), "description": "19.✌Mommy to Grace Marie 2.8.14 ❤ MSJU Nursing major. #MountUp c/o 2018", "followers_count": 346, "friends_count": 229, "statues_count": 23603 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Evendale, OH", "id": "f1ab6c065f16adca", "name": "Evendale", "place_type": "city", "bounding_box": rectangle("-84.452048,39.231753 -84.394593,39.269474") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3925802, "cityName": "Evendale" } }
+{ "create_at": datetime("2015-11-26T10:00:15.000Z"), "id": 669938731143421952, "text": "Yay! The Christmas Season begins when Santa comes down the street in the #MacysParade We're on!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MacysParade" }}, "user": { "id": 55470753, "name": "Hector Garcia", "screen_name": "hectorsdallas", "lang": "en", "location": "Dallas, TX", "create_at": date("2009-07-09"), "description": "I am a working actor on TV, film, radio, print, and theater represented in Dallas by The Kim Dawson Talent Agency & work as Dev Mgr of Corp Gifts at AT&T PAC.", "followers_count": 406, "friends_count": 404, "statues_count": 2695 }, "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-26T10:00:15.000Z"), "id": 669938731227353089, "text": "Can you recommend anyone for this #Retail #job? https://t.co/sN0myqllNQ #Honolulu, HI #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "Honolulu", "Hiring", "CareerArc" }}, "user": { "id": 28625039, "name": "TMJ-HON Retail Jobs", "screen_name": "tmj_hon_retail", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 370, "friends_count": 311, "statues_count": 457 }, "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("2015-11-26T10:00:15.000Z"), "id": 669938731256823808, "text": "Union County Co is hiring! Business Analys #jobs in CRANFORD Apply today https://t.co/S5xg6kPlpF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.303959,40.656317"), "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": 166, "friends_count": 42, "statues_count": 68842 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cranford, NJ", "id": "de7b13455f6b23f3", "name": "Cranford", "place_type": "city", "bounding_box": rectangle("-74.327484,40.633736 -74.28071,40.67891") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union" } }
+{ "create_at": datetime("2015-11-26T10:00:15.000Z"), "id": 669938731592216576, "text": "[Thu Nov 26th, 12:00pm] Temperature: 65.3 F, 18.5 C; Humidity: 48.9%; Light: 7.52%. https://t.co/UI5qWnQ21d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 269430133, "name": "Shoemaker Sandbox", "screen_name": "shoe_sandbox", "lang": "en", "location": "Earth (for now). ", "create_at": date("2011-03-20"), "description": "I am a bot. Developer account for @shoemaker.", "followers_count": 7, "friends_count": 1, "statues_count": 23546 }, "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("2015-11-26T10:00:15.000Z"), "id": 669938731676246016, "text": "Home wouldn't be home without getting in an argument with my grandpa about the stupidity of Fox News.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 887804036, "name": "Acacia.", "screen_name": "acacia_clarkson", "lang": "en", "location": "The place by Mentor.", "create_at": date("2012-10-17"), "description": "Professional time traveler. Nocturnal goddess. Your local vampire. Ok. JCU '19.", "followers_count": 355, "friends_count": 639, "statues_count": 9281 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kirtland, OH", "id": "83fb00e66729dcd3", "name": "Kirtland", "place_type": "city", "bounding_box": rectangle("-81.391497,41.543257 -81.309644,41.640817") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39085, "countyName": "Lake", "cityID": 3940642, "cityName": "Kirtland" } }
+{ "create_at": datetime("2015-11-26T10:00:15.000Z"), "id": 669938731780931584, "text": "Cloudy this afternoon, high 73 (23 C). Low 64 (18 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575255, "name": "Shreveport Weather", "screen_name": "ShreveportWX", "lang": "en", "location": "Shreveport, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Shreveport, LA", "followers_count": 691, "friends_count": 92, "statues_count": 7829 }, "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("2015-11-26T10:00:15.000Z"), "id": 669938731780980736, "text": "How most holidays start at the Faymous Estate. | 11.26.15 #thanksgiving #FaymousHoliday #TurkeyDay @… https://t.co/NgTixASTUi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.15078461,41.52748867"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thanksgiving", "FaymousHoliday", "TurkeyDay" }}, "user": { "id": 184572300, "name": "Amanda Fay", "screen_name": "AmandaFaymous", "lang": "en", "location": "New York, New York", "create_at": date("2010-08-29"), "description": "http://TheFaymousLife.com/ | Strategy Lead | Sex Educator | Neuroscience fan | Half Marathoner | Foodie | Love for truffles, & windows down driving.", "followers_count": 651, "friends_count": 1731, "statues_count": 15245 }, "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" } }
+{ "create_at": datetime("2015-11-26T10:00:15.000Z"), "id": 669938732221403136, "text": "and it's only 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": 1146742944, "name": "Hunter♡", "screen_name": "hunteerrr_", "lang": "en", "location": "sac", "create_at": date("2013-02-03"), "description": "•I ain't callin you a truther• ʝfк '16", "followers_count": 519, "friends_count": 733, "statues_count": 14325 }, "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("2015-11-26T10:00:15.000Z"), "id": 669938732229750784, "text": "& don't you ever forget it .. https://t.co/l3x1xywqfi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 279369166, "name": "J O R D A N", "screen_name": "LLCoolJ_Hunt", "lang": "en", "location": "Hou$ton•CollegeStation ,Texas", "create_at": date("2011-04-08"), "description": "dont smoke. dont drink. I just do me . GOD † LaMarque: W3$† sc:j_hunt3 #blinn16 ™", "followers_count": 1813, "friends_count": 986, "statues_count": 96122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:15.000Z"), "id": 669938732233916416, "text": "See our latest #FtWorth, TX #job and click to apply: Operation Technician II - https://t.co/x4spysVNcU #Manufacturing #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.3462545,32.8128038"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FtWorth", "job", "Manufacturing", "Hiring" }}, "user": { "id": 1511327138, "name": "Linde CareersUS", "screen_name": "LindeCareersUS", "lang": "en", "location": "New Jersey, USA", "create_at": date("2013-06-12"), "description": "A member of The Linde Group, a world leading gases and engineering company with around 63,000 employees working in more than 100 countries worldwide.", "followers_count": 342, "friends_count": 227, "statues_count": 992 }, "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("2015-11-26T10:00:15.000Z"), "id": 669938732292788224, "text": "Union County Co is hiring! SharePoint Admi #jobs in CRANFORD Apply today https://t.co/2qLufmsFG1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.303959,40.656317"), "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": 166, "friends_count": 42, "statues_count": 68843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cranford, NJ", "id": "de7b13455f6b23f3", "name": "Cranford", "place_type": "city", "bounding_box": rectangle("-74.327484,40.633736 -74.28071,40.67891") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union" } }
+{ "create_at": datetime("2015-11-26T10:00:15.000Z"), "id": 669938732967968769, "text": "@cenzostellato not saying", "in_reply_to_status": 669938589917122560, "in_reply_to_user": 461639974, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 461639974 }}, "user": { "id": 2253600491, "name": "pat kav", "screen_name": "YaBoyPattyKav", "lang": "en", "location": "gab poe", "create_at": date("2013-12-30"), "description": "if a man has no sauce he is lost, but the same man can get lost in the sauce. | prep '18 | sc: ItsPattyKav", "followers_count": 446, "friends_count": 398, "statues_count": 17924 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Millville, NJ", "id": "01629d4c3339d743", "name": "Millville", "place_type": "city", "bounding_box": rectangle("-75.101446,39.342878 -74.989836,39.429603") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34011, "countyName": "Cumberland", "cityID": 3446680, "cityName": "Millville" } }
+{ "create_at": datetime("2015-11-26T10:00:15.000Z"), "id": 669938732984901632, "text": "Miss my best friend https://t.co/GlMO2XJmqO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 507855677, "name": "Jahdora's Box.", "screen_name": "lustclouds", "lang": "en", "location": "Harlem, NY", "create_at": date("2012-02-28"), "description": "null", "followers_count": 2640, "friends_count": 747, "statues_count": 178685 }, "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("2015-11-26T10:00:15.000Z"), "id": 669938733009907713, "text": "Today would have been a perfect day to hit the links", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 414707044, "name": "Ian Tworzydlo™", "screen_name": "IanTworzydlo", "lang": "en", "location": "null", "create_at": date("2011-11-17"), "description": "Hustle comes first", "followers_count": 420, "friends_count": 166, "statues_count": 3411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mullica Hill, NJ", "id": "f518107274776950", "name": "Mullica Hill", "place_type": "city", "bounding_box": rectangle("-75.268797,39.708688 -75.180206,39.772597") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34015, "countyName": "Gloucester", "cityID": 3449440, "cityName": "Mullica Hill" } }
+{ "create_at": datetime("2015-11-26T10:00:15.000Z"), "id": 669938733135736832, "text": "good afternoon mentions? RT/FAV ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1959579655, "name": "♡☁ 19.8k ♡☁", "screen_name": "itscrewsworldTV", "lang": "en", "location": "Houston, TX", "create_at": date("2013-10-13"), "description": "17 | recording artist | song-writer | actress | promoter | model | booking-business inquires contact: crewloveeemusic2@gmail.com | #TheCrewTeam | #RIPRikoV ♡♕.", "followers_count": 19846, "friends_count": 8806, "statues_count": 110976 }, "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("2015-11-26T10:00:15.000Z"), "id": 669938733211394048, "text": "������������ https://t.co/MbsjxX4Pmz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 450737591, "name": "kylo ren", "screen_name": "B1GLEEK", "lang": "en", "location": "null", "create_at": date("2011-12-30"), "description": "Philippians 4:13 I can do anything i put my mind to! #LaFamila #SushiGang", "followers_count": 1024, "friends_count": 1096, "statues_count": 40657 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scotchtown, NY", "id": "d6e3868f1e64c0f7", "name": "Scotchtown", "place_type": "city", "bounding_box": rectangle("-74.405271,41.4573 -74.344193,41.518179") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36071, "countyName": "Orange", "cityID": 3665882, "cityName": "Scotchtown" } }
+{ "create_at": datetime("2015-11-26T10:00:15.000Z"), "id": 669938733517524992, "text": "@faustolozada @dianalimongi love you buddy! Hoping to get together soon ��", "in_reply_to_status": 669899878340952065, "in_reply_to_user": 16689151, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16689151, 137884931 }}, "user": { "id": 1232450052, "name": "Ludo Gabriele", "screen_name": "LudoGabriele", "lang": "en", "location": "Astoria - New York", "create_at": date("2013-03-01"), "description": "Father II CEO of @DLGMediaNYC II Kindness & Multiculturalism II French II Latino de Corazón", "followers_count": 1450, "friends_count": 1337, "statues_count": 3715 }, "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("2015-11-26T10:00:15.000Z"), "id": 669938733521743872, "text": "https://t.co/fb3bORhsN5 RT imagen EL QUE CON LOBOS SE JUNTA...\n@brozoxmiswebs @DeniseDresserG @epigmenioibarra @kdartigues", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 107120856, 435299501, 54649261, 64638057 }}, "user": { "id": 445329392, "name": "Paco Silva A.", "screen_name": "fsilvaa_", "lang": "es", "location": "El Paso, TX", "create_at": date("2011-12-24"), "description": "Orgulloso Chilango en El Paso Tx.", "followers_count": 132, "friends_count": 149, "statues_count": 3076 }, "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("2015-11-26T10:00:15.000Z"), "id": 669938733894905856, "text": "aww this is so cute https://t.co/ghNBlCi25K", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1531098415, "name": "jazmine", "screen_name": "jazminecanales_", "lang": "en", "location": "null", "create_at": date("2013-06-19"), "description": "null", "followers_count": 720, "friends_count": 560, "statues_count": 10785 }, "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("2015-11-26T10:00:15.000Z"), "id": 669938733928464384, "text": "Happy Thanksgiving everyone. \n\nDo me a favor and make someone smile today.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 384108538, "name": "Beat Oregon", "screen_name": "DavidJMays", "lang": "en", "location": "Lebanon, OR", "create_at": date("2011-10-02"), "description": "#gobeavs #canucks #trailblazers #FireTheCannons #quakes74 #padres Oregon Native", "followers_count": 636, "friends_count": 715, "statues_count": 21745 }, "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("2015-11-26T10:00:15.000Z"), "id": 669938733941047297, "text": "Wait. That wasn't thunder???? �� https://t.co/lZQMJEbZod", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 260979367, "name": "Ayanna ☺️", "screen_name": "AyannaMakaila__", "lang": "en", "location": "Somewhere In My Thoughts", "create_at": date("2011-03-04"), "description": "8.6.14 forever #RIPAshley Donny Savage is my husband", "followers_count": 1155, "friends_count": 988, "statues_count": 15957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richton Park, IL", "id": "044f809bcfa0fb4b", "name": "Richton Park", "place_type": "city", "bounding_box": rectangle("-87.790221,41.469705 -87.703087,41.492177") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1763706, "cityName": "Richton Park" } }
+{ "create_at": datetime("2015-11-26T10:00:15.000Z"), "id": 669938734100426752, "text": "Product Manager - The Creative Group: (#Matthews, NC) https://t.co/MsKnxBw3bV #Marketing #TheCreativeGroup #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.7236804,35.1168131"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Matthews", "Marketing", "TheCreativeGroup", "Job", "Jobs", "Hiring" }}, "user": { "id": 793777357, "name": "TCG Jobs", "screen_name": "TCGJobs", "lang": "en", "location": "null", "create_at": date("2012-08-31"), "description": "TCG is a staffing firm that places design, interactive and marketing professionals in freelance and full-time jobs. See all our Jobs here!", "followers_count": 190, "friends_count": 7, "statues_count": 18714 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Matthews, NC", "id": "827f88eab243bdf1", "name": "Matthews", "place_type": "city", "bounding_box": rectangle("-80.764486,35.081194 -80.653965,35.173287") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3741960, "cityName": "Matthews" } }
+{ "create_at": datetime("2015-11-26T10:00:15.000Z"), "id": 669938734129876992, "text": "Thanksgiving Day selfie @ Ferger Place Historic District https://t.co/NUio2jIfq4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.28444444,35.025"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21869494, "name": "Chastity King", "screen_name": "chaseking2", "lang": "en", "location": "South Carolina, USA", "create_at": date("2009-02-25"), "description": "null", "followers_count": 98, "friends_count": 219, "statues_count": 1031 }, "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("2015-11-26T10:00:15.000Z"), "id": 669938734197026817, "text": "A1 since Day 1 #Cowboyz #Family #MyAce @ Wild Dunes Resort https://t.co/C6GugsgGMc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.73869541,32.80366345"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Cowboyz", "Family", "MyAce" }}, "user": { "id": 872949733, "name": "Breanna Weathers", "screen_name": "SuccessDriven91", "lang": "en", "location": "Somewhere missing my granddaddy", "create_at": date("2012-10-10"), "description": "RIP GRANDDADDY", "followers_count": 325, "friends_count": 320, "statues_count": 28348 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isle of Palms, SC", "id": "c27e1641f82bab8a", "name": "Isle of Palms", "place_type": "city", "bounding_box": rectangle("-79.811695,32.773811 -79.717639,32.827388") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45019, "countyName": "Charleston", "cityID": 4536115, "cityName": "Isle of Palms" } }
+{ "create_at": datetime("2015-11-26T10:00:15.000Z"), "id": 669938734322884608, "text": "\"Why even bother watching the game? They stink. We don't even have a quarter back.\" -Dad \n\nHe speaks truth. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1546317362, "name": "Sabrina Marie", "screen_name": "sabrinamarie61", "lang": "en", "location": "null", "create_at": date("2013-06-25"), "description": "messy hair dont care.", "followers_count": 115, "friends_count": 183, "statues_count": 2176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Medford Lakes, NJ", "id": "018fa46822880a42", "name": "Medford Lakes", "place_type": "city", "bounding_box": rectangle("-74.835036,39.789322 -74.735139,39.87614") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34005, "countyName": "Burlington", "cityID": 3445210, "cityName": "Medford Lakes" } }
+{ "create_at": datetime("2015-11-26T10:00:15.000Z"), "id": 669938734503084032, "text": "Mostly cloudy this afternoon, high 77 (25 C). Low 64 (18 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575312, "name": "Lake Charles Weather", "screen_name": "LakeCharlesWX", "lang": "en", "location": "Lake Charles, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Lake Charles, LA", "followers_count": 536, "friends_count": 92, "statues_count": 7813 }, "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-26T10:00:15.000Z"), "id": 669938734503239684, "text": "It's really not a myth https://t.co/IakNFFZak1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 224390264, "name": "Jalen Saunders", "screen_name": "JSaunders9", "lang": "en", "location": "Virginia ", "create_at": date("2010-12-08"), "description": "null", "followers_count": 841, "friends_count": 616, "statues_count": 124415 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lochearn, MD", "id": "35b58c251fb1789f", "name": "Lochearn", "place_type": "city", "bounding_box": rectangle("-76.748415,39.319362 -76.711027,39.377081") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2447450, "cityName": "Lochearn" } }
+{ "create_at": datetime("2015-11-26T10:00:15.000Z"), "id": 669938734767304704, "text": "@HeyItsMack why not ����", "in_reply_to_status": 669938636364771329, "in_reply_to_user": 449189259, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 449189259 }}, "user": { "id": 86423348, "name": "D☄", "screen_name": "Foxxxybrown_", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2009-10-30"), "description": "Saginaw lady rough rider #21", "followers_count": 1704, "friends_count": 1178, "statues_count": 64659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:15.000Z"), "id": 669938734884720640, "text": "#HappyThanksgiving! Be filled with deep peace, love and joy, and most of all gratitude!\nAll my deepest love always!\nxxoo. #Thanksgiving", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HappyThanksgiving", "Thanksgiving" }}, "user": { "id": 369104982, "name": "Danilo Gabrielli", "screen_name": "DaniloGabrielli", "lang": "en", "location": "NewYork, London.", "create_at": date("2011-09-06"), "description": "Welcome to the fashion world of Danilo Gabrielli. Follow us for the life, travel and sneak peaks of our brand. Contact: Press@danilogabrielli.com", "followers_count": 4378, "friends_count": 1737, "statues_count": 8262 }, "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("2015-11-26T10:00:15.000Z"), "id": 669938734884765696, "text": "Oh wow I overslept smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3276582726, "name": "black power pagan", "screen_name": "lonnybro", "lang": "en", "location": "In the flesh", "create_at": date("2015-07-11"), "description": "null", "followers_count": 80, "friends_count": 72, "statues_count": 82 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:15.000Z"), "id": 669938735182548992, "text": ".@CoachBrandonKU starts practice off in style #KUwbb https://t.co/Iz3ODMEffs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "KUwbb" }}, "user_mentions": {{ 199718942 }}, "user": { "id": 436871866, "name": "Kansas Women's Bball", "screen_name": "KUWBball", "lang": "en", "location": "Lawrence, KS", "create_at": date("2011-12-14"), "description": "Official Twitter for the University of Kansas Women's Basketball team! #kuwbb #rockchalk", "followers_count": 3718, "friends_count": 106, "statues_count": 3862 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University Park, TX", "id": "f90baaeac804e75f", "name": "University Park", "place_type": "city", "bounding_box": rectangle("-96.815054,32.836523 -96.771049,32.865496") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4874492, "cityName": "University Park" } }
+{ "create_at": datetime("2015-11-26T10:00:16.000Z"), "id": 669938735283220480, "text": "Happy thanksgiving everyone. God bless.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1031584111, "name": "Ryan", "screen_name": "King_Higgs", "lang": "en", "location": "the lab", "create_at": date("2012-12-23"), "description": "charging | 650", "followers_count": 136, "friends_count": 97, "statues_count": 924 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938735325315073, "text": "Thankful for these people ❤️ https://t.co/UPLksJaKhZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3247106651, "name": "worldsfinestwasian", "screen_name": "robert_nguyenn", "lang": "en", "location": "Brooklyn Center, MN", "create_at": date("2015-05-11"), "description": "|PC2016|UWEC2020| wrestling somewhere or hanging out with my gf @aimeealmen -I can do everything through Christ who strengthens me Philippines 4:13", "followers_count": 142, "friends_count": 137, "statues_count": 1148 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berlin, WI", "id": "c0d00d9f4acb42c0", "name": "Berlin", "place_type": "city", "bounding_box": rectangle("-88.969329,43.953868 -88.920745,43.986812") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55047, "countyName": "Green Lake", "cityID": 5506925, "cityName": "Berlin" } }
+{ "create_at": datetime("2015-11-26T10:00:16.000Z"), "id": 669938735354478592, "text": "So thankful for my smelly ugly dog ❤️����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 470669601, "name": "Darian.", "screen_name": "Darianadree", "lang": "en", "location": "null", "create_at": date("2012-01-21"), "description": "snapchat: darianadree", "followers_count": 837, "friends_count": 472, "statues_count": 11517 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salina, KS", "id": "e7bae49f1ac7f22e", "name": "Salina", "place_type": "city", "bounding_box": rectangle("-97.676631,38.768801 -97.557719,38.885242") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20169, "countyName": "Saline", "cityID": 2062700, "cityName": "Salina" } }
+{ "create_at": datetime("2015-11-26T10:00:16.000Z"), "id": 669938735472078849, "text": "#INDvsATL\nMrs. #50 @ BirdLady Cares https://t.co/leoCfyyE8Y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.7815323,32.8520584"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "INDvsATL" }}, "user": { "id": 39609068, "name": "Falcons BirdLady", "screen_name": "FalconsBirdLady", "lang": "en", "location": "Atlanta, GA + Macon, GA", "create_at": date("2009-05-12"), "description": "The Real Atlanta Falcons BirdLady\r\nProfessional Sports Uber Fan\r\nMy colorful antics & winged outfits energize the team and millions more. 478-335-4850 HOOTY HOO", "followers_count": 1340, "friends_count": 504, "statues_count": 8836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lizella, GA", "id": "0074a89d168f997d", "name": "Lizella", "place_type": "city", "bounding_box": rectangle("-83.825693,32.798794 -83.742311,32.869919") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13021, "countyName": "Bibb", "cityID": 1349008, "cityName": "Macon-Bibb County" } }
+{ "create_at": datetime("2015-11-26T10:00:16.000Z"), "id": 669938735522250752, "text": "Thanksgiving = sweatpants", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2316076640, "name": "Noah", "screen_name": "jonjon_rose", "lang": "en", "location": "Houston,TX", "create_at": date("2014-01-28"), "description": "17 || Vine/Snapchat/IG:JonJonNoah || senior c/o 2016 || Basketball and God", "followers_count": 1362, "friends_count": 916, "statues_count": 16516 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938735631302656, "text": "Go to Sisters>Couch>Football>Eat>Nap>Football>Eat>Nap>Consider Eating More>Eat>Football>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": 6581222, "name": "David Jakes", "screen_name": "djakes", "lang": "en", "location": "Naperville, Illinois, USA", "create_at": date("2007-06-04"), "description": "Chief Design Officer, David Jakes Designs, LLC", "followers_count": 7406, "friends_count": 787, "statues_count": 24654 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938735727771648, "text": "@OCSpurs1882 @ArizonaHotspur I trust him. Also I'm pretty sure many workers died building Qarabag's stadium.", "in_reply_to_status": 669938356646625281, "in_reply_to_user": 1191295542, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1191295542, 1866014533 }}, "user": { "id": 1485811434, "name": "jeffrey m. forsyth", "screen_name": "LATottenhamfan", "lang": "en", "location": "Brentwood", "create_at": date("2013-06-05"), "description": "When life gives you lemons...fuck you. That's what you get. #TTID", "followers_count": 684, "friends_count": 663, "statues_count": 10439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:16.000Z"), "id": 669938735744577536, "text": "Atole?? Don't mind if 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": 126414126, "name": "Mr. Beefy ", "screen_name": "TheBossJose24", "lang": "en", "location": "yo mammas house ", "create_at": date("2010-03-25"), "description": "when life hands me lemons i make beef stew! #LALakers #DallasCowboys #MMA #Family #TeamTaken Philippians 4:13", "followers_count": 677, "friends_count": 830, "statues_count": 12014 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938735748771840, "text": "MY NEW NAME UNCLE ICE FROM OFF PAID N FULL BECAUSE I KIDNAP SHIT !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 271753827, "name": "KIDNAP DA $$$ BRICO", "screen_name": "YungggBillaaa", "lang": "en", "location": "BraeVille/DedrickWorld", "create_at": date("2011-03-24"), "description": "Excuse My Tweets I Just Wanna Be Heard #TeamHardOnAHoe #Free103DammitD RIP Stunnah Bam 103 Dedrick BIP BRAVOE BYIP YOG EMOE RIP LIL WILL RIP DILLO KIP PJ", "followers_count": 2879, "friends_count": 1940, "statues_count": 230139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:16.000Z"), "id": 669938736008826880, "text": "Want to work at CVS Health? We're #hiring in #Lincoln, NE! Click for details: https://t.co/88dBwpXbMy #Retail #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.681679,40.806862"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Lincoln", "Retail", "Job", "Jobs", "CareerArc" }}, "user": { "id": 59877273, "name": "TMJ-NE Retail Jobs", "screen_name": "tmj_ne_retail", "lang": "en", "location": "Nebraska", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in Nebraska Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 316, "friends_count": 301, "statues_count": 146 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938736281468928, "text": "Caught my reflection and now I'm worried @Delta will kick me off the plane for being too good looking. https://t.co/rCN2pCnetH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 5920532 }}, "user": { "id": 16281175, "name": "itsmachupicchu", "screen_name": "itsmachupicchu", "lang": "en", "location": "Seattle, best city in world ", "create_at": date("2008-09-14"), "description": "I am a beautiful person.", "followers_count": 173, "friends_count": 202, "statues_count": 18783 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938736298258432, "text": "I'm thankful for all of these people and a lot more ❤️ https://t.co/EVQDgWfwNN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1738539984, "name": "jade", "screen_name": "Jadensmithh17", "lang": "en", "location": "null", "create_at": date("2013-09-06"), "description": "sshs(:", "followers_count": 239, "friends_count": 264, "statues_count": 840 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938736956862464, "text": "I probably done fucked your bitch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2415481583, "name": "DD$", "screen_name": "rateddee1", "lang": "en", "location": "parkwood ", "create_at": date("2014-03-16"), "description": "Soundcloud: @Dee1one 300%", "followers_count": 445, "friends_count": 278, "statues_count": 6648 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Groveland, FL", "id": "01f5a9551eb80375", "name": "Groveland", "place_type": "city", "bounding_box": rectangle("-81.878167,28.548979 -81.77213,28.631422") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12069, "countyName": "Lake", "cityID": 1227800, "cityName": "Groveland" } }
+{ "create_at": datetime("2015-11-26T10:00:16.000Z"), "id": 669938736986132480, "text": "#Nursing #Job in #FortSmith, AR: Nurse Extern at Mercy Health https://t.co/8tCDgHNEb9 #Jobs #Hiring", "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": {{ "Nursing", "Job", "FortSmith", "Jobs", "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": 282, "friends_count": 4, "statues_count": 4224 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938737124478976, "text": "@LoveKenziiee @kailalove_ I know ��", "in_reply_to_status": 669785346624778240, "in_reply_to_user": 353478652, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 353478652, 756246194 }}, "user": { "id": 928540050, "name": "Blake Biby", "screen_name": "blake_biby", "lang": "en", "location": "Rancho Cucacompton", "create_at": date("2012-11-05"), "description": "Romans 8:31", "followers_count": 551, "friends_count": 839, "statues_count": 1568 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938737141403648, "text": "@ivanacruz01 I surrender ! ��", "in_reply_to_status": 669938135376244736, "in_reply_to_user": 1692786763, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1692786763 }}, "user": { "id": 65152269, "name": "نقي | Naqi", "screen_name": "uptowninfluence", "lang": "en", "location": "NYC", "create_at": date("2009-08-12"), "description": "Poor is the man whose pleasure depends on the permission of another.", "followers_count": 211, "friends_count": 356, "statues_count": 1983 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938737170612224, "text": "Cloudy this afternoon, high 77 (25 C). Low 64 (18 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575370, "name": "Lafayette Weather", "screen_name": "LafayetteWX", "lang": "en", "location": "Lafayette, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Lafayette, LA", "followers_count": 464, "friends_count": 92, "statues_count": 7852 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938737304879104, "text": "Detroit Lion are so USC Trojans 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": 303543035, "name": "RF", "screen_name": "TheRFontillas", "lang": "en", "location": "way up", "create_at": date("2011-05-22"), "description": "IMIW", "followers_count": 583, "friends_count": 761, "statues_count": 13175 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938737476845569, "text": "Man i can't even sell my own supply without people think im getting high off it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 156342793, "name": "[a☀️s]", "screen_name": "Awwpts", "lang": "en", "location": "Slums, Tx", "create_at": date("2010-06-16"), "description": "VIEWER DISCRETION IS ADVISED", "followers_count": 558, "friends_count": 343, "statues_count": 47818 }, "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-26T10:00:16.000Z"), "id": 669938737485209600, "text": "Planning our trip to NYC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 238715116, "name": "You Welcome...✨", "screen_name": "lahshiney", "lang": "en", "location": "Sacramento, CA", "create_at": date("2011-01-15"), "description": "18 & Blessed ❤️.", "followers_count": 1447, "friends_count": 1005, "statues_count": 31503 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938737636204544, "text": "thankful for all u babes too����❤️ https://t.co/Vg5G1vXO5L", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 385256983, "name": "☯ balenciaga kitty☽", "screen_name": "cat_evenson", "lang": "en", "location": " hutch ks ", "create_at": date("2011-10-04"), "description": ". work w the negatives to make a better picture . sad kids club . never settle . be kind always", "followers_count": 742, "friends_count": 704, "statues_count": 29440 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938737686511616, "text": "I'm in a slave town . https://t.co/HhxlSKuGYQ", "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": "Louisiana", "create_at": date("2011-04-30"), "description": "sc: dnashasherae", "followers_count": 2131, "friends_count": 1957, "statues_count": 47402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cottonport, LA", "id": "c2b5f0cfd0aa7fab", "name": "Cottonport", "place_type": "city", "bounding_box": rectangle("-92.060194,30.971779 -92.031263,31.008837") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22009, "countyName": "Avoyelles", "cityID": 2217880, "cityName": "Cottonport" } }
+{ "create_at": datetime("2015-11-26T10:00:16.000Z"), "id": 669938737720238081, "text": "Make your own luck.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 409693411, "name": "Timothy E. Brutus", "screen_name": "TimEBrutus", "lang": "en", "location": "Timebrutus @ Instagram", "create_at": date("2011-11-10"), "description": "Geochemist, Water Supply, Superfund Engineer, Risk Manager, Conservation Social Worker, Biker, Casablanca Fan, Vegetarian, Environmentalist Rockaway Park - NYC", "followers_count": 1245, "friends_count": 971, "statues_count": 19666 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938737820909568, "text": "Today's Word: #portioncontrol #portiondistortion #yuuup #truth #noexcuses #todaystruth #goodmorning… https://t.co/vRbD41fLqn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.84333333,35.22694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "portioncontrol", "portiondistortion", "yuuup", "truth", "noexcuses", "todaystruth", "goodmorning" }}, "user": { "id": 75206827, "name": "Lee H. Dahlen", "screen_name": "LeeDahlen", "lang": "en", "location": "Charlotte, NC", "create_at": date("2009-09-17"), "description": "Truck Driver, Father, Business Owner, and Addicted Boater!!", "followers_count": 169, "friends_count": 173, "statues_count": 11210 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938737867063296, "text": "What a throw omg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1643153515, "name": "Chad Michael Micheal", "screen_name": "AboveAverage_Ed", "lang": "en", "location": "snapchat- ealexsports", "create_at": date("2013-08-03"), "description": "Student Athlete at GVSU⚓. Avid Shower singer. Licensed Archery instructor and piano tutor. I watch NASCAR so that's different.", "followers_count": 862, "friends_count": 492, "statues_count": 41827 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938738110201856, "text": "My little cousin asked me to play 2k with him. I'm shooting nothing but threes and I'm still 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": 231997551, "name": "RAY", "screen_name": "_Thatniggaray_", "lang": "en", "location": " Houston ", "create_at": date("2010-12-29"), "description": "Texas boy", "followers_count": 550, "friends_count": 346, "statues_count": 34572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:16.000Z"), "id": 669938738190000128, "text": "#HappyThanksgiving to all the world family and friends. Enjoy your holiday!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HappyThanksgiving" }}, "user": { "id": 35996651, "name": "Don", "screen_name": "DonHawthorneJr", "lang": "en", "location": "Worldwide", "create_at": date("2009-04-27"), "description": "First Class Gentleman -Father, Son, Entrepreneur, Educator, Sportsman, and Friend!", "followers_count": 34, "friends_count": 57, "statues_count": 359 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, MS", "id": "6e6ed692ac0b92c9", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-89.203925,30.331174 -89.124696,30.383762") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28047, "countyName": "Harrison", "cityID": 2841680, "cityName": "Long Beach" } }
+{ "create_at": datetime("2015-11-26T10:00:16.000Z"), "id": 669938738210975744, "text": "My phone must be broken or something because everyone I texted happy thanksgiving to didn't answer! #thankful", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thankful" }}, "user": { "id": 116216092, "name": "Annie Seiderer", "screen_name": "annieseiderer", "lang": "en", "location": "null", "create_at": date("2010-02-21"), "description": "null", "followers_count": 917, "friends_count": 812, "statues_count": 14906 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938738319912961, "text": "@Juanesha_Maria tell moms she still owe me a plate so I'm thinkin today is the day ��", "in_reply_to_status": 669938215973838848, "in_reply_to_user": 299905534, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 299905534 }}, "user": { "id": 2156231298, "name": "✍", "screen_name": "FollowMEBiznyee", "lang": "en", "location": "null", "create_at": date("2013-10-25"), "description": "null", "followers_count": 792, "friends_count": 525, "statues_count": 30474 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938738638798848, "text": "@RileyJoHarris @FillWerrell truly our lives ��❤️", "in_reply_to_status": 669938562738069504, "in_reply_to_user": 874612651, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 874612651, 386189099 }}, "user": { "id": 214869140, "name": "Shelby Harris", "screen_name": "Shelby_Gk", "lang": "en", "location": "null", "create_at": date("2010-11-12"), "description": "Avid fan of Jesus, hydrangeas and the LA Clippers / ChooseToShine", "followers_count": 637, "friends_count": 222, "statues_count": 16630 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938738651250689, "text": "Thankful for all my friends and family ❣��❤️ https://t.co/0kHDTf2Q8z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3229357484, "name": "Kaleigh", "screen_name": "BorgmannKaleigh", "lang": "en", "location": "Temecula, CA", "create_at": date("2015-05-28"), "description": "chs//water polo//swim//lacrosse//soccer", "followers_count": 202, "friends_count": 619, "statues_count": 951 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938738718498816, "text": "Starting bad. #eaglestalk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "eaglestalk" }}, "user": { "id": 405024417, "name": "Keni Thomas", "screen_name": "thomcat23tweets", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2011-11-04"), "description": "I am an obsessive creative with a habit of creating stuff...constantly.", "followers_count": 486, "friends_count": 839, "statues_count": 9999 }, "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-26T10:00:16.000Z"), "id": 669938739079155712, "text": "take it back out and splash on some seasoning . https://t.co/9aV6HlFGCE", "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": "kaedoll ❤️", "screen_name": "slaekae", "lang": "en", "location": "thatbitch", "create_at": date("2015-04-23"), "description": "know your place . i'm that bitch .", "followers_count": 1373, "friends_count": 985, "statues_count": 19685 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938739137769472, "text": "\"Good news it was just the burns that beaned\"\n#thanksgivingmixup", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thanksgivingmixup" }}, "user": { "id": 2335583346, "name": "Jordan Warner", "screen_name": "Jordan_Wowner", "lang": "en", "location": "null", "create_at": date("2014-02-09"), "description": "doctor, professional talker, world traveler", "followers_count": 98, "friends_count": 243, "statues_count": 204 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Blue Springs, MO", "id": "24af70ccf8cd2bfb", "name": "Blue Springs", "place_type": "city", "bounding_box": rectangle("-94.337321,38.962193 -94.220922,39.067528") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2906652, "cityName": "Blue Springs" } }
+{ "create_at": datetime("2015-11-26T10:00:16.000Z"), "id": 669938739163103232, "text": "Want to grub out already", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 554843886, "name": "Danny Gonzalez ✌", "screen_name": "laker214", "lang": "en", "location": "Nebraska", "create_at": date("2012-04-15"), "description": "|SanLuisPotosi| | SFV/L.A.☀| |Leo ♌| ¤NortenasConSax¤Jaripeos¤Bailes¤ #Lakers#Raider#Dodgers#Barcelona#USC", "followers_count": 1813, "friends_count": 1859, "statues_count": 27137 }, "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("2015-11-26T10:00:16.000Z"), "id": 669938739183939584, "text": "Idk why it never occurred to me that Thanksgiving is an American holiday... You'd think the whole pilgrims thing would have been a good hint", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2723035961, "name": "queen wasp", "screen_name": "Kailynmackenzie", "lang": "en", "location": "Scranton, PA", "create_at": date("2014-07-24"), "description": "Kailyn | 22| God Free | Sober | Horror, Makeup & MIW | ☕️", "followers_count": 237, "friends_count": 224, "statues_count": 658 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colchester, CT", "id": "019f214344e13e7e", "name": "Colchester", "place_type": "city", "bounding_box": rectangle("-72.410747,41.497723 -72.302088,41.627288") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9011, "countyName": "New London", "cityID": 915840, "cityName": "Colchester" } }
+{ "create_at": datetime("2015-11-26T10:00:16.000Z"), "id": 669938739385380864, "text": "Bears & Packers today ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2324044202, "name": "Delgado", "screen_name": "Ozziee25", "lang": "en", "location": "The 6ix", "create_at": date("2014-02-02"), "description": "| Bears | Bulls | BlackHawks | Cubs | Smile more (: Happily married to my soulmate- C.A❤️ U.S Sailor Peruvian-American", "followers_count": 174, "friends_count": 145, "statues_count": 6380 }, "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("2015-11-26T10:00:17.000Z"), "id": 669938739456696321, "text": "@Purpurnium @wieherbuh Was? Das ist doch eindeutig eine Ziege. -.-", "in_reply_to_status": 669935390388830208, "in_reply_to_user": 1254975265, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user_mentions": {{ 1254975265, 225374768 }}, "user": { "id": 800908321, "name": "Gehirnchirurg", "screen_name": "Gehirnchirurg", "lang": "de", "location": "Where the Saw is the Law", "create_at": date("2012-09-03"), "description": "Im Knast bringen dir deine Neck Deep CDs nichts.", "followers_count": 252, "friends_count": 143, "statues_count": 17853 }, "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("2015-11-26T10:00:17.000Z"), "id": 669938739465072641, "text": "@allyruggieri_ hahaha love you too https://t.co/cJDBKqDxHP", "in_reply_to_status": 669937977888538624, "in_reply_to_user": 600357662, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 600357662 }}, "user": { "id": 1255728193, "name": "Kellie Sabo", "screen_name": "Kellie_Sabo", "lang": "en", "location": "null", "create_at": date("2013-03-09"), "description": "Leslie Knope at work, April Ludgate in life", "followers_count": 253, "friends_count": 212, "statues_count": 608 }, "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("2015-11-26T10:00:17.000Z"), "id": 669938739594981377, "text": "What I am thankful for: THE HATERS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16188861, "name": "Melinda", "screen_name": "melindagale", "lang": "en", "location": "Hollywood, CA", "create_at": date("2008-09-08"), "description": "NC to CA. Too southern for my own good. Ex-cheerleader turned Ent Publicist. Campbell U alum. UNC fan. Nerd. Foo Fighters. Marty Crain. Views are my own.", "followers_count": 2213, "friends_count": 432, "statues_count": 66027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:17.000Z"), "id": 669938739687374848, "text": "@Sabrii7_ @_shirleytemple_ @chasbaby1245 @dtalley51 UGH miss you girls!!!!! Happy thanksgiving love you ����", "in_reply_to_status": 669932096211820545, "in_reply_to_user": 503407475, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 503407475, 2248655537, 2956173497, 758532846 }}, "user": { "id": 381138877, "name": "Asia", "screen_name": "asiaaaamarie", "lang": "en", "location": "null", "create_at": date("2011-09-27"), "description": "St. John's University", "followers_count": 431, "friends_count": 318, "statues_count": 19522 }, "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("2015-11-26T10:00:17.000Z"), "id": 669938739725139969, "text": "Been doing the unbelievable lately", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 60188680, "name": "dec14.", "screen_name": "BriCollinsssA1", "lang": "en", "location": "null", "create_at": date("2009-07-25"), "description": "null", "followers_count": 1147, "friends_count": 911, "statues_count": 29257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wynne, AR", "id": "946721918784cd35", "name": "Wynne", "place_type": "city", "bounding_box": rectangle("-90.82803,35.185508 -90.762821,35.25747") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5037, "countyName": "Cross", "cityID": 577090, "cityName": "Wynne" } }
+{ "create_at": datetime("2015-11-26T10:00:17.000Z"), "id": 669938739834064896, "text": "Cloudy this afternoon, high 70 (21 C). Low 58 (14 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575409, "name": "Fort Smith Weather", "screen_name": "FortSmithWX", "lang": "en", "location": "Fort Smith, AR", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Fort Smith, AR", "followers_count": 440, "friends_count": 92, "statues_count": 7901 }, "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("2015-11-26T10:00:17.000Z"), "id": 669938739897069568, "text": "My abuela makes Coxinha and Pão de Queijo better than 99% of Brasilians", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 2970585620, "name": "Ecuador 4-0", "screen_name": "__Sharkbates", "lang": "en", "location": "Broward", "create_at": date("2015-01-09"), "description": "i heart @kchelee tts", "followers_count": 494, "friends_count": 121, "statues_count": 19097 }, "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("2015-11-26T10:00:17.000Z"), "id": 669938740211527681, "text": "You tell no lies. https://t.co/lrpQdT6Exb", "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": "Kaela Malkova", "screen_name": "KaelaKolossal", "lang": "en", "location": "null", "create_at": date("2009-07-14"), "description": "Big Forehead + Dominican so u know the box fire | Die hard Mario Chalmers fan, so yes I'm hurt that he got traded. #HeatNATION (9-4) est. 08 | KU Jayhawks (1-1)", "followers_count": 4947, "friends_count": 943, "statues_count": 177917 }, "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-26T10:00:17.000Z"), "id": 669938740274556928, "text": "Just saw Santa! Let the holiday season officially begin. #MacysParade", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MacysParade" }}, "user": { "id": 117666041, "name": "Kristin Ross", "screen_name": "kriskross22", "lang": "en", "location": "Syracuse/Topeka/London", "create_at": date("2010-02-25"), "description": "Passionate writer of travel, theater and people who do cool things. Duck lover. Short enough to stand in a plane. @NewhouseSU+@dailyorange+@equaltimemag alumna.", "followers_count": 971, "friends_count": 995, "statues_count": 8267 }, "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("2015-11-26T10:00:17.000Z"), "id": 669938740408643584, "text": "Happy Thanksgiving Stingray Band Family!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1426774394, "name": "Vale MS Band", "screen_name": "ValeMSBand", "lang": "en", "location": "San Antonio, TX", "create_at": date("2013-05-13"), "description": "The Official Twitter Page for the Vale MS Stingray Band Program in Northside ISD.", "followers_count": 134, "friends_count": 3, "statues_count": 339 }, "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.604641") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4833146, "cityName": "Helotes" } }
+{ "create_at": datetime("2015-11-26T10:00:17.000Z"), "id": 669938740509437952, "text": "@iam_2cold you straight bra ?", "in_reply_to_status": -1, "in_reply_to_user": 323511620, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 323511620 }}, "user": { "id": 2981448198, "name": "Young Prophet ️", "screen_name": "globoatrvpp", "lang": "en", "location": "BKM.", "create_at": date("2015-01-13"), "description": "All about a bankroll | follow me on ig : sm.joker// jam was here ❤️☺️ @_jam_nojelly | W.C.B.A.R.K.G.Y.T.F |", "followers_count": 315, "friends_count": 212, "statues_count": 3328 }, "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("2015-11-26T10:00:17.000Z"), "id": 669938740622712833, "text": "So excited to get closer to my next competition and the one I've been preping for the whole time:… https://t.co/nbTchEyuVb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.475147,33.995332"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 413840305, "name": "Stanimal", "screen_name": "Stanimal9", "lang": "en", "location": "Los Angeles", "create_at": date("2011-11-16"), "description": "Ifbb Pro Physique - Fitness model - online personal trainer - IG/Periscope : @stanimal9 / Facebook : Stanimal / YouTube : Stan Imal", "followers_count": 459, "friends_count": 225, "statues_count": 1566 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:17.000Z"), "id": 669938740689829888, "text": "Best dogs and glow in the dark relish in the city and burbs! https://t.co/MShJi9LJ0S", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 205829420, "name": "Mike Valentino", "screen_name": "michvatino", "lang": "en", "location": "null", "create_at": date("2010-10-21"), "description": "Chicago: My Kind of Town! Demon Blue!", "followers_count": 50, "friends_count": 50, "statues_count": 2343 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cary, IL", "id": "4b03b11ec52e251e", "name": "Cary", "place_type": "city", "bounding_box": rectangle("-88.290812,42.190136 -88.174076,42.242307") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17111, "countyName": "McHenry", "cityID": 1711592, "cityName": "Cary" } }
+{ "create_at": datetime("2015-11-26T10:00:17.000Z"), "id": 669938740761001986, "text": "@JonnyGranados10 not talking about cod... #⚽️...", "in_reply_to_status": 669938649778143233, "in_reply_to_user": 3151024358, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3151024358 }}, "user": { "id": 2600559787, "name": "Ash", "screen_name": "AshtonRimer7", "lang": "en", "location": "Paradise", "create_at": date("2014-07-02"), "description": "Livin the dream", "followers_count": 372, "friends_count": 254, "statues_count": 10050 }, "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-26T10:00:17.000Z"), "id": 669938740832243712, "text": "Temp 66.6°F Rising, Pressure 30.304in Falling, Dew point 55.7°, Wind N 0mph, Rain today 0.00in https://t.co/gPtnr2dDtV #STLwx #MOwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.35555556,38.47694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "STLwx", "MOwx" }}, "user": { "id": 1191304230, "name": "Oakville Weather", "screen_name": "OakvilleWx", "lang": "en", "location": "Oakville, Missouri, USA", "create_at": date("2013-02-17"), "description": "Current conditions on the hour brought to you by @DaveStLou", "followers_count": 86, "friends_count": 80, "statues_count": 23306 }, "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("2015-11-26T10:00:17.000Z"), "id": 669938741083934720, "text": "Happy Thanksgiving��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 591147810, "name": "❗️", "screen_name": "KisssMyBlunt", "lang": "en", "location": "Inglewood Raised Me", "create_at": date("2012-05-26"), "description": "CSUN❤️", "followers_count": 911, "friends_count": 599, "statues_count": 4323 }, "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("2015-11-26T10:00:17.000Z"), "id": 669938741734064128, "text": "@KAY_XXIII he pushed it back to tomorrow. It was supposed to drop today tho", "in_reply_to_status": 669938365400293376, "in_reply_to_user": 25960202, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25960202 }}, "user": { "id": 2953662166, "name": "Warren Lowe-Bradd", "screen_name": "W_Bradd16", "lang": "en", "location": "18.", "create_at": date("2014-12-31"), "description": "Self-praise is for losers. Be a winner. Stand for something. Always have class, and be humble.", "followers_count": 561, "friends_count": 544, "statues_count": 11713 }, "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-26T10:00:17.000Z"), "id": 669938741801152512, "text": "#ImThankfulFor BANDS, Family, and Friends", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ImThankfulFor" }}, "user": { "id": 3045230936, "name": "Emily", "screen_name": "emilyflores07", "lang": "en", "location": "Somewhere in Neverland", "create_at": date("2015-02-26"), "description": "Find something that makes you happy and don't let anyone take it away from you. -@Luke5SOS", "followers_count": 1166, "friends_count": 2099, "statues_count": 9752 }, "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("2015-11-26T10:00:17.000Z"), "id": 669938741939585024, "text": "Well I hit my first 10K ���� on my way to Mom's for #Thanksgiving...sure was a pleasure #Camaro https://t.co/7m9R9USgzA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Thanksgiving", "Camaro" }}, "user": { "id": 719144738, "name": "William E. Roberts", "screen_name": "wilumm777", "lang": "en", "location": "Oklahoma City, OK", "create_at": date("2012-07-26"), "description": "Saved by the grace and love of Jesus Christ. Camaro lover, video gamist, home cook, two cat furr babies, & much more.", "followers_count": 112, "friends_count": 443, "statues_count": 3952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wewoka, OK", "id": "d7a5f7dab908925d", "name": "Wewoka", "place_type": "city", "bounding_box": rectangle("-96.510849,35.130379 -96.476678,35.16461") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40133, "countyName": "Seminole", "cityID": 4080550, "cityName": "Wewoka" } }
+{ "create_at": datetime("2015-11-26T10:00:17.000Z"), "id": 669938742283513856, "text": "I really like this picture in black and white, but you're not a black and white kind of girl. You're… https://t.co/ucqFFwpFa4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.1292,33.2163"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 289705887, "name": "Taco Tori", "screen_name": "tori_keki", "lang": "en", "location": "den10, TX", "create_at": date("2011-04-28"), "description": "Avoiding all of the responsibilities. UNT. ADPi. I really like cats and Pepsi.", "followers_count": 569, "friends_count": 525, "statues_count": 14686 }, "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-26T10:00:17.000Z"), "id": 669938742338039808, "text": "Cloudy this afternoon, high 69 (21 C). Low 61 (16 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575444, "name": "Little Rock Weather", "screen_name": "LittleRockWX", "lang": "en", "location": "Little Rock, AR", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Little Rock, AR", "followers_count": 1834, "friends_count": 92, "statues_count": 7775 }, "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("2015-11-26T10:00:17.000Z"), "id": 669938742409367552, "text": "Thankful to know I can always count on MARYJANE��❤️���� https://t.co/C6av8eBvp7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1478941069, "name": "Marilu ❤❤", "screen_name": "Mariluuuu_02", "lang": "en", "location": "null", "create_at": date("2013-06-02"), "description": "SHS. 15 years young .\n I Dont Fuck With You ..", "followers_count": 324, "friends_count": 616, "statues_count": 1672 }, "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("2015-11-26T10:00:17.000Z"), "id": 669938742451240960, "text": "thankful for my girls ! Our 5 people squad ���� been sticking with them since 10th grade and they 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": 2503205006, "name": "jj", "screen_name": "_jjorozco", "lang": "en", "location": "Los Angeles ", "create_at": date("2014-05-17"), "description": "bitch you thought", "followers_count": 305, "friends_count": 242, "statues_count": 9061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:17.000Z"), "id": 669938742464028672, "text": "When you get no \"I'm thankful for you blah blah\" texts but it's cool bc you thankful for yourself ���� https://t.co/0oW407cFqO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 95681292, "name": "Ni.", "screen_name": "ShamaBby_", "lang": "en", "location": "Homewood, IL", "create_at": date("2009-12-09"), "description": "*self-love, poppin', and not trippin'* snapchat:shamabby", "followers_count": 558, "friends_count": 279, "statues_count": 44196 }, "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("2015-11-26T10:00:17.000Z"), "id": 669938742484840450, "text": "OML!!!!! I love you bro LMAO ������������������ https://t.co/IGhMJjw9Vu", "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": "bby.", "screen_name": "MayaTaughtYou_", "lang": "en", "location": "dtx", "create_at": date("2012-01-14"), "description": "what it izzzzzz", "followers_count": 1741, "friends_count": 874, "statues_count": 80585 }, "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("2015-11-26T10:00:17.000Z"), "id": 669938742635941888, "text": "Happy Thanksgiving Minna-san https://t.co/LXS6aydp7Z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 199486820, "name": "Spicy Kewpie", "screen_name": "KewCake", "lang": "en", "location": "Brooklyn", "create_at": date("2010-10-06"), "description": "it's okay if it's a n i m e *(*´∀`*)☆", "followers_count": 302, "friends_count": 219, "statues_count": 5648 }, "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("2015-11-26T10:00:17.000Z"), "id": 669938743252402177, "text": "Eh. I agree with some 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": 270114854, "name": "Tempa (RIP Depz)", "screen_name": "QuickTempa", "lang": "en", "location": "Jamaica ✈ Texas ✈️ Washington", "create_at": date("2011-03-21"), "description": "IF YOU DONT LIKE ME FIGHT ME. Boxer. Army Vet. I fw @ayannacapsalot. @JLeighHaggarty is my fav❤️ http://favstar.fm/users/QuickTempa", "followers_count": 6615, "friends_count": 1503, "statues_count": 305251 }, "place": { "country": "United States", "country_code": "United States", "full_name": "DuPont, WA", "id": "01a11fc47c538df9", "name": "DuPont", "place_type": "city", "bounding_box": rectangle("-122.686029,47.086453 -122.621768,47.126519") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5318965, "cityName": "DuPont" } }
+{ "create_at": datetime("2015-11-26T10:00:17.000Z"), "id": 669938743365771264, "text": "Lexi told me I'm ungrateful for what I have bc I would be on the streets shooting up without her.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.2200126,42.73440602"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 389655096, "name": "Rachel Saunders", "screen_name": "_rachelsaunders", "lang": "en", "location": "Lowell, MA", "create_at": date("2011-10-12"), "description": "null", "followers_count": 188, "friends_count": 176, "statues_count": 1146 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Methuen Town, MA", "id": "01597161672b6499", "name": "Methuen Town", "place_type": "city", "bounding_box": rectangle("-71.255938,42.6704 -71.115547,42.794273") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25009, "countyName": "Essex", "cityID": 2540710, "cityName": "Methuen Town" } }
+{ "create_at": datetime("2015-11-26T10:00:17.000Z"), "id": 669938743395094528, "text": "This #Thanksgiving I'm thankful my wife still loves me after seeing embarrassing childhood videos.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Thanksgiving" }}, "user": { "id": 282227476, "name": "Dave Hogg", "screen_name": "DavidJHogg", "lang": "en", "location": "Orlando, FL", "create_at": date("2011-04-14"), "description": "Christ Follower. Husband to @mrscourtneyhogg. 1/5 of @nativeculturefl. Scotsman. UCF AlumKnight.", "followers_count": 414, "friends_count": 274, "statues_count": 3213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakeside, FL", "id": "86aaf36aee0e8a2c", "name": "Lakeside", "place_type": "city", "bounding_box": rectangle("-81.809243,30.100059 -81.721169,30.18923") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12019, "countyName": "Clay", "cityID": 1238813, "cityName": "Lakeside" } }
+{ "create_at": datetime("2015-11-26T10:00:17.000Z"), "id": 669938743487430656, "text": "I'm in there https://t.co/CSibN0ZoxF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1340432827, "name": "Lexie McGraw", "screen_name": "lexie_liciouss", "lang": "en", "location": "410", "create_at": date("2013-04-09"), "description": "null", "followers_count": 1221, "friends_count": 492, "statues_count": 68674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ocean Pines, MD", "id": "90e73699b4312055", "name": "Ocean Pines", "place_type": "city", "bounding_box": rectangle("-75.19021,38.336216 -75.123442,38.408362") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24047, "countyName": "Worcester", "cityID": 2458275, "cityName": "Ocean Pines" } }
+{ "create_at": datetime("2015-11-26T10:00:18.000Z"), "id": 669938743655071744, "text": "NIGGA \"AND 14 OTHER FILES\" WHAT HAHAHAHAH #NoCeilings2 https://t.co/tQdHqu1M1d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NoCeilings2" }}, "user": { "id": 248456332, "name": "Mazee Cascade", "screen_name": "Mazee__", "lang": "en", "location": "null", "create_at": date("2011-02-06"), "description": "CWU", "followers_count": 677, "friends_count": 692, "statues_count": 63457 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938743676043265, "text": "I gotta visit 3 families today ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2231321186, "name": "PLAY MONEY JORDAN", "screen_name": "jaythadoncash", "lang": "en", "location": "Alief, TX", "create_at": date("2013-12-05"), "description": "I fear no man but God. #FREEGUCCI", "followers_count": 1508, "friends_count": 757, "statues_count": 4644 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938743697014784, "text": "Gotta be thankful for mayo squad��✊�� https://t.co/HF2icUXXkf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2745140656, "name": "Jonny Alterman", "screen_name": "_____jonnyy", "lang": "en", "location": "null", "create_at": date("2014-08-16"), "description": "You just gotta keep on livin man L-I-V-I-N", "followers_count": 395, "friends_count": 308, "statues_count": 2301 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938743760035840, "text": "People seek too hard with these thankful tweets", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 345749217, "name": "Cameron Spangler", "screen_name": "SPNGOMO", "lang": "en", "location": "null", "create_at": date("2011-07-30"), "description": "null", "followers_count": 460, "friends_count": 384, "statues_count": 4144 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salem, VA", "id": "73c702136798426b", "name": "Salem", "place_type": "city", "bounding_box": rectangle("-80.120692,37.252554 -80.013519,37.324336") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51775, "countyName": "Salem", "cityID": 5170000, "cityName": "Salem" } }
+{ "create_at": datetime("2015-11-26T10:00:18.000Z"), "id": 669938743856537602, "text": "@pastapadre did you notice that they said they updated their warriors court to match their hardwood classic jerseys but then deleted it", "in_reply_to_status": -1, "in_reply_to_user": 5443032, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 5443032 }}, "user": { "id": 186252668, "name": "Torrey Garrett", "screen_name": "Dytraider", "lang": "en", "location": "Columbus, Ohio", "create_at": date("2010-09-02"), "description": "Living life one day at a time. Devotedly the. I will father avid Oakland Raiders and Detroit Pistons fan.", "followers_count": 62, "friends_count": 502, "statues_count": 4207 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938744271757313, "text": "@kainbreezy love you bub. Thankful for your kind soul ��", "in_reply_to_status": 669938615347249153, "in_reply_to_user": 2149104369, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2149104369 }}, "user": { "id": 405562984, "name": "Aaliyaha Hajek☪", "screen_name": "liyahkaye", "lang": "en", "location": "null", "create_at": date("2011-11-05"), "description": "null", "followers_count": 1401, "friends_count": 588, "statues_count": 36900 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938744284200960, "text": "Fucked up out here ... I gotta go to work", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4262429415, "name": "ᗰiYᗩ✨", "screen_name": "idvnceee", "lang": "en", "location": "вιт¢н ι яєαℓℓу ∂σ ιт ✨", "create_at": date("2015-11-23"), "description": "null", "followers_count": 179, "friends_count": 323, "statues_count": 168 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938744343007233, "text": "❤️lmaoo https://t.co/1Zw0ToKqhT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3561313763, "name": "November 30th", "screen_name": "ray_nevermore", "lang": "en", "location": "Jersey", "create_at": date("2015-09-05"), "description": "Every saint has a past. Every sinner has a future. #14 ⚾️ LHS '17", "followers_count": 95, "friends_count": 87, "statues_count": 1536 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938744414363648, "text": "you you you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2222594068, "name": "han", "screen_name": "hannah_shaner", "lang": "en", "location": "null", "create_at": date("2013-12-12"), "description": "hay is life:)))", "followers_count": 666, "friends_count": 1286, "statues_count": 3927 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bethel Park, PA", "id": "7c1a8644282d6a92", "name": "Bethel Park", "place_type": "city", "bounding_box": rectangle("-80.072649,40.287528 -79.998015,40.359255") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4206064, "cityName": "Bethel Park" } }
+{ "create_at": datetime("2015-11-26T10:00:18.000Z"), "id": 669938744577798144, "text": "Want to work in #Lubbock, TX? View our latest opening: https://t.co/QMw9niRKou #Nursing #nurse #criticalcare #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-101.8919288,33.5881366"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Lubbock", "Nursing", "nurse", "criticalcare", "Job", "Jobs", "Hiring" }}, "user": { "id": 2940007148, "name": "UMC Nursing Jobs", "screen_name": "UMCNursingJobs", "lang": "en", "location": "null", "create_at": date("2014-12-22"), "description": "UMC Health System is the leader in comprehensive healthcare delivery in West Texas and Eastern New Mexico. Follow us to learn about our Nursing Jobs!", "followers_count": 82, "friends_count": 279, "statues_count": 62 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938744628084736, "text": "I used to be a nerd nigga texting everyone in my contacts Happy Thanksgiving smh never again", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1121514170, "name": "$$$ Meem", "screen_name": "Sylver__24", "lang": "en", "location": "The Bay", "create_at": date("2013-01-26"), "description": "#LongLiveLeel | #GOERISM | Trying to find a baddie to wife so my tweets could suck | Y'all don't really hear me tho", "followers_count": 1154, "friends_count": 748, "statues_count": 162286 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938744745529344, "text": "Want to work at Aerotek? We're #hiring in #Honolulu, HI! Click for details: https://t.co/l6rMf1Gw2Z #Manufacturing #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Honolulu", "Manufacturing", "Job", "Jobs" }}, "user": { "id": 23527770, "name": "TMJ - HON Manuf Jobs", "screen_name": "tmj_hon_manuf", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Manufacturing job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 224, "friends_count": 202, "statues_count": 1 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938744879767553, "text": "ok I'm thankful for @izaiahjoaquin too even though he pisses me off ���� https://t.co/lGPiW9FZq1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1665985608 }}, "user": { "id": 1607480544, "name": "Juicy Jaycee", "screen_name": "Jaayylmao_", "lang": "en", "location": "bay area", "create_at": date("2013-07-19"), "description": "tired", "followers_count": 603, "friends_count": 574, "statues_count": 18317 }, "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-26T10:00:18.000Z"), "id": 669938744942661632, "text": "Why https://t.co/UWQBAnfMlk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1583866542, "name": "Alexis Winbush.", "screen_name": "leximw_", "lang": "en", "location": "Lake Charles, LA", "create_at": date("2013-07-10"), "description": "instagram: leximw_", "followers_count": 1004, "friends_count": 502, "statues_count": 35970 }, "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-26T10:00:18.000Z"), "id": 669938745244692480, "text": "Want to work at Software Guidance & Assistance? We're #hiring in #NewYork, NY! Click for details: https://t.co/N6NWVTg0CI #IT #NettempsJobs", "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": {{ "hiring", "NewYork", "IT", "NettempsJobs" }}, "user": { "id": 20832036, "name": "NYC IT Jobs", "screen_name": "tmj_nyc_it", "lang": "en", "location": "New York, NY", "create_at": date("2009-02-13"), "description": "Follow this account for geo-targeted Software Dev. - General/IT job tweets in New York, NY. Need help? Tweet us at @CareerArc!", "followers_count": 803, "friends_count": 363, "statues_count": 634 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938745269993473, "text": "������chauuuuu ������������ @ The Beverly Hilton https://t.co/tNDrgXHcPT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.4129506,34.06621709"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 85523608, "name": "Dana Fleyser♉️", "screen_name": "DANAFLEYSER", "lang": "es", "location": "Malibu CA ", "create_at": date("2009-10-27"), "description": "No me Importa si un animal es capaz de razonar .Solo se que es capaz de sufrir ,y por ello lo considero mi prójimo.//No respondo MD y #100%QuemeraSoy", "followers_count": 8108, "friends_count": 6766, "statues_count": 41475 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938745630552065, "text": "I haven't done my hair , makeup , & I still don't know what I'm going to wear. Yet I'm laying down , like I have the whole time 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": 1337414959, "name": "Bri", "screen_name": "_briannaaa10", "lang": "en", "location": "Dallas, TX", "create_at": date("2013-04-08"), "description": "null", "followers_count": 378, "friends_count": 354, "statues_count": 11383 }, "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-26T10:00:18.000Z"), "id": 669938745634779136, "text": "thank you to everyone that has come and left or come and stayed in my life, you made me who 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": 2955470576, "name": "Ken", "screen_name": "kpooler51", "lang": "en", "location": "null", "create_at": date("2015-01-01"), "description": "chill | cehs | varsity soccer", "followers_count": 216, "friends_count": 270, "statues_count": 2075 }, "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-26T10:00:18.000Z"), "id": 669938745735450624, "text": "See our latest #Smithton, IL #job and click to apply: Teller - Part Time - 25 hours (Smithton) - https://t.co/5XDAlJXGCk #regions #Banking", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.9920501,38.4086616"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Smithton", "job", "regions", "Banking" }}, "user": { "id": 72653327, "name": "Regions Bank", "screen_name": "regionsjobs", "lang": "en", "location": "Birmingham, Alabama", "create_at": date("2009-09-08"), "description": "At Regions, we believe associates deserve more than just a job. We believe in offering performance-driven individuals a place where they can build a career", "followers_count": 773, "friends_count": 45, "statues_count": 11947 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Smithton, IL", "id": "00db92147195917a", "name": "Smithton", "place_type": "city", "bounding_box": rectangle("-90.017549,38.36283 -89.978249,38.425836") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17163, "countyName": "St. Clair", "cityID": 1770252, "cityName": "Smithton" } }
+{ "create_at": datetime("2015-11-26T10:00:18.000Z"), "id": 669938745756540933, "text": "Listening to Adele and helping am cook", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3091352851, "name": "valquiria", "screen_name": "iamvalquiria", "lang": "en", "location": "null", "create_at": date("2015-03-16"), "description": "✨SOCIAL MEDIA ADDICT✨⠀⠀⠀⠀⠀ ⠀⠀⠀⠀⠀⠀#E4A ❤️ #E4K⠀⠀⠀⠀ ⠀⠀ ⠀⠀ ⠀⠀⠀ ⠀⠀ ⠀⠀⠀", "followers_count": 1165, "friends_count": 1030, "statues_count": 18547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brandywine, MD", "id": "01da6bc11ade1443", "name": "Brandywine", "place_type": "city", "bounding_box": rectangle("-76.904938,38.665234 -76.830783,38.733672") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2409325, "cityName": "Brandywine" } }
+{ "create_at": datetime("2015-11-26T10:00:18.000Z"), "id": 669938745903177728, "text": "��thankful for the people in my life that make me smile when I'm down", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2996560218, "name": "serena", "screen_name": "QTlovers", "lang": "en", "location": "null", "create_at": date("2015-01-25"), "description": "i think i'm pretty cool :)", "followers_count": 116, "friends_count": 138, "statues_count": 686 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938745978687488, "text": "#Hospitality #Job in #SantaBarbara, CA: Laundry Attendant/ Linen Driver, Full time at Hilton Worldwide https://t.co/nWfBf8hKgU #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.7107494,34.420334"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "Job", "SantaBarbara", "Jobs" }}, "user": { "id": 99812776, "name": "TMJ-CAA HRTA Jobs", "screen_name": "tmj_CAA_HRTA", "lang": "en", "location": "Santa Barbara, CA", "create_at": date("2009-12-27"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Santa Barbara, CA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 278, "friends_count": 268, "statues_count": 45 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938746045820928, "text": "We're #hiring! Read about our latest #job opening here: Baker - Night - https://t.co/v41Z35fBHb #parttime #Louisville, KY #Hospitality", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.7584557,38.2526647"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "parttime", "Louisville", "Hospitality" }}, "user": { "id": 22510326, "name": "TMJ-SDF HRTA Jobs", "screen_name": "tmj_sdf_hrta", "lang": "en", "location": "Louisville, KY", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Louisville, KY. Need help? Tweet us at @CareerArc!", "followers_count": 360, "friends_count": 291, "statues_count": 419 }, "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-26T10:00:18.000Z"), "id": 669938746205208577, "text": "HE'S ABOUT TO LEAVE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1957518890, "name": "kim ❁", "screen_name": "kimmaay", "lang": "en", "location": "sfsu ☁️", "create_at": date("2013-10-12"), "description": "Royce makes every day feel like another day in paradise, it's true, beautiful in every way. ✨☁️", "followers_count": 1719, "friends_count": 469, "statues_count": 24214 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:18.000Z"), "id": 669938746347888641, "text": "When u eat too much skittles ������ \"taste the rainbow\" https://t.co/usuSNFZTtz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2555894791, "name": "Sabrina✨", "screen_name": "sabrinahammoud2", "lang": "en", "location": "Dearborn Heights, MI", "create_at": date("2014-06-08"), "description": "18 | Lebanese ☪ |", "followers_count": 731, "friends_count": 716, "statues_count": 7494 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938746398146560, "text": "#Honolulu, HI #internship #Job: CAE in Cyber Operations Summer Intern Program at National Security Agency https://t.co/5nGfXa6j4C #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Honolulu", "internship", "Job", "Jobs" }}, "user": { "id": 154601054, "name": "TMJ-HON Intern Jobs", "screen_name": "tmj_HON_intern", "lang": "en", "location": "Honolulu, HI", "create_at": date("2010-06-11"), "description": "Follow this account for geo-targeted Internships job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 263, "friends_count": 256, "statues_count": 3 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938746456940545, "text": "A few forgotten essentials! (@ Rouses Market in Gulf Shores, AL) https://t.co/oOOvCnpz7l", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.68981243,30.26588885"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18173334, "name": "RaginCajunKate", "screen_name": "RaginCajunKate", "lang": "en", "location": "null", "create_at": date("2008-12-16"), "description": "Life is to be lived, not endured.- Fred LeBlanc", "followers_count": 872, "friends_count": 516, "statues_count": 10098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gulf Shores, AL", "id": "dd4fb0ab4e6cd837", "name": "Gulf Shores", "place_type": "city", "bounding_box": rectangle("-87.753372,30.239427 -87.659759,30.334228") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1003, "countyName": "Baldwin", "cityID": 132272, "cityName": "Gulf Shores" } }
+{ "create_at": datetime("2015-11-26T10:00:18.000Z"), "id": 669938746679103488, "text": "Oh https://t.co/aN8JAzUIi8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 384896282, "name": "$as$ ma$teR", "screen_name": "__SassyKassie", "lang": "en", "location": "hogwarts ", "create_at": date("2011-10-04"), "description": "Kassie Bennett, 12, Massachusetts, IG: Sasssykasssiee, Snapchat:Kassie_Bennett #Aquarius ✨i gotta man✨", "followers_count": 1304, "friends_count": 1051, "statues_count": 66582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Saugus, MA", "id": "48ef19debe0f30ca", "name": "Saugus", "place_type": "city", "bounding_box": rectangle("-71.053569,42.429752 -70.972136,42.505195") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25009, "countyName": "Essex", "cityID": 2560050, "cityName": "Saugus" } }
+{ "create_at": datetime("2015-11-26T10:00:18.000Z"), "id": 669938747039997954, "text": "#Nursing #Job alert: LPN Med Care Manager | Sunrise Senior Living | #Smithtown, NY https://t.co/XZ02CiYzGa #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.2006687,40.8559314"), "retweet_count": 0, "lang": "da", "is_retweet": false, "hashtags": {{ "Nursing", "Job", "Smithtown", "Jobs", "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": 294, "friends_count": 9, "statues_count": 2870 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938747111141376, "text": "@801_SkyGuy @cf_fratto and in grateful for you guys. Known Skyker for while haven't been close few years. Lived by Cody for 10. Love ya guys", "in_reply_to_status": 669938249943531520, "in_reply_to_user": 3024576769, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3024576769, 601245817 }}, "user": { "id": 3265718840, "name": "Chris☠", "screen_name": "ChrisHogan105", "lang": "en", "location": "West Jordan, UT", "create_at": date("2015-07-01"), "description": "I am a Pisces. Friday the 13th is my day. My closest friends whom I love dearly. Cody, Ashley, Kory, Skyler, Nick, Jade, Sara. I love basketball☺️", "followers_count": 170, "friends_count": 460, "statues_count": 2125 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938747299905537, "text": "Just posted a photo @ Pasapoga https://t.co/rxA0xfyAS6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.44699088,25.70029248"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 848474808, "name": "Carlos Godoy", "screen_name": "cgodoy2930", "lang": "en", "location": "Miami Florida", "create_at": date("2012-09-26"), "description": "null", "followers_count": 216, "friends_count": 1207, "statues_count": 17147 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kendall West, FL", "id": "007ba9db4c5baedf", "name": "Kendall West", "place_type": "city", "bounding_box": rectangle("-80.4965,25.683801 -80.4299,25.728697") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1236121, "cityName": "Kendall West" } }
+{ "create_at": datetime("2015-11-26T10:00:18.000Z"), "id": 669938747379699712, "text": "@iAmTaTianaa when do you get out dude", "in_reply_to_status": 669938472413712385, "in_reply_to_user": 361045127, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 361045127 }}, "user": { "id": 1316525306, "name": "ㅤki", "screen_name": "kistinker", "lang": "en", "location": " john", "create_at": date("2013-03-30"), "description": "null", "followers_count": 1102, "friends_count": 498, "statues_count": 35908 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938747383918592, "text": "@xocaaat well you know me ��������", "in_reply_to_status": 669938591674580993, "in_reply_to_user": 992156941, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 992156941 }}, "user": { "id": 2950720902, "name": "jazmin", "screen_name": "jazmin_lopezcx", "lang": "en", "location": "Maravatio Del Encinal, GTO.", "create_at": date("2014-12-29"), "description": "Cole World ☁☁️☁️ | Wihs | Lincoln Park", "followers_count": 169, "friends_count": 93, "statues_count": 4329 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938747497058305, "text": "Temp: 45.2°F. Wind: --- @ 0.0mph. Pressure: 32.800\" Falling. Precip today: 0.02\". More info: https://t.co/xoa3CJkwlT. #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": 39, "friends_count": 114, "statues_count": 5191 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938747580923904, "text": "Sleeping in your car sucks", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1484107436, "name": "metal mcmetalhead", "screen_name": "AlexWaverley", "lang": "en", "location": "null", "create_at": date("2013-06-04"), "description": "Beer, booze, bikes, and bullshit", "followers_count": 112, "friends_count": 129, "statues_count": 2378 }, "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("2015-11-26T10:00:18.000Z"), "id": 669938747736203264, "text": "@WilliamsSonoma Your brining bags are garbage. Even w/ plastic tabs the seal wasn't secure. Leaked out Fridge full of brine. Floors a mess.", "in_reply_to_status": -1, "in_reply_to_user": 29247574, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 29247574 }}, "user": { "id": 20717772, "name": "Matt Dicker", "screen_name": "MattDicker", "lang": "en", "location": "Chicago", "create_at": date("2009-02-12"), "description": "Communications Manager at @LevyRestaurants. Chicagoan by geography, Angeleno at heart. And sometimes I write about music and film.", "followers_count": 141, "friends_count": 233, "statues_count": 691 }, "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-26T10:00:19.000Z"), "id": 669938747866275845, "text": "Thankful.... https://t.co/YwwJCPrPYO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 67105964, "name": "1 Jazzy Chic", "screen_name": "brwnsknshwty", "lang": "en", "location": "null", "create_at": date("2009-08-19"), "description": "Realist of tha real....Successful Hair Stylist.. Passion for fashion...Ready to network. ONLY GOD CAN JUDGE ME!", "followers_count": 107, "friends_count": 99, "statues_count": 17772 }, "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-26T10:00:19.000Z"), "id": 669938748222808064, "text": "Wind 4.0 mph SSE. Barometer 30.538 in, Falling slowly. Temperature 53.9 °F. Rain today 0.00 in. Humidity 64%", "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": 23346 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938748239384577, "text": "Wintry mix this afternoon, high 30 (-1 C). Low 21 (-6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26199170, "name": "Omaha Weather", "screen_name": "OmahaWX", "lang": "en", "location": "Omaha, Nebraska", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Omaha, Nebraska", "followers_count": 2573, "friends_count": 88, "statues_count": 7623 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938748868587520, "text": "@Orsonb63 have a fantastic thanksgiving with family and friends. Faithful Grimm watcher", "in_reply_to_status": 669918925761609728, "in_reply_to_user": 104058970, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 104058970 }}, "user": { "id": 2865159687, "name": "My Info", "screen_name": "chuckievanilla", "lang": "en", "location": "null", "create_at": date("2014-11-06"), "description": "null", "followers_count": 4, "friends_count": 24, "statues_count": 54 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938749078380545, "text": "Happy thanksgiving again��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3142531547, "name": "Elijah", "screen_name": "ElijahLelo", "lang": "en", "location": "null", "create_at": date("2015-04-06"), "description": "OBJ GANGGGGG #GIANTSS", "followers_count": 229, "friends_count": 192, "statues_count": 2535 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938749216702464, "text": "I hate thanksgiving dinner, 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": 318184259, "name": "kass", "screen_name": "_kassienelson", "lang": "en", "location": "superior", "create_at": date("2011-06-15"), "description": "snapchat - kassienelson5", "followers_count": 724, "friends_count": 373, "statues_count": 21914 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938749241856000, "text": "@J_Spicee_jklenk this is deep #blessed #inspired", "in_reply_to_status": 669938530806820864, "in_reply_to_user": 1099760582, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "blessed", "inspired" }}, "user_mentions": {{ 1099760582 }}, "user": { "id": 1641441054, "name": "gab", "screen_name": "byrne_gabrielle", "lang": "en", "location": "null", "create_at": date("2013-08-02"), "description": "pvi '18", "followers_count": 444, "friends_count": 780, "statues_count": 4000 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938749258756096, "text": "Couldn't be more excited for the road ahead!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 40967435, "name": "still a chano", "screen_name": "GracePetr", "lang": "en", "location": "Washington, DC", "create_at": date("2009-05-18"), "description": "from 'consin to congress", "followers_count": 375, "friends_count": 268, "statues_count": 7004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wauwatosa, WI", "id": "d378ed20e9e8e270", "name": "Wauwatosa", "place_type": "city", "bounding_box": rectangle("-88.067098,43.030534 -87.983987,43.104541") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5584675, "cityName": "Wauwatosa" } }
+{ "create_at": datetime("2015-11-26T10:00:19.000Z"), "id": 669938749661384704, "text": "@ThangggK @whitmoneyy lol she sent me through a loop��", "in_reply_to_status": 669938609860972544, "in_reply_to_user": 2737676213, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2737676213, 315325124 }}, "user": { "id": 864870924, "name": "asia❣", "screen_name": "aceaikoo_", "lang": "en", "location": "Tx", "create_at": date("2012-10-06"), "description": "new life..who dis?", "followers_count": 1846, "friends_count": 1351, "statues_count": 62515 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938749665492992, "text": "I'm thankful for @onedirection! Cause who needs friends when ya got 1D! Am I right?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 209708391 }}, "user": { "id": 52218982, "name": "Cindy✗❤O", "screen_name": "C_Rod2497", "lang": "en", "location": "Disneyland ", "create_at": date("2009-06-29"), "description": "A Young Heart With An Old Soul ❤", "followers_count": 518, "friends_count": 657, "statues_count": 17849 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938749715824640, "text": "Want to work in #SanDiego, CA? View our latest opening: https://t.co/9RxUdxULoi #Clerical #Veterans #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.1572551,32.7153292"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SanDiego", "Clerical", "Veterans", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 23018727, "name": "TMJ-SAN Cleric. Jobs", "screen_name": "tmj_san_cler", "lang": "en", "location": "San Diego, CA", "create_at": date("2009-03-05"), "description": "Follow this account for geo-targeted Office/Clerical job tweets in San Diego, CA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 296, "friends_count": 200, "statues_count": 134 }, "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-26T10:00:19.000Z"), "id": 669938749715841028, "text": "These people out here think they need a woman or man. Focus on your grind, and the right one will come along.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2148212570, "name": "Blake", "screen_name": "krueger_blake", "lang": "en", "location": "null", "create_at": date("2013-10-21"), "description": "MIA", "followers_count": 319, "friends_count": 460, "statues_count": 5821 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938749845848064, "text": "RIP Coach. ��☝ https://t.co/ScCJxpIyAX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 343179400, "name": "Carlos Reyes", "screen_name": "los3_Reyes", "lang": "en", "location": "null", "create_at": date("2011-07-26"), "description": "Life, Love & Loot", "followers_count": 116, "friends_count": 283, "statues_count": 2780 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938749883707393, "text": "HAPPY BIRTHDAY YUNG GRIFF!!!!!! Miss u @grifwitit I hope you didn't die at the bars! �������� https://t.co/nCwivr41Ty", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 337504960 }}, "user": { "id": 356462937, "name": "bobi-juan kenobi", "screen_name": "kbobes", "lang": "en", "location": "Trap Palace, Omaha", "create_at": date("2011-08-16"), "description": "this Twitter page is lit", "followers_count": 839, "friends_count": 981, "statues_count": 22235 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Papillion, NE", "id": "43212288cc83156e", "name": "Papillion", "place_type": "city", "bounding_box": rectangle("-96.078103,41.117927 -95.962936,41.183482") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31153, "countyName": "Sarpy", "cityID": 3138295, "cityName": "Papillion" } }
+{ "create_at": datetime("2015-11-26T10:00:19.000Z"), "id": 669938749891985412, "text": "#Nursing #Job in #Soldotna, AK: RN - Intensive Care Unit at Central Peninsula Hospital https://t.co/wn5CN4PFpc #MedField #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-151.0583332,60.4877778"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "hashtags": {{ "Nursing", "Job", "Soldotna", "MedField", "Jobs", "Hiring" }}, "user": { "id": 3278999924, "name": "CPH Jobs", "screen_name": "CPHJobs", "lang": "en", "location": "Soldotna, AK", "create_at": date("2015-07-13"), "description": "At CPH every employee is considered a caregiver, an integral part of our health care team. Follow us to learn more!", "followers_count": 3, "friends_count": 0, "statues_count": 50 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Soldotna, AK", "id": "350518f7c1ac93ff", "name": "Soldotna", "place_type": "city", "bounding_box": rectangle("-151.193024,60.458405 -151.048563,60.511008") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2122, "countyName": "Kenai Peninsula", "cityID": 271640, "cityName": "Soldotna" } }
+{ "create_at": datetime("2015-11-26T10:00:19.000Z"), "id": 669938750093262848, "text": "I really wanna go Black Friday shopping ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2444405531, "name": "ashley☄", "screen_name": "_AshleyTiffany2", "lang": "en", "location": "null", "create_at": date("2014-03-29"), "description": "me.myself.& i", "followers_count": 358, "friends_count": 346, "statues_count": 22124 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938750122684416, "text": "@zoerakaya_ happy thanksgiving shorty!! Stay blessed! ����", "in_reply_to_status": 669918207994564608, "in_reply_to_user": 285503403, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 285503403 }}, "user": { "id": 2850335494, "name": "Mari", "screen_name": "_Calicago", "lang": "en", "location": "null", "create_at": date("2014-10-29"), "description": "I'm fresh asl that's all you need to know #College bound #Chiraq #Valpo19 My Latest Hit https://soundcloud.com/hudso97 Pisces 18", "followers_count": 255, "friends_count": 202, "statues_count": 2832 }, "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-26T10:00:19.000Z"), "id": 669938750173093888, "text": "@MichaelJohnLive ZUMBA event combined with a food drive, a walk through the woods,later at Mom's where I will make an apple pie disappear!", "in_reply_to_status": 669893736709648384, "in_reply_to_user": 1899392053, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1899392053 }}, "user": { "id": 69573679, "name": "donna moy-bruno", "screen_name": "donnamoybruno", "lang": "en", "location": "framingham, ma, USA", "create_at": date("2009-08-28"), "description": "Realtor helping home buyers or sellers in Metrowest Boston areas.residential property, a mom, a doglover, Independent associate, Viridian Green Energy", "followers_count": 407, "friends_count": 590, "statues_count": 1363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Framingham, MA", "id": "dbd1411661903bb0", "name": "Framingham", "place_type": "city", "bounding_box": rectangle("-71.496927,42.257263 -71.374193,42.352931") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2524960, "cityName": "Framingham" } }
+{ "create_at": datetime("2015-11-26T10:00:19.000Z"), "id": 669938750244257792, "text": "happy birthday�� @KaribianIslands", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2841811717 }}, "user": { "id": 2326386572, "name": "camryn➰", "screen_name": "cam_sawicki", "lang": "en", "location": "Berkley '19 Gators MRL⚽️", "create_at": date("2014-02-03"), "description": "null", "followers_count": 483, "friends_count": 425, "statues_count": 3368 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Royal Oak, MI", "id": "47cfe29f57708110", "name": "Royal Oak", "place_type": "city", "bounding_box": rectangle("-83.204418,42.474131 -83.111076,42.540305") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2670040, "cityName": "Royal Oak" } }
+{ "create_at": datetime("2015-11-26T10:00:19.000Z"), "id": 669938750324011008, "text": "When in dought, 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": 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": 543, "friends_count": 348, "statues_count": 5626 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938750340788224, "text": "BAYADA Home Health Care: Community Liaison (Part Time) (#Honolulu, HI) https://t.co/GX3GnGio36 #Marketing #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "et", "is_retweet": false, "hashtags": {{ "Honolulu", "Marketing", "Job", "Jobs", "Hiring" }}, "user": { "id": 23493115, "name": "TMJ-HON Advert Jobs", "screen_name": "tmj_hon_adv", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Marketing/Ad/PR job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 289, "friends_count": 193, "statues_count": 9 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938750508527616, "text": "This what happens when I'm in a group chat with my best friends. Tough love is still love right? https://t.co/YBTCX4jrT9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 953070698, "name": "Chris Brown", "screen_name": "cereal_killa666", "lang": "en", "location": "San Diego, CA", "create_at": date("2012-11-16"), "description": "From confidence to self doubt in 60 seconds. ", "followers_count": 363, "friends_count": 670, "statues_count": 12705 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938750529658880, "text": "Seriously guys? Just change the logo to a potato if you love the name so much! Cc @Redskins https://t.co/DAKMwszSOi", "in_reply_to_status": 669878560123940865, "in_reply_to_user": 36375662, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 36375662 }}, "user": { "id": 15576787, "name": "rodney simba masarir", "screen_name": "simba22", "lang": "en", "location": "chicago", "create_at": date("2008-07-23"), "description": "If you come at the king, do not miss! My views are my own... unless if I retweet you then that's your view....\nhttp://about.me/actionscientist", "followers_count": 583, "friends_count": 578, "statues_count": 23558 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938750600822784, "text": "Mi jueves \nComer➡��➡comer\n ⬆ ⬇\n ��⬅Comer ⬅ ⚽", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 266016176, "name": "Jones Captain®", "screen_name": "jonescaptain", "lang": "en", "location": "propiedad de Sandy-California", "create_at": date("2011-03-14"), "description": "Red Lips? no because I'll fall in love", "followers_count": 166, "friends_count": 289, "statues_count": 11680 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938750680670208, "text": "First thanksgiving without 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": 2403718391, "name": "Donnie Azoff", "screen_name": "ZackMcDonald84", "lang": "en", "location": "'twas lit", "create_at": date("2014-03-11"), "description": "6-23-15❤️8-18-15❤️9-19-15", "followers_count": 400, "friends_count": 480, "statues_count": 1881 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waltham, MA", "id": "0633ab4aa27b0bf7", "name": "Waltham", "place_type": "city", "bounding_box": rectangle("-71.289902,42.352885 -71.194241,42.436021") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2572600, "cityName": "Waltham" } }
+{ "create_at": datetime("2015-11-26T10:00:19.000Z"), "id": 669938750718218242, "text": "Just another day. https://t.co/MGuvgApu3m", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 485358987, "name": "Joey", "screen_name": "Joey_Greene15", "lang": "en", "location": "Nampa, ID", "create_at": date("2012-02-06"), "description": "I can do all things through Christ who strengthens me. Philippians 4:13", "followers_count": 476, "friends_count": 201, "statues_count": 6857 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938750751813635, "text": "My hair has gotten so long wtf��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 633496280, "name": "alondra hernandez✨", "screen_name": "lolaalondraa", "lang": "en", "location": "null", "create_at": date("2012-07-11"), "description": "gawhs senior | ♡@seanyouu♡", "followers_count": 419, "friends_count": 340, "statues_count": 3212 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938750755966977, "text": "What is thanksgiving? #work", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "work" }}, "user": { "id": 935807425, "name": "joslynn", "screen_name": "joslynn_nicole", "lang": "en", "location": "nmjc; eunice; cna @ hhc ", "create_at": date("2012-11-08"), "description": "all I need in this life is your crazy love, @howboutdaniel ♡", "followers_count": 447, "friends_count": 483, "statues_count": 33421 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hobbs, NM", "id": "fc045378889ef7d6", "name": "Hobbs", "place_type": "city", "bounding_box": rectangle("-103.201769,32.680274 -103.093307,32.803199") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35025, "countyName": "Lea", "cityID": 3532520, "cityName": "Hobbs" } }
+{ "create_at": datetime("2015-11-26T10:00:19.000Z"), "id": 669938750793760768, "text": "Mostly cloudy this afternoon, high 66 (19 C). Low 54 (12 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433220, "name": "St. Louis Weather", "screen_name": "StLouisWeather", "lang": "en", "location": "St. Louis Metro Area", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for St. Louis Metro Area", "followers_count": 3676, "friends_count": 88, "statues_count": 7596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "St. Louis, MO", "id": "60e6df5778ff9dac", "name": "St. Louis", "place_type": "city", "bounding_box": rectangle("-90.320515,38.531852 -90.166409,38.774346") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29510, "countyName": "St. Louis", "cityID": 2965000, "cityName": "St. Louis" } }
+{ "create_at": datetime("2015-11-26T10:00:19.000Z"), "id": 669938750831656962, "text": "Today is going to be a special erotic thanksgiving edition of Twitter", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 277283433, "name": "DildoEngine98", "screen_name": "Fryedad", "lang": "en", "location": "Huntington, WV", "create_at": date("2011-04-04"), "description": "Doom", "followers_count": 115, "friends_count": 97, "statues_count": 1092 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938750890180608, "text": "i wish i was able to feel my toes but they have turned into ice cubes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 713888221, "name": "allegra", "screen_name": "idkallegra", "lang": "en", "location": "live like jay", "create_at": date("2012-07-24"), "description": "amber is the color of your energy", "followers_count": 571, "friends_count": 286, "statues_count": 24870 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:19.000Z"), "id": 669938751062175744, "text": "#KAPOLEI, HI #Labor #Job: Truck Driver – CDL Class A or B at CVS Health https://t.co/1DIrG17wfX #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-158.056892,21.3354034"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "KAPOLEI", "Labor", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 118676555, "name": "TMJ-HI Labor Jobs", "screen_name": "tmj_HI_LABOR", "lang": "en", "location": "Hawaii", "create_at": date("2010-03-01"), "description": "Follow this account for geo-targeted General Labor job tweets in Hawaii Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 284, "friends_count": 279, "statues_count": 7 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938751322193920, "text": "Because Wickett's so cool he gets a seat. #DisneyWorld #HollywoodStudios #StarWars #Wickett #Ewok @… https://t.co/PCkmRzzQLG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.55983139,28.35616232"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DisneyWorld", "HollywoodStudios", "StarWars", "Wickett", "Ewok" }}, "user": { "id": 1673229649, "name": "Brittany Sisson Lee", "screen_name": "brittsissonlee", "lang": "en", "location": "Raymond, Mississippi", "create_at": date("2013-08-15"), "description": "OFFICIAL Twitter Account of Brittany Lee. Independent Scentsy Consultant. Follow me on Instagram @brittsissonlee and http://facebook.com/brittanysisson", "followers_count": 54, "friends_count": 61, "statues_count": 1652 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938751523545088, "text": "@CameronCrouch BIRD GANG", "in_reply_to_status": 669938484191211520, "in_reply_to_user": 430876040, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 430876040 }}, "user": { "id": 587178669, "name": "Brandon Jack", "screen_name": "Brandon_McG21", "lang": "en", "location": "SoCal ✈️ Sacramento, CA", "create_at": date("2012-05-21"), "description": "My middle name is Jack for a reason // Instagram: mcking_", "followers_count": 517, "friends_count": 453, "statues_count": 16888 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938751557140481, "text": "My mama not cooking dinner ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2181423948, "name": "ItzHot", "screen_name": "rizzoorizzoo", "lang": "en", "location": "Houston, TX (TheSplash)", "create_at": date("2013-11-07"), "description": "THE SAUCE FAMILIA ||| The FLAVA GOD ||| Press: @QueenBobbi ||| Booking: @rocksotrill (713) 305-8037 RIP @fooliboytim #FreeMoneyTon #FreeFredB", "followers_count": 5903, "friends_count": 408, "statues_count": 76133 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:19.000Z"), "id": 669938751561428994, "text": "#CowboyNation #CowboyColors #Gameday @ AT&T Stadium https://t.co/DmkXB65PPP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.0926324,32.74803743"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CowboyNation", "CowboyColors", "Gameday" }}, "user": { "id": 167267403, "name": "Karen L. Fields", "screen_name": "KarenLFields", "lang": "en", "location": "Texas", "create_at": date("2010-07-15"), "description": "Owner Be Found Be Heard Media\nbe_found_be_heard@me.com \nCheck out my blog: http://thefieldsreport.com\nAlways seeking, Always learning!", "followers_count": 574, "friends_count": 1669, "statues_count": 14250 }, "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("2015-11-26T10:00:19.000Z"), "id": 669938751758450688, "text": "This is your tribe – feel it in your trusty gut.\nhttps://t.co/P62Zq24xOu #SantaMonica #Dublin #SF #Happiness #Boulder #NYC #Malibu #London", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SantaMonica", "Dublin", "SF", "Happiness", "Boulder", "NYC", "Malibu", "London" }}, "user": { "id": 288339494, "name": "Dr. Glen Hepker", "screen_name": "AGlimpsofHeaven", "lang": "en", "location": "All in the same common boat...", "create_at": date("2011-04-26"), "description": "Author - A Glimpse of Heaven: The Philosophy of True Health; doctorates in psych/wellness arts; counselor/wellness coach; tai chi/chi kung/wellness arts #ASMSG", "followers_count": 10901, "friends_count": 10908, "statues_count": 143757 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ankeny, IA", "id": "265f5cd93d66b1a1", "name": "Ankeny", "place_type": "city", "bounding_box": rectangle("-93.677966,41.688221 -93.545095,41.777677") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1902305, "cityName": "Ankeny" } }
+{ "create_at": datetime("2015-11-26T10:00:19.000Z"), "id": 669938751787810816, "text": "@DanAndShay right there with ya.. Family is important but so are dogs.", "in_reply_to_status": 669937582164217856, "in_reply_to_user": 1379587099, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1379587099 }}, "user": { "id": 19458529, "name": "Nicole", "screen_name": "13reathe", "lang": "en", "location": "Missouri State University", "create_at": date("2009-01-24"), "description": "Camp Staff. MSU. Early Childhood Education Major. ΣΣΣ. ❤️", "followers_count": 388, "friends_count": 554, "statues_count": 16131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Acreage, FL", "id": "0012a07f55190853", "name": "The Acreage", "place_type": "city", "bounding_box": rectangle("-80.346402,26.692974 -80.197405,26.824966") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1271564, "cityName": "The Acreage" } }
+{ "create_at": datetime("2015-11-26T10:00:19.000Z"), "id": 669938751800532993, "text": "�� why so late https://t.co/f918syf8tJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 341767367, "name": "Symone'", "screen_name": "Hola_imbre", "lang": "en", "location": "null", "create_at": date("2011-07-24"), "description": "NCCU17", "followers_count": 795, "friends_count": 698, "statues_count": 15355 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938752052191232, "text": "Happy thanksgiving all you Soldiers of the Royal Council. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36021288, "name": "Sebastian Danzig", "screen_name": "SebastianDanzig", "lang": "en", "location": "null", "create_at": date("2009-04-28"), "description": "Society needs more individuals like me... Guitarist & organist of fashion-art band Palaye Royale. Follow: @PalayeRoyale", "followers_count": 14157, "friends_count": 2518, "statues_count": 5928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beachwood, OH", "id": "fa5bd676191716ee", "name": "Beachwood", "place_type": "city", "bounding_box": rectangle("-81.532635,41.449669 -81.48037,41.501448") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3904500, "cityName": "Beachwood" } }
+{ "create_at": datetime("2015-11-26T10:00:20.000Z"), "id": 669938752207327234, "text": "Happy thanksgiving pilgrims", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1711453920, "name": "SHNAAJERRYSHNAAA", "screen_name": "jerrbear37", "lang": "en", "location": "null", "create_at": date("2013-08-29"), "description": "shnaaa shnaaa", "followers_count": 376, "friends_count": 354, "statues_count": 6477 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938752211431424, "text": "Thankful for this cutie aka mr bridges @joseph_hennigan #bestie https://t.co/fjPxubo94y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "bestie" }}, "user_mentions": {{ 1598720612 }}, "user": { "id": 436238437, "name": "John Dalton", "screen_name": "DatDudeJD24", "lang": "en", "location": "Lake Charles, LA", "create_at": date("2011-12-13"), "description": "IX/XXV/XIV - Don't take my life or tweets too seriously", "followers_count": 511, "friends_count": 584, "statues_count": 12985 }, "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-26T10:00:20.000Z"), "id": 669938752387575808, "text": "@Mzzzpriss it's on Ethan's page", "in_reply_to_status": 669938668354666496, "in_reply_to_user": 34488283, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 34488283 }}, "user": { "id": 253369322, "name": "Victoria Turner", "screen_name": "MJnCBREEZY4LYFE", "lang": "en", "location": "Birmingham, Al", "create_at": date("2011-02-16"), "description": "College student, living life #MJFam,#TeamBreezy yeah im reppin it! @mombreezy followed 3/23/12 @chrisbrown followed 5/21/12 http://symphoniclovefoundation.org", "followers_count": 1871, "friends_count": 958, "statues_count": 104827 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938752395972608, "text": "Straub Clinic & Hospital #Healthcare #Job: Clinical Assistant - Resource Team (#Honolulu, HI) https://t.co/fXyRSCvWKe #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8480364,21.3136151"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "Job", "Honolulu", "Jobs", "Hiring" }}, "user": { "id": 23526678, "name": "TMJ- HON Health Jobs", "screen_name": "tmj_hon_health", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Healthcare job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 406, "friends_count": 304, "statues_count": 171 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938752437944320, "text": "@AydenMyBlessing wooosah", "in_reply_to_status": 669938500670611457, "in_reply_to_user": 488881212, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 488881212 }}, "user": { "id": 26179647, "name": "Mse: 5-5 Texans", "screen_name": "_BoobieJohnson", "lang": "en", "location": "MSEGANG, Tx", "create_at": date("2009-03-23"), "description": "Age is a state of mind and life is what you make it.", "followers_count": 1634, "friends_count": 809, "statues_count": 247861 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938752639234050, "text": "This RT is for you creative types\n#happythanksgiving https://t.co/lhrZZLXpjS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "happythanksgiving" }}, "user": { "id": 2787130489, "name": "Ken Herrera", "screen_name": "kenherreragroup", "lang": "en", "location": "Las Vegas ", "create_at": date("2014-09-02"), "description": "1) I love Funyuns\n2) My kids are awesome\n3) I help people put a roof over their heads\n4) I have a boss at home too\n5) See #1", "followers_count": 124, "friends_count": 222, "statues_count": 1675 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938752756817920, "text": "Happy Thanksgiving!! #TurkeyTrot done, watching the National dog show #DogThanking and getting ready for Turkey!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TurkeyTrot", "DogThanking" }}, "user": { "id": 109138555, "name": "Suzanne Hopkins", "screen_name": "Suzie_Hopkins", "lang": "en", "location": "Boston", "create_at": date("2010-01-27"), "description": "Mobile, Technology, & Business Enthusiast! Interest in innovative ideas & thought provoking topics! A Jersey Girl, NY Sports Fan, Runner, & Bostonite!", "followers_count": 232, "friends_count": 396, "statues_count": 326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mountainside, NJ", "id": "c76d79c902d9ffbd", "name": "Mountainside", "place_type": "city", "bounding_box": rectangle("-74.387516,40.664275 -74.331282,40.700078") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3448510, "cityName": "Mountainside" } }
+{ "create_at": datetime("2015-11-26T10:00:20.000Z"), "id": 669938752903446528, "text": "Thankful4U https://t.co/L86kq3qrRP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 410215109, "name": "emilyy", "screen_name": "ylimEnenraeTcM", "lang": "en", "location": "null", "create_at": date("2011-11-11"), "description": "Let your past make you better, not bitter.", "followers_count": 458, "friends_count": 266, "statues_count": 18919 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938752911904768, "text": "Interested in a #Sales #job near #FortWorth, TX? This could be a great fit: https://t.co/wpaoSGlkcc #makeup #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.3973863,32.6793391"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "job", "FortWorth", "makeup", "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": 624, "friends_count": 0, "statues_count": 9178 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938753188720640, "text": "Why do we gotta be shits https://t.co/g2UEZwN51K", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 441695020, "name": "$TËFÅNÎ", "screen_name": "gwizzzzllee", "lang": "en", "location": "Texas, USA", "create_at": date("2011-12-20"), "description": "✌️✌✌✌", "followers_count": 506, "friends_count": 287, "statues_count": 11170 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Schertz, TX", "id": "4c9ae2d113edcdd8", "name": "Schertz", "place_type": "city", "bounding_box": rectangle("-98.306055,29.534242 -98.211415,29.634958") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48187, "countyName": "Guadalupe", "cityID": 4866128, "cityName": "Schertz" } }
+{ "create_at": datetime("2015-11-26T10:00:20.000Z"), "id": 669938753230639104, "text": "@Jerumatic yaaaaaaas", "in_reply_to_status": 669736082578104320, "in_reply_to_user": 1053674022, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1053674022 }}, "user": { "id": 564098740, "name": "Carissa Lynn ", "screen_name": "CarissaLynnK", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-04-26"), "description": "singer, music maker, dancer, reader, dreamer, fun maker", "followers_count": 150, "friends_count": 211, "statues_count": 964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:20.000Z"), "id": 669938753310318592, "text": "Rain this afternoon, high 62 (17 C). Low 31 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433608, "name": "Kansas City Weather", "screen_name": "KansasCityWX", "lang": "en", "location": "Kansas City Metro Area", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Kansas City Metro Area", "followers_count": 1921, "friends_count": 88, "statues_count": 7716 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "ddabe75a8f44fe5a", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.765917,38.827176 -94.385522,39.356662") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2015-11-26T10:00:20.000Z"), "id": 669938753364955136, "text": "@Nat_Walls I LOVE YOU NAT!!!", "in_reply_to_status": 669930204316475392, "in_reply_to_user": 2275480569, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2275480569 }}, "user": { "id": 311821226, "name": "Dakota Fleck", "screen_name": "kotaFbaby", "lang": "en", "location": "null", "create_at": date("2011-06-05"), "description": "Cherish today becuase tomorrow is never promised. RIP BMM ❤️ RIP KWK", "followers_count": 646, "friends_count": 605, "statues_count": 12194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Green Spring, KY", "id": "01749f332b880af2", "name": "Green Spring", "place_type": "city", "bounding_box": rectangle("-85.625807,38.302441 -85.581931,38.321555") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2132986, "cityName": "Green Spring" } }
+{ "create_at": datetime("2015-11-26T10:00:20.000Z"), "id": 669938753469722625, "text": "#Banking #Job in #Pittsburgh, PA: Personal Banker Univ at Citizens Financial Group https://t.co/KEMdZeOsjk #cfgjobs #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.9958864,40.4406248"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Banking", "Job", "Pittsburgh", "cfgjobs", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22787034, "name": "TMJ-PIT Bank Jobs", "screen_name": "tmj_pit_banking", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2009-03-04"), "description": "Follow this account for geo-targeted Banking job tweets in Pittsburgh, PA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 370, "friends_count": 316, "statues_count": 83 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938753696215040, "text": "You ready? #CityNeverSleeps https://t.co/7ddgrVuiNi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CityNeverSleeps" }}, "user": { "id": 14423603, "name": "Beats By Dre", "screen_name": "beatsbydre", "lang": "en", "location": "null", "create_at": date("2008-04-17"), "description": "This holiday don't blow it. #GetBeats", "followers_count": 943565, "friends_count": 1008, "statues_count": 23148 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:20.000Z"), "id": 669938753876574208, "text": "Damn that was a tight throw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1228838736, "name": "Hustle Crowe", "screen_name": "RugbyCardigan", "lang": "en", "location": "Florida", "create_at": date("2013-02-28"), "description": "you share that girl", "followers_count": 444, "friends_count": 217, "statues_count": 31735 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938754178674688, "text": "Beyond thankful that Starbucks remains open on thanksgiving!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 164296654, "name": "Kel-c", "screen_name": "KelseyEMcNeill", "lang": "en", "location": "null", "create_at": date("2010-07-08"), "description": "Class of 17", "followers_count": 452, "friends_count": 686, "statues_count": 1810 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938754249822208, "text": "I feel like UCLA football is fools gold. Players seem to hate playing there but the recruits keep pick to play there.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 173051469, "name": "TheChocoMilkCowboy", "screen_name": "SLCHershey", "lang": "en", "location": "Salt Lake City", "create_at": date("2010-07-31"), "description": "University of Utah Grad; Sports enthusiast; Married to a smokin hot wife; Entrepreneur wannabe. LDS", "followers_count": 434, "friends_count": 686, "statues_count": 9077 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938754480533504, "text": "When you getting paid no attention cause mfs playing the game ���� , https://t.co/1ejDtBsH7N", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355704082, "name": "shay", "screen_name": "ShaayMoneey", "lang": "en", "location": "makingmoney", "create_at": date("2011-08-15"), "description": "Chicago ✈️ Dallas . ripLA ripmarcus ❣", "followers_count": 2889, "friends_count": 2082, "statues_count": 60626 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938754874814464, "text": "If you're a #Sales professional in #Gainesville, FL, check out this #job: https://t.co/mAOVMnDwZU #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.3248262,29.6516344"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "Gainesville", "job", "Hiring" }}, "user": { "id": 1193694967, "name": "Helzberg Diamonds", "screen_name": "HelzbergJobs", "lang": "en", "location": "null", "create_at": date("2013-02-18"), "description": "null", "followers_count": 77, "friends_count": 0, "statues_count": 560 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938754887352320, "text": "I'm thankful for @ToriaChan6 @b0ngi0vanni and @Richieboy_16. You guys have put up with me through the years and I'm grateful. 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_mentions": {{ 295971995, 75474758, 536213067 }}, "user": { "id": 244246223, "name": "Rokaan Al-Zeer", "screen_name": "ROKU9413", "lang": "en", "location": "Tucson,AZ USA ", "create_at": date("2011-01-28"), "description": "In a world, divided by fear. One man must stand alone and tweet against the forces of darkness. instagram:@therealroku", "followers_count": 594, "friends_count": 1336, "statues_count": 30258 }, "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-26T10:00:20.000Z"), "id": 669938754895880192, "text": "Thankful for all of these amazing people in my life �� https://t.co/idVa0S1JWA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 888546834, "name": "Emily Weist", "screen_name": "WeistEmily", "lang": "en", "location": "null", "create_at": date("2012-10-18"), "description": "null", "followers_count": 644, "friends_count": 475, "statues_count": 3501 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938754929299457, "text": "@AlizehSheraze YES", "in_reply_to_status": 669919792917143552, "in_reply_to_user": 40995037, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 40995037 }}, "user": { "id": 163977086, "name": "Kristen Johnson", "screen_name": "SpecialllKayyy", "lang": "en", "location": "Wells Branch, TX", "create_at": date("2010-07-07"), "description": "my dream is to spread the Gospel and fill up my passport. time is love.", "followers_count": 616, "friends_count": 163, "statues_count": 25518 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938754950336512, "text": "very thankful for my cute friends �� https://t.co/ovN6W3N2Bo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1350154207, "name": "katy", "screen_name": "katyjane_98", "lang": "en", "location": "murray, utah", "create_at": date("2013-04-13"), "description": "awkward is my speciality", "followers_count": 247, "friends_count": 287, "statues_count": 3328 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murray, UT", "id": "5811c847bb5f82b6", "name": "Murray", "place_type": "city", "bounding_box": rectangle("-111.932066,40.623952 -111.862386,40.684428") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4953230, "cityName": "Murray" } }
+{ "create_at": datetime("2015-11-26T10:00:20.000Z"), "id": 669938754983882752, "text": "Thankful for you gals ��❤️�� https://t.co/8p3ikBJzK2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1330900430, "name": "Niki ✌️", "screen_name": "niki_patelio", "lang": "en", "location": "Atwater, CA", "create_at": date("2013-04-05"), "description": "#lonerforlife mohi '18", "followers_count": 299, "friends_count": 234, "statues_count": 11331 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938755038412800, "text": "someone can have my heart I don't use it anyways.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 324860820, "name": "chic✨", "screen_name": "cvrlsfordayz", "lang": "en", "location": "null", "create_at": date("2011-06-27"), "description": "out of your league | spoiled brat tbh✨", "followers_count": 1517, "friends_count": 728, "statues_count": 115813 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938755071930369, "text": "@TooTurntTina11 love you and hangout soon plz", "in_reply_to_status": 669938648637292544, "in_reply_to_user": 2811882686, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2811882686 }}, "user": { "id": 326399941, "name": "Mal", "screen_name": "Malorie_Moore3", "lang": "en", "location": "null", "create_at": date("2011-06-29"), "description": "you will fall in love with someone who doesn't love you, for not loving someone who did", "followers_count": 563, "friends_count": 504, "statues_count": 7509 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hoxie, AR", "id": "6479deb366513d06", "name": "Hoxie", "place_type": "city", "bounding_box": rectangle("-90.999356,36.023647 -90.957852,36.060998") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5075, "countyName": "Lawrence", "cityID": 533580, "cityName": "Hoxie" } }
+{ "create_at": datetime("2015-11-26T10:00:20.000Z"), "id": 669938755155853312, "text": "Next is my girlfriend because she's the ultimate source of happiness through thick and thin and I'm truly thankful https://t.co/BLd4oXH8an", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 768115417, "name": "Warriors 82-0", "screen_name": "pingadebebe", "lang": "en", "location": "chelsea.com", "create_at": date("2012-08-19"), "description": "livin lavish", "followers_count": 570, "friends_count": 442, "statues_count": 12643 }, "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-26T10:00:20.000Z"), "id": 669938755361378304, "text": "@tjcastle ?", "in_reply_to_status": 669938654958096385, "in_reply_to_user": 275397231, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 275397231 }}, "user": { "id": 496597857, "name": "Thankful Homebear", "screen_name": "PapaHogieBear", "lang": "en", "location": "Amegakure", "create_at": date("2012-02-18"), "description": "Hogan | HHH | Native | HPC Photography | ( ̄^ ̄)ゞ| Cokeboyz | SC & IG : PapaHogieBear |", "followers_count": 1635, "friends_count": 542, "statues_count": 110411 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938755411640320, "text": "���� https://t.co/cq91KVAJv1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3165200658, "name": "кιαяα", "screen_name": "BangsisTae", "lang": "en", "location": "null", "create_at": date("2015-04-20"), "description": "|@ιмgσттιнσє\n|@вσутнαтѕѕ.тαє\nкιαяα ѕнαитαє\nвιgg.∂α∂∂у.тαє\nʝєѕυѕ ℓσνє уσυ ι ∂σи'т❎", "followers_count": 466, "friends_count": 794, "statues_count": 2011 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938755428417536, "text": "@BeeTearra she fake tho https://t.co/vHXBOPF8Br", "in_reply_to_status": -1, "in_reply_to_user": 2307386550, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2307386550 }}, "user": { "id": 334720732, "name": "FINESSIN LIKE DEX", "screen_name": "TrillDarius", "lang": "en", "location": "Port Arthur | Houston", "create_at": date("2011-07-13"), "description": "#StupidBLESSED. 8⃣:13 #LongliveChelsey 5⃣:3 #RipYungDex #SleepLateLoseWeight #CFG. Me Gusta Perras Hispanas!", "followers_count": 1435, "friends_count": 909, "statues_count": 22519 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:20.000Z"), "id": 669938755550113792, "text": "@Irelandreid You have a awesome Thanksgiving", "in_reply_to_status": 669937480787820544, "in_reply_to_user": 1397563902, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1397563902 }}, "user": { "id": 1041135666, "name": "Antonio perdomo", "screen_name": "tonyas123", "lang": "en", "location": "stanton,Ca", "create_at": date("2012-12-27"), "description": "I am workaholic,gamer,runner,basketball player, tv junkie, anime ,Marvel fan,and DC fan. I'm a who likes everything about this world we live on. :-)", "followers_count": 113, "friends_count": 550, "statues_count": 3107 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938755755573249, "text": "There's an unfollow button for a reason what if I feel that they disrespected my cranberry sauce https://t.co/BOWWwLtoiy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1672411460, "name": "$e bae $tian", "screen_name": "sea_bass918", "lang": "en", "location": "layin pipe an stayin hype", "create_at": date("2013-08-14"), "description": "the Y is silent in (y)our girI", "followers_count": 362, "friends_count": 725, "statues_count": 3776 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938755801714689, "text": "that's it. I can't fw this dude ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 302338619, "name": "princess", "screen_name": "TeeNaaasty", "lang": "en", "location": "Port Arthur, Texas ", "create_at": date("2011-05-20"), "description": "get the fuck", "followers_count": 2858, "friends_count": 1752, "statues_count": 19891 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938755839488000, "text": "Partly cloudy this afternoon, high 71 (22 C). Low 57 (14 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433972, "name": "Memphis Weather", "screen_name": "MemphisWX", "lang": "en", "location": "Memphis, TN", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Memphis, TN", "followers_count": 1687, "friends_count": 88, "statues_count": 7628 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938755868844032, "text": "Neon live body painting wallstniteclub with the ever bad ass jill_em_all #bodyart #bodypainting… https://t.co/uGs5qK1mLk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.00192553,39.96517749"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "bodyart", "bodypainting" }}, "user": { "id": 30907136, "name": "Coreroc", "screen_name": "COREROC", "lang": "en", "location": "Columbus, Ohio ", "create_at": date("2009-04-13"), "description": "crush groove meets broken window theory in an all inclusive hip hop monster. #allkings #nojesters", "followers_count": 537, "friends_count": 583, "statues_count": 4417 }, "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("2015-11-26T10:00:20.000Z"), "id": 669938755969642496, "text": "@maarniefaaith jk I mean fracked disregard that", "in_reply_to_status": -1, "in_reply_to_user": 2767544532, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2767544532 }}, "user": { "id": 1407752598, "name": "mr. logan", "screen_name": "SuperLogann", "lang": "en", "location": "Oswego, IL", "create_at": date("2013-05-06"), "description": "I like to heely my way through life. Hammock and chill? YL", "followers_count": 851, "friends_count": 682, "statues_count": 7833 }, "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("2015-11-26T10:00:21.000Z"), "id": 669938756418445312, "text": "A very Happy Thanksgiving to everyone. My first here in the US. Lots to be thankful for today and wonderful new friends.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15297533, "name": "Andy Sinclair-Harris", "screen_name": "185peracre", "lang": "en", "location": "Orlando, FL", "create_at": date("2008-07-02"), "description": "Creative designer for Walt Disney Imagineering, Florida- husband, father, dreamer. All tweets are personal and not affiliated with the Walt Disney Company.", "followers_count": 3976, "friends_count": 549, "statues_count": 12460 }, "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("2015-11-26T10:00:21.000Z"), "id": 669938756493975552, "text": "Sorry but her tweets explain my life https://t.co/YUjs61Jqlv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 484352431, "name": "Kara Kane", "screen_name": "KaraKane3", "lang": "en", "location": "null", "create_at": date("2012-02-05"), "description": "null", "followers_count": 395, "friends_count": 238, "statues_count": 19512 }, "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("2015-11-26T10:00:21.000Z"), "id": 669938756544147456, "text": "This is the scariest picture I've ever seen https://t.co/4E8kbbpL6C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 334964271, "name": "hawley jolly", "screen_name": "Clairee_hawleyy", "lang": "en", "location": "Nevada, USA", "create_at": date("2011-07-13"), "description": "junior at UCLA majoring in emo culture", "followers_count": 405, "friends_count": 267, "statues_count": 10938 }, "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("2015-11-26T10:00:21.000Z"), "id": 669938756665774084, "text": "We're #hiring! Read about our latest #job opening here: Assistant Pantry - On-Call - https://t.co/pBWokywiy4 #Honolulu, HI #Hospitality", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.86,21.31"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Honolulu", "Hospitality" }}, "user": { "id": 23897767, "name": "TMJ-HON Hosp. Jobs", "screen_name": "tmj_hon_hrta", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-11"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 407, "friends_count": 292, "statues_count": 260 }, "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("2015-11-26T10:00:21.000Z"), "id": 669938756711919616, "text": "Interested in a #job near #Livonia, MI? This could be a great fit: https://t.co/4yJrW1e9Y2 #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.3682539,42.4279422"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Livonia", "Hiring", "CareerArc" }}, "user": { "id": 38039576, "name": "TMJ-DTW Jobs", "screen_name": "tmj_dtw_jobs", "lang": "en", "location": "Detroit, MI", "create_at": date("2009-05-05"), "description": "Follow this account for geo-targeted Other job tweets in Detroit, MI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 611, "friends_count": 537, "statues_count": 231 }, "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("2015-11-26T10:00:21.000Z"), "id": 669938756799954945, "text": "@FreddyVentura21 Thanks bro!!!", "in_reply_to_status": 669938635710459904, "in_reply_to_user": 511177341, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 511177341 }}, "user": { "id": 607046304, "name": "Birthday Boy", "screen_name": "Thatmandoulie", "lang": "en", "location": "University of Kansas", "create_at": date("2012-06-13"), "description": "⚡️ #RIPKDB", "followers_count": 722, "friends_count": 706, "statues_count": 8056 }, "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("2015-11-26T10:00:21.000Z"), "id": 669938756976115712, "text": "Inception 3 was the best mw3 montage of all time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2843727943, "name": "NLMB", "screen_name": "VashaadG", "lang": "en", "location": "null", "create_at": date("2014-10-07"), "description": "One Day.", "followers_count": 117, "friends_count": 114, "statues_count": 4253 }, "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("2015-11-26T10:00:21.000Z"), "id": 669938757001342976, "text": "@destafresh I love you!", "in_reply_to_status": 669919955098308608, "in_reply_to_user": 168765131, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 168765131 }}, "user": { "id": 88275396, "name": "✿ॐ✿", "screen_name": "SamanthaaBeee", "lang": "en", "location": "null", "create_at": date("2009-11-07"), "description": "Artist, making my 22nd trip around the sun, experiencing life in LV, ISFJ my thoughts are listed below #GoodVibeTribe", "followers_count": 531, "friends_count": 208, "statues_count": 10838 }, "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("2015-11-26T10:00:21.000Z"), "id": 669938757056004096, "text": "Perfect way to start off thanksgiving break ������thank you @chancetherapper… https://t.co/6ATMkrjZMT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.28132344,34.02337585"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 181572333 }}, "user": { "id": 27577082, "name": "Shahul Habib", "screen_name": "shawhoool", "lang": "en", "location": "the cosmos", "create_at": date("2009-03-29"), "description": "Student. Brother. Blogger. Hip-Hop Head. Music Head. City Boy. Sick Boi.", "followers_count": 205, "friends_count": 263, "statues_count": 9607 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:21.000Z"), "id": 669938757131366400, "text": "@8100blocboii ������ I got the Cranberry Sauce ��������", "in_reply_to_status": 669936337034723328, "in_reply_to_user": 3255960420, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3255960420 }}, "user": { "id": 387388301, "name": "S H A E E", "screen_name": "Trvpvrelli_", "lang": "en", "location": "null", "create_at": date("2011-10-08"), "description": "18||Employed||Studewood Texas|Fmoig:@Trvpvrelli", "followers_count": 832, "friends_count": 927, "statues_count": 7466 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:21.000Z"), "id": 669938757307494400, "text": "This #Retail #job might be a great fit for you: Retail Store Shift Supervisor - https://t.co/Z8LR2oB1y2 #Beaumont, TX #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.1265562,30.080174"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "Beaumont", "Hiring", "CareerArc" }}, "user": { "id": 59694310, "name": "TMJ-TX Retail Jobs", "screen_name": "tmj_tx_retail", "lang": "en", "location": "Texas", "create_at": date("2009-07-23"), "description": "Follow this account for geo-targeted Retail job tweets in Texas Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 509, "friends_count": 308, "statues_count": 7229 }, "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("2015-11-26T10:00:21.000Z"), "id": 669938757316034560, "text": "@jasoncoxphoto lmao I just watched that last night!", "in_reply_to_status": 669938537182175233, "in_reply_to_user": 377713320, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 377713320 }}, "user": { "id": 141426117, "name": "Erick Endres", "screen_name": "erickend", "lang": "en", "location": "Hamburg, New Jersey", "create_at": date("2010-05-07"), "description": "Coffee|Music|Vinyl|Cube Driver|Clothing Brand Owner|Co-Founder @theunderstitch|Social Media Coordinator @lamplighterNJ|@brokenangel33|#supportindie", "followers_count": 3060, "friends_count": 3457, "statues_count": 88004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hamburg, NJ", "id": "005a3c2737b144d2", "name": "Hamburg", "place_type": "city", "bounding_box": rectangle("-74.630634,41.124006 -74.540931,41.19408") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34037, "countyName": "Sussex", "cityID": 3429220, "cityName": "Hamburg" } }
+{ "create_at": datetime("2015-11-26T10:00:21.000Z"), "id": 669938757424943105, "text": "@tucci_bitch what's wrong ��", "in_reply_to_status": 669937772308750336, "in_reply_to_user": 3122584050, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3122584050 }}, "user": { "id": 426093157, "name": "December 3rd ✨", "screen_name": "DomiiDoe", "lang": "en", "location": "MSL❤️", "create_at": date("2011-12-01"), "description": "@jaaynikolee ❤️ @sdjm_love❤️", "followers_count": 2372, "friends_count": 1698, "statues_count": 71621 }, "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("2015-11-26T10:00:21.000Z"), "id": 669938757504802818, "text": "I'm thankful to 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": 321819023, "name": "*AmBurr_Ki", "screen_name": "SimplyAmburr17", "lang": "en", "location": "where you wanna be! ", "create_at": date("2011-06-21"), "description": "AyKay47-Burr IG:SimplyAmburr snapchat:simplyamburr MMTC16", "followers_count": 837, "friends_count": 674, "statues_count": 19605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Andrews, SC", "id": "31da722ae929462a", "name": "Andrews", "place_type": "city", "bounding_box": rectangle("-79.586204,33.437392 -79.536461,33.471052") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45043, "countyName": "Georgetown", "cityID": 4501450, "cityName": "Andrews" } }
+{ "create_at": datetime("2015-11-26T10:00:21.000Z"), "id": 669938757919862784, "text": "���������������� https://t.co/OlbXBXjTIr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 578703803, "name": "Tyla Monroe❤️", "screen_name": "Salutemeimtyy", "lang": "en", "location": "Humble", "create_at": date("2012-05-12"), "description": "Im gonna finish unforgettable", "followers_count": 168, "friends_count": 190, "statues_count": 347 }, "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-26T10:00:21.000Z"), "id": 669938758167437313, "text": "Can't forget about being thankful for these studs. Just all around great guys! ���� https://t.co/SEyTCXLWln", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 755652260, "name": "Bri", "screen_name": "Bri_Nicole0918", "lang": "en", "location": "Johnstown, PA", "create_at": date("2012-08-13"), "description": "Don't fall asleep yet. Contrary to popular belief, that's not where dreams get accomplished. Watsky // Drunk Text Messages To God", "followers_count": 419, "friends_count": 411, "statues_count": 11759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Johnstown, PA", "id": "a93a516308451c2d", "name": "Johnstown", "place_type": "city", "bounding_box": rectangle("-78.963644,40.289545 -78.882152,40.388684") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42021, "countyName": "Cambria", "cityID": 4238288, "cityName": "Johnstown" } }
+{ "create_at": datetime("2015-11-26T10:00:21.000Z"), "id": 669938758356213760, "text": "Over here looking angelic 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": 2481307004, "name": "Mack", "screen_name": "Kenz_Salem", "lang": "en", "location": "null", "create_at": date("2014-05-06"), "description": "bitch.", "followers_count": 457, "friends_count": 450, "statues_count": 5122 }, "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("2015-11-26T10:00:21.000Z"), "id": 669938758947463168, "text": "HAPPY AMERICAN THANKSGIVING, YANKS. Go forth and stuff yourselves. I will join you. ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 67790402, "name": "Moody Sunflower", "screen_name": "MoodySunflower", "lang": "en", "location": "Canadian ex-pat in PDX", "create_at": date("2009-08-21"), "description": "Photos, cats, owls, books, flowers, Rush (the band), feminism, moodiness, dick jokes, Justin Trudeau.", "followers_count": 505, "friends_count": 992, "statues_count": 14924 }, "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("2015-11-26T10:00:21.000Z"), "id": 669938759341707265, "text": "33rd annual Millersville Metrics track and field meet �������� #tbt #throwback #TrackNation @… https://t.co/DXInDB4Jla", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.3543243,39.9975624"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "tbt", "throwback", "TrackNation" }}, "user": { "id": 181232131, "name": "Elè ✨", "screen_name": "MayaaAdele_", "lang": "en", "location": "JERSEY FAM | jw", "create_at": date("2010-08-21"), "description": "I'm your State Champ, All American sprinter & Collegiate Track Athlete #FollowMeOn Snapchat: mayaaadele & IG: mayaadele_ | @OBJ_3 ❤️", "followers_count": 2185, "friends_count": 1870, "statues_count": 24686 }, "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("2015-11-26T10:00:21.000Z"), "id": 669938759454949376, "text": "#BookOnline & use #PromoCode #STUFFED & get 3hrs for $100! #OfferEnds 11/27 #BassHed #Recording… https://t.co/P7609CX38X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.8583553,38.87996358"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BookOnline", "PromoCode", "STUFFED", "OfferEnds", "BassHed", "Recording" }}, "user": { "id": 211213222, "name": "Basshed Music Group", "screen_name": "BassHedMusicGrp", "lang": "en", "location": "@BassHedStudio", "create_at": date("2010-11-02"), "description": "@BassHedAb @BassHedstudio BasshedMusicGroup@gmail.com Basshed Produced projects on http://t.co/cAvpL0aFYZ. Bookings: https://t.co/vRT0hcJbsW", "followers_count": 2308, "friends_count": 2318, "statues_count": 4103 }, "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("2015-11-26T10:00:21.000Z"), "id": 669938759597604864, "text": "Tatawag ako laters", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 2945855071, "name": "Łauren", "screen_name": "bleilysm", "lang": "en", "location": "San Jose, CA", "create_at": date("2014-12-27"), "description": "null", "followers_count": 114, "friends_count": 86, "statues_count": 8455 }, "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-26T10:00:21.000Z"), "id": 669938759702573057, "text": "I'm sad and confused! Why didn't @DoveCameron perform �� I was looking forward to her performance", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 317114365 }}, "user": { "id": 28204847, "name": "Erin Taylor Smith⚓️", "screen_name": "erintaylorrrx", "lang": "en", "location": "Mobile, AL", "create_at": date("2009-04-01"), "description": "•Just a city girl that fell in love with her redneck romeo •UWA •Elementary Ed. Major •@DylanVick 3/7/15 •{God is within her she will not fall}", "followers_count": 1473, "friends_count": 1408, "statues_count": 8993 }, "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("2015-11-26T10:00:21.000Z"), "id": 669938759975239680, "text": "Besides everything https://t.co/x7mGwYC9LH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 422298391, "name": "Jefe", "screen_name": "_jcole30_", "lang": "en", "location": "Ohio, USA", "create_at": date("2011-11-26"), "description": "Go to sleep with a dream, wake up with a purpose | Witt '19", "followers_count": 388, "friends_count": 258, "statues_count": 246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gahanna, OH", "id": "c97807ac2cd60207", "name": "Gahanna", "place_type": "city", "bounding_box": rectangle("-82.905845,39.987076 -82.802554,40.05651") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3929106, "cityName": "Gahanna" } }
+{ "create_at": datetime("2015-11-26T10:00:21.000Z"), "id": 669938760180629504, "text": "Just got here @ dallas finna hop into another plane to Oklahoma", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.040863,32.898038"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1390684946, "name": "Nach8_", "screen_name": "Perez_tgod", "lang": "es", "location": "Houston Tx", "create_at": date("2013-04-29"), "description": "Follow me on IG: Nach8_", "followers_count": 341, "friends_count": 230, "statues_count": 3302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas/Fort Worth International Airport (DFW)", "id": "07d9f31c8ac83000", "name": "Dallas/Fort Worth International Airport (DFW)", "place_type": "poi", "bounding_box": rectangle("-97.0408631,32.8980379 -97.040863,32.898038") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4830644, "cityName": "Grapevine" } }
+{ "create_at": datetime("2015-11-26T10:00:21.000Z"), "id": 669938760209969152, "text": "@mollymargaret_ It does, first time hearing about this. I have to look for that recipe as well", "in_reply_to_status": 669938465375707140, "in_reply_to_user": 830261755, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 830261755 }}, "user": { "id": 72676925, "name": "Albert Mensah-Bekoe", "screen_name": "amensbekoe", "lang": "en", "location": "Chicago/Carbondale, IL ", "create_at": date("2009-09-08"), "description": "A Christian and a die-hard sports fan. #Bulls, #Bears, #Cubs, #Blackhawks. #Virgo #Spurs #Colts #NYGiants #Illini #NotreDame #Ghana #SIUC #ManchesterUnited", "followers_count": 503, "friends_count": 973, "statues_count": 43639 }, "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("2015-11-26T10:00:21.000Z"), "id": 669938760356786176, "text": "About to make my 87 year old great grandfather listen to Plies.. I'll let y'all know how this goes", "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": "free gucci mane. patriots 10-0.", "followers_count": 2077, "friends_count": 836, "statues_count": 292143 }, "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("2015-11-26T10:00:22.000Z"), "id": 669938760449138688, "text": "Us on Saturday @Jackiee_xoxo1 https://t.co/7mC73zKDp9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3015548050 }}, "user": { "id": 2442776883, "name": "kasandra", "screen_name": "kasandra1043", "lang": "en", "location": "null", "create_at": date("2014-03-28"), "description": "FHS 2016", "followers_count": 210, "friends_count": 315, "statues_count": 2074 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Framingham, MA", "id": "dbd1411661903bb0", "name": "Framingham", "place_type": "city", "bounding_box": rectangle("-71.496927,42.257263 -71.374193,42.352931") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2524960, "cityName": "Framingham" } }
+{ "create_at": datetime("2015-11-26T10:00:22.000Z"), "id": 669938760537260032, "text": "@alexkcobb hahahaha", "in_reply_to_status": 669937762162900992, "in_reply_to_user": 1306215434, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 1306215434 }}, "user": { "id": 246068988, "name": "Carl Leist", "screen_name": "cleist5", "lang": "en", "location": "Athens, OH ", "create_at": date("2011-02-01"), "description": "Roanoke College Men's Soccer, Roanoke '14, Ohio U. '16", "followers_count": 299, "friends_count": 510, "statues_count": 3826 }, "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("2015-11-26T10:00:22.000Z"), "id": 669938760876843008, "text": "Very thankful for you guys. I love you�� @lexv65 @bmpulliam @CoreyLottes @Joey092414 @demirovicanel @brookeherbert23 https://t.co/qAOpYheiE6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2602975586, 2309131500, 3122052864, 2903153783, 1694978048, 3396185381 }}, "user": { "id": 3587796258, "name": "♑️", "screen_name": "Samanth03738525", "lang": "en", "location": "Imperial, MO", "create_at": date("2015-09-16"), "description": "null", "followers_count": 187, "friends_count": 316, "statues_count": 762 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Herculaneum, MO", "id": "01b8fb0c14d42873", "name": "Herculaneum", "place_type": "city", "bounding_box": rectangle("-90.428005,38.23708 -90.369741,38.280619") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29099, "countyName": "Jefferson", "cityID": 2931708, "cityName": "Herculaneum" } }
+{ "create_at": datetime("2015-11-26T10:00:22.000Z"), "id": 669938760935542784, "text": "@colelep hey Man U should come chill https://t.co/jjyF5CHTU1", "in_reply_to_status": -1, "in_reply_to_user": 1657319132, "favorite_count": 0, "coordinate": point("-113.93239754,53.54665842"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1657319132 }}, "user": { "id": 3463354453, "name": "Riley bond", "screen_name": "natecore132", "lang": "en", "location": "canada", "create_at": date("2015-09-05"), "description": "im from canada and a gamer so come join me and chill ps4 and youtube https://m.youtube.com/channel/UCketY01JEB27z6FGglQtIkg", "followers_count": 489, "friends_count": 302, "statues_count": 875 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Spruce Grove, Alberta", "id": "06fa737d271983ea", "name": "Spruce Grove", "place_type": "city", "bounding_box": rectangle("-113.959228,53.519488 -113.847894,53.571368") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2015-11-26T10:00:22.000Z"), "id": 669938761011064832, "text": "Partly cloudy this afternoon, high 68 (20 C). Low 53 (12 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119697197, "name": "Huntsville Weather", "screen_name": "HuntsvilleWX", "lang": "en", "location": "null", "create_at": date("2010-03-04"), "description": "null", "followers_count": 665, "friends_count": 65, "statues_count": 5939 }, "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("2015-11-26T10:00:22.000Z"), "id": 669938761023778816, "text": "Happy thanksgiving", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2950686142, "name": "Mike Fragola", "screen_name": "FragolaMike", "lang": "en", "location": "null", "create_at": date("2014-12-29"), "description": "manorville NY", "followers_count": 55, "friends_count": 181, "statues_count": 61 }, "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("2015-11-26T10:00:22.000Z"), "id": 669938761149620225, "text": "I'm scared��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3087874638, "name": "bri", "screen_name": "heybrilove", "lang": "en", "location": "kcomiismybff", "create_at": date("2015-03-15"), "description": "bless up", "followers_count": 780, "friends_count": 511, "statues_count": 32947 }, "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("2015-11-26T10:00:22.000Z"), "id": 669938761174683648, "text": "@JJJordynjones @brookelgreen That is so sweet Brooke your so sweet! ������❤️ Luv u guys!!", "in_reply_to_status": 669814315067641856, "in_reply_to_user": 441598184, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 441598184, 560342044 }}, "user": { "id": 611330232, "name": "Luedizer~Jordynator~", "screen_name": "karlyannecook", "lang": "en", "location": "Utah, USA", "create_at": date("2012-06-17"), "description": "@perfectcarson on ig! Stay strong & live life to its fullest! ~Carson @jjjordynsgoals on ig! Change is never easy, but I know you can do it! ~Jordyn", "followers_count": 1177, "friends_count": 1404, "statues_count": 9435 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spanish Fork, UT", "id": "e0b39a6ce653c735", "name": "Spanish Fork", "place_type": "city", "bounding_box": rectangle("-111.693317,40.075174 -111.592655,40.156086") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4971290, "cityName": "Spanish Fork" } }
+{ "create_at": datetime("2015-11-26T10:00:22.000Z"), "id": 669938761262723073, "text": "@DA_Sticks Happy Thanksgiving Bro! Thankful I was smart enough to roll with you, the MFL site guru. #Grafix #Code #CustomerService #Respect", "in_reply_to_status": -1, "in_reply_to_user": 612868308, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Grafix", "Code", "CustomerService", "Respect" }}, "user_mentions": {{ 612868308 }}, "user": { "id": 852244236, "name": "Truffle Shuffle", "screen_name": "JEFosselman", "lang": "en", "location": "Indio, CA", "create_at": date("2012-09-28"), "description": "Husband, Father, Gentleman, and Scholar. Yinzer, Freak, X's and O's Geek.", "followers_count": 56, "friends_count": 235, "statues_count": 455 }, "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("2015-11-26T10:00:22.000Z"), "id": 669938761292259329, "text": "@alex15monreal thankful ��❤️ https://t.co/jYddg5G1vP", "in_reply_to_status": -1, "in_reply_to_user": 475249838, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 475249838 }}, "user": { "id": 38114378, "name": "DWH", "screen_name": "DWHeinzen", "lang": "en", "location": "Alaska ➡️Minnesota ✈️Chicago", "create_at": date("2009-05-05"), "description": "Lions dont lose sleep over the opinions of sheep. Veteran, fine dining, expert marksman, all around alpha male.", "followers_count": 268, "friends_count": 195, "statues_count": 23680 }, "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-26T10:00:22.000Z"), "id": 669938761342521347, "text": "Most definitely thankful for my #1 coach. @__CoachKita 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_mentions": {{ 211015307 }}, "user": { "id": 1493885166, "name": "✨Jade✨", "screen_name": "Awink_Wink", "lang": "en", "location": "null", "create_at": date("2013-06-08"), "description": "Follow @paidd_vicc Follow @AdvpeSwim_", "followers_count": 289, "friends_count": 198, "statues_count": 1601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Romulus, MI", "id": "781e991b9f95b37f", "name": "Romulus", "place_type": "city", "bounding_box": rectangle("-83.446302,42.179271 -83.306006,42.268212") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2669420, "cityName": "Romulus" } }
+{ "create_at": datetime("2015-11-26T10:00:22.000Z"), "id": 669938761740849152, "text": "No one expected stone hands to catch that. No one. Lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 334949677, "name": "Ben", "screen_name": "ben_deeznutz", "lang": "en", "location": "313", "create_at": date("2011-07-13"), "description": "Pass me the damn ball.", "followers_count": 1784, "friends_count": 2047, "statues_count": 68116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Economy, PA", "id": "018861ac17da0567", "name": "Economy", "place_type": "city", "bounding_box": rectangle("-80.240273,40.588144 -80.144845,40.718972") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42007, "countyName": "Beaver", "cityID": 4222264, "cityName": "Economy" } }
+{ "create_at": datetime("2015-11-26T10:00:22.000Z"), "id": 669938761837359104, "text": "every year there is always a huge ass fight at the Adams house and this year is no different ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 872442642, "name": "liss", "screen_name": "alissabbqadams", "lang": "en", "location": "null", "create_at": date("2012-10-10"), "description": "Just a punk with no ambition. Hmu for the good. mess with me I'll put a hammer down on you quick.", "followers_count": 353, "friends_count": 184, "statues_count": 9727 }, "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("2015-11-26T10:00:22.000Z"), "id": 669938762084827137, "text": "I'm thankful for #family #thankfulforweek #Thanksgiving", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "family", "thankfulforweek", "Thanksgiving" }}, "user": { "id": 60663466, "name": "Cory Mitchell", "screen_name": "WiscoLibrarian", "lang": "en", "location": "Menomonie, WI", "create_at": date("2009-07-27"), "description": "Wisconsin Gen X Librarian working at UW-Stout. Husband. Father. Interested in history, science, books, trends, the future, transforming libraries and MORE!", "followers_count": 254, "friends_count": 747, "statues_count": 134 }, "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("2015-11-26T10:00:22.000Z"), "id": 669938762173046784, "text": "Link in my bio @ Kenner, Louisiana https://t.co/CY2VdMKYUP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.2506,30.0096"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 311540169, "name": "TheyKnowDp", "screen_name": "Theyknowdp", "lang": "en", "location": "Kenner, Louisiana", "create_at": date("2011-06-05"), "description": "TheyKnowDp", "followers_count": 329, "friends_count": 337, "statues_count": 8182 }, "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("2015-11-26T10:00:22.000Z"), "id": 669938762433101824, "text": "@wichitacomedy @MidWest_Digest @ubreastbelieve got beat by 2 girls. I didn't even know girls were funny #FunniestPersonInWichita @smrtgrls", "in_reply_to_status": -1, "in_reply_to_user": 2859945982, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FunniestPersonInWichita" }}, "user_mentions": {{ 2859945982, 3021799742, 3418863694, 113439399 }}, "user": { "id": 601028120, "name": "Travis Cagle", "screen_name": "Trav_is_Funny", "lang": "en", "location": "Wichita, KS", "create_at": date("2012-06-06"), "description": "Stand Up Comedian, runner up in FoD's #Road2Oddball in 2014. Attitude of Matthew McCaughey in the body of Dakota Fanning.", "followers_count": 301, "friends_count": 364, "statues_count": 5916 }, "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("2015-11-26T10:00:22.000Z"), "id": 669938762605047808, "text": "It's beautiful out", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.98095721,39.66809588"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 422680479, "name": "bean", "screen_name": "UhOh_Oreo33", "lang": "en", "location": "null", "create_at": date("2011-11-27"), "description": "you need a girl like me around", "followers_count": 1492, "friends_count": 591, "statues_count": 77245 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monroe, NJ", "id": "0089474a03d2aa78", "name": "Monroe", "place_type": "city", "bounding_box": rectangle("-75.003738,39.645781 -74.961134,39.700209") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34015, "countyName": "Gloucester", "cityID": 3481380, "cityName": "Williamstown" } }
+{ "create_at": datetime("2015-11-26T10:00:22.000Z"), "id": 669938762684751872, "text": "I'm thankful ion smoke mid", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2940771983, "name": "BKING", "screen_name": "BaileyKing78", "lang": "en", "location": "864 | 812", "create_at": date("2014-12-25"), "description": "SBW", "followers_count": 505, "friends_count": 386, "statues_count": 2505 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shepherdsville, KY", "id": "df1c1006357a743f", "name": "Shepherdsville", "place_type": "city", "bounding_box": rectangle("-85.767399,37.963139 -85.624467,38.035906") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21029, "countyName": "Bullitt", "cityID": 2170086, "cityName": "Shepherdsville" } }
+{ "create_at": datetime("2015-11-26T10:00:22.000Z"), "id": 669938762986577920, "text": "@RichieMaselow thanks richard����", "in_reply_to_status": 669936365346263041, "in_reply_to_user": 377694160, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 377694160 }}, "user": { "id": 412685393, "name": "Sarah Sherrick", "screen_name": "sarahhnicolee_", "lang": "en", "location": "null", "create_at": date("2011-11-14"), "description": "null", "followers_count": 477, "friends_count": 463, "statues_count": 5403 }, "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("2015-11-26T10:00:22.000Z"), "id": 669938763280220161, "text": "Mood https://t.co/sW6JrOATZw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 840379986, "name": "A-Kennedy", "screen_name": "AaronBro16", "lang": "en", "location": "Houston, TX", "create_at": date("2012-09-22"), "description": "Navarro college. I'm Mysterious. 6'2 190", "followers_count": 1009, "friends_count": 892, "statues_count": 10128 }, "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("2015-11-26T10:00:22.000Z"), "id": 669938763536031744, "text": "Partly cloudy this afternoon, high 69 (21 C). Low 53 (12 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119698223, "name": "Birmingham Weather", "screen_name": "BirminghamWX", "lang": "en", "location": "Birmingham, AL", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Birmingham, AL", "followers_count": 502, "friends_count": 68, "statues_count": 5951 }, "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("2015-11-26T10:00:22.000Z"), "id": 669938763791896576, "text": "#SUFFOLK, VA #Labor #Job: Hospital at Remedy Intelligent Staffing https://t.co/PnXyBAzWuS #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.5950555,36.6644676"), "retweet_count": 0, "lang": "no", "is_retweet": false, "hashtags": {{ "SUFFOLK", "Labor", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 22779528, "name": "TMJ-ORF Labor Jobs", "screen_name": "tmj_orf_labor", "lang": "en", "location": "Norfolk area, VA", "create_at": date("2009-03-04"), "description": "Follow this account for geo-targeted General Labor job tweets in Norfolk area, VA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 300, "friends_count": 290, "statues_count": 12 }, "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": 51800, "countyName": "Suffolk", "cityID": 5176432, "cityName": "Suffolk" } }
+{ "create_at": datetime("2015-11-26T10:00:22.000Z"), "id": 669938763968155649, "text": "Time Traveler with a mission! Is he just and man? a superhero? or something more? find out from the beginning! https://t.co/LWWtItgxlh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3235735445, "name": "Michael Marcondes", "screen_name": "authormrmike059", "lang": "en", "location": "Massachusetts, USA", "create_at": date("2015-05-05"), "description": "Author sci-f- bk The Traveler the beginning picked up by Tate Publishing available now & Christian non-fiction Life Explained a Journey to Selfless Love 6/15", "followers_count": 1174, "friends_count": 1999, "statues_count": 2075 }, "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("2015-11-26T10:00:22.000Z"), "id": 669938764316323840, "text": "I'm just ready to eat... 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": 2298798373, "name": "|\\| ATHANIEL Cox", "screen_name": "Nathaniel_cox25", "lang": "en", "location": "null", "create_at": date("2014-01-18"), "description": "Rampage Hockey ~ #25 •OBKY•", "followers_count": 335, "friends_count": 77, "statues_count": 1488 }, "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("2015-11-26T10:00:22.000Z"), "id": 669938764391817216, "text": "not even gonna 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": 1539045884, "name": "Jen Green", "screen_name": "_jeeennn_", "lang": "en", "location": "Livonia, MI", "create_at": date("2013-06-22"), "description": "Fhs // Brendan", "followers_count": 876, "friends_count": 842, "statues_count": 11205 }, "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("2015-11-26T10:00:22.000Z"), "id": 669938764396011520, "text": "@mariacirincio Part 2 https://t.co/IVlAKXawM3", "in_reply_to_status": -1, "in_reply_to_user": 2416406834, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2416406834 }}, "user": { "id": 2416406834, "name": "Mar", "screen_name": "mariacirincio", "lang": "en", "location": "null", "create_at": date("2014-03-28"), "description": "my names blurry face", "followers_count": 364, "friends_count": 465, "statues_count": 1662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Medina, OH", "id": "00eb8de9015a3185", "name": "Medina", "place_type": "city", "bounding_box": rectangle("-81.907176,41.089056 -81.774633,41.215965") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39103, "countyName": "Medina", "cityID": 3948790, "cityName": "Medina" } }
+{ "create_at": datetime("2015-11-26T10:00:22.000Z"), "id": 669938764563763200, "text": "https://t.co/OoTElE6bF6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1000811852, "name": "John Wilcox", "screen_name": "John__Wilcox", "lang": "en", "location": "United States", "create_at": date("2012-12-09"), "description": "South Forsyth\nsc-redwings23", "followers_count": 109, "friends_count": 489, "statues_count": 431 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Farmington Hills, MI", "id": "0b40afc99807b6ac", "name": "Farmington Hills", "place_type": "city", "bounding_box": rectangle("-83.437523,42.439001 -83.316839,42.529556") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2627440, "cityName": "Farmington Hills" } }
+{ "create_at": datetime("2015-11-26T10:00:23.000Z"), "id": 669938764785954816, "text": "RT TO VOTE \n\n#MTVStars One Direction https://t.co/4JByRxf2U2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MTVStars" }}, "user": { "id": 2304698872, "name": "meg || 0/4 0/5", "screen_name": "narrytomycake", "lang": "en", "location": "luke's bed", "create_at": date("2014-01-25"), "description": "my favorite season is concert season", "followers_count": 1545, "friends_count": 1353, "statues_count": 23975 }, "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("2015-11-26T10:00:23.000Z"), "id": 669938764886634496, "text": "I be dumb yea. But 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": 2770558272, "name": "Donovan MF Mann〽️", "screen_name": "BasketballMann5", "lang": "en", "location": "Omaha, NE", "create_at": date("2014-08-26"), "description": "God First & Everything Else Will Follow ✊ Creighton Blue Jays #CU19 ➿", "followers_count": 313, "friends_count": 299, "statues_count": 5023 }, "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-26T10:00:23.000Z"), "id": 669938764932739072, "text": "And lunch and dinner apparently lol. https://t.co/U0dv6MZsfo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 38817415, "name": "Stelio Savante", "screen_name": "StelioSavante", "lang": "en", "location": "null", "create_at": date("2009-05-09"), "description": "SouthAfrican NYER @USAMovieAwards winner @SAGawards nominee Celiac @EisensteinGuana #TheMakingOfTheMob @AMC_TV @WTRROmovie", "followers_count": 12795, "friends_count": 915, "statues_count": 17758 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:23.000Z"), "id": 669938764945338372, "text": "pissed my whole soul off", "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": "queeenri✨", "screen_name": "richbastard", "lang": "en", "location": "boolin w/ my meesh ❤️", "create_at": date("2015-07-26"), "description": "null", "followers_count": 559, "friends_count": 458, "statues_count": 20958 }, "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("2015-11-26T10:00:23.000Z"), "id": 669938764983230464, "text": "I need this https://t.co/L9HDJIKXTl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 380509449, "name": "REAL MVP", "screen_name": "GKRONSKI", "lang": "en", "location": "New York, USA", "create_at": date("2011-09-26"), "description": "Don't choose the one who is beautiful to the world. But rather, choose the one who makes your world beautiful.", "followers_count": 333, "friends_count": 1503, "statues_count": 2259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dunkirk, NY", "id": "7561ff4fba7e382d", "name": "Dunkirk", "place_type": "city", "bounding_box": rectangle("-79.400179,42.45681 -79.295345,42.506593") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36013, "countyName": "Chautauqua", "cityID": 3621105, "cityName": "Dunkirk" } }
+{ "create_at": datetime("2015-11-26T10:00:23.000Z"), "id": 669938765087969280, "text": "I haven met a rapper that's richer than a trapper ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 323010858, "name": "MikeMike™ ®", "screen_name": "SBMIKEMIKE", "lang": "en", "location": "Straight Outta Compton", "create_at": date("2011-06-23"), "description": "I'm the type to make a prayer then go get what I prayed for #TeamTatted", "followers_count": 1412, "friends_count": 1008, "statues_count": 41556 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Culpeper, VA", "id": "d48d37537bc28a6f", "name": "Culpeper", "place_type": "city", "bounding_box": rectangle("-78.028546,38.445607 -77.972087,38.497639") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51047, "countyName": "Culpeper", "cityID": 5120752, "cityName": "Culpeper" } }
+{ "create_at": datetime("2015-11-26T10:00:23.000Z"), "id": 669938765327036416, "text": "Just saw a boob. I feel weird with public breastfeeding", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17631797, "name": "Nissa", "screen_name": "nissarevenge", "lang": "en", "location": "null", "create_at": date("2008-11-25"), "description": "Taking one day at a time.", "followers_count": 294, "friends_count": 1023, "statues_count": 11066 }, "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-26T10:00:23.000Z"), "id": 669938765490622464, "text": "#IT #Job in #Seattle, Washington: Delivery Process Manager at Nordstrom https://t.co/TN3MzsZfQn #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.3320708,47.6062095"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "IT", "Job", "Seattle", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 21676969, "name": "TMJ - SEA IT Jobs", "screen_name": "tmj_sea_it", "lang": "en", "location": "USA-WA-Seattle Metro", "create_at": date("2009-02-23"), "description": "Follow this account for geo-targeted Software Dev. - General/IT job tweets in Seattle, WA. Need help? Tweet us at @CareerArc!", "followers_count": 536, "friends_count": 329, "statues_count": 681 }, "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-26T10:00:23.000Z"), "id": 669938765532545025, "text": "I'm thankful my work isn't open on thanksgiving ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1589860183, "name": "daddy", "screen_name": "sammymor_", "lang": "en", "location": "LV to HB", "create_at": date("2013-07-12"), "description": "beets, bears, battlestar galactica | intersectional feminist", "followers_count": 252, "friends_count": 179, "statues_count": 4492 }, "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("2015-11-26T10:00:23.000Z"), "id": 669938765729632256, "text": "My goal is to try all the high end foundations and see how much 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": 2949439387, "name": "Tay", "screen_name": "TheTaylahh", "lang": "en", "location": "null", "create_at": date("2014-12-28"), "description": "on my own wave", "followers_count": 727, "friends_count": 366, "statues_count": 13928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:23.000Z"), "id": 669938765754970113, "text": "Won't let nobody hurt you #ALDUBDontGiveUpOnUs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ALDUBDontGiveUpOnUs" }}, "user": { "id": 3702822976, "name": "strawberry cream", "screen_name": "noralyn_bello", "lang": "en", "location": "Fort Lauderdale, FL", "create_at": date("2015-09-18"), "description": "null", "followers_count": 310, "friends_count": 364, "statues_count": 24810 }, "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("2015-11-26T10:00:23.000Z"), "id": 669938766052618241, "text": "Mostly sunny this afternoon, high 73 (23 C). Low 59 (15 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119698549, "name": "Mobile Weather", "screen_name": "MobileWX", "lang": "en", "location": "Mobile, Al", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Mobile, Al", "followers_count": 387, "friends_count": 68, "statues_count": 6026 }, "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("2015-11-26T10:00:23.000Z"), "id": 669938766455271424, "text": "Sit at the big boy tables", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2250903397, "name": "P-Low", "screen_name": "pablomarquez21", "lang": "en", "location": "Don't trip ", "create_at": date("2013-12-17"), "description": "I'm a savage", "followers_count": 394, "friends_count": 329, "statues_count": 5125 }, "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("2015-11-26T10:00:23.000Z"), "id": 669938766493057024, "text": "Every morning I wake up wishing everything was different and in place.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157947100, "name": "wildfl☮wer", "screen_name": "viva__nessa", "lang": "en", "location": "front left, by the speakers", "create_at": date("2010-06-21"), "description": "Vibing & Thriving ❁ To each her own", "followers_count": 1120, "friends_count": 1443, "statues_count": 21110 }, "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("2015-11-26T10:00:23.000Z"), "id": 669938766501408768, "text": "Your nobody , till somebody kills you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3155115396, "name": "alice ashley.", "screen_name": "Whitegirlalicee", "lang": "en", "location": "null", "create_at": date("2015-04-14"), "description": "19", "followers_count": 222, "friends_count": 87, "statues_count": 2070 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:23.000Z"), "id": 669938766577016836, "text": "BØRNS was too good yesterday �� bornsmusic kdorado @ The Observatory Orange County https://t.co/vP5Bn8cZB3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.91769226,33.69974996"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 740302880, "name": "Kevin Kwan-Martinez", "screen_name": "kevin_929", "lang": "en", "location": "Diamond Bar, CA", "create_at": date("2012-08-06"), "description": "null", "followers_count": 115, "friends_count": 133, "statues_count": 1809 }, "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("2015-11-26T10:00:23.000Z"), "id": 669938766816002048, "text": "@IanHousman yes we should", "in_reply_to_status": 669938494488231936, "in_reply_to_user": 525903644, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 525903644 }}, "user": { "id": 281288900, "name": "Andrew Hua", "screen_name": "Skull_God", "lang": "en", "location": "The Current Universe", "create_at": date("2011-04-12"), "description": "Part of Gopher Squad, Anthromorphic personification of Sarcasm, Fan of Secret Dota, and PPMD/M2k Smash, Smash Tag is Coward", "followers_count": 139, "friends_count": 189, "statues_count": 7955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:23.000Z"), "id": 669938766866489344, "text": "OMFFGGGGGG https://t.co/ggDZtnsope", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "sv", "is_retweet": false, "user": { "id": 3429265804, "name": "pumpkin(pie)icolle", "screen_name": "partycarty_", "lang": "en", "location": "Florida, USA", "create_at": date("2015-08-17"), "description": "logy", "followers_count": 287, "friends_count": 502, "statues_count": 913 }, "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("2015-11-26T10:00:23.000Z"), "id": 669938767180922881, "text": "While I might be doing it alone, this is the first thanksgiving I've gotten to enjoy in six years.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 572303745, "name": "Ava", "screen_name": "norveblue", "lang": "en", "location": "Minot, ND", "create_at": date("2012-05-05"), "description": "Furry artist, e621 admin, laboratory stockroom assistant, biology major/chemistry minor.", "followers_count": 57, "friends_count": 14, "statues_count": 4370 }, "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("2015-11-26T10:00:23.000Z"), "id": 669938767487193089, "text": "@sntn300 what nigga", "in_reply_to_status": 669938688038711296, "in_reply_to_user": 2491960436, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2491960436 }}, "user": { "id": 3602608036, "name": "xxx", "screen_name": "mafi_jr", "lang": "en", "location": "null", "create_at": date("2015-09-09"), "description": "null", "followers_count": 136, "friends_count": 124, "statues_count": 442 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Guttenberg, NJ", "id": "b6f395ab4c5d530d", "name": "Guttenberg", "place_type": "city", "bounding_box": rectangle("-74.015657,40.788742 -73.996756,40.79885") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3428650, "cityName": "Guttenberg" } }
+{ "create_at": datetime("2015-11-26T10:00:23.000Z"), "id": 669938767541592064, "text": "@love_me327 I love you ��❤️��", "in_reply_to_status": 669938391706787840, "in_reply_to_user": 1869547062, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1869547062 }}, "user": { "id": 2660387574, "name": "Masochist♠️", "screen_name": "jacypimpin", "lang": "en", "location": "Somewhere in Panama ", "create_at": date("2014-07-19"), "description": "10.17.15✨ PHS Guard✊ Panamanian", "followers_count": 942, "friends_count": 1225, "statues_count": 1130 }, "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("2015-11-26T10:00:23.000Z"), "id": 669938767575191552, "text": "Hey @PandaExpress I'm spending thanksgiving with you. Alone. 1500 miles away from home. Help me out? I just want double orange chicken.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24192548 }}, "user": { "id": 2416832007, "name": "David Standlee", "screen_name": "thechubbywubby", "lang": "en", "location": "On Tour", "create_at": date("2014-03-16"), "description": "tour manager. front of house engineer. merch guy. disc golfer. bearded guy. bye.", "followers_count": 173, "friends_count": 81, "statues_count": 1057 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:23.000Z"), "id": 669938767621394432, "text": "@LeBronJames Anthony Davis", "in_reply_to_status": 667366328987054080, "in_reply_to_user": 342601720, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 342601720 }}, "user": { "id": 404634862, "name": "448", "screen_name": "DayOne_DUDLEY", "lang": "en", "location": "Chasin' Paper", "create_at": date("2011-11-03"), "description": "mama called me on my phone ask me if I'm still trappin? I'm like no ma'am!", "followers_count": 782, "friends_count": 862, "statues_count": 34124 }, "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("2015-11-26T10:00:23.000Z"), "id": 669938767755501568, "text": "Team snapchat is the only person who sends me cute thanksgiving things https://t.co/18RRQVH1VF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 499228089, "name": "cal", "screen_name": "calliemelancon", "lang": "en", "location": "Texas, USA", "create_at": date("2012-02-21"), "description": "i probably like ur dog more than i like u snapchat//calliemelancon", "followers_count": 293, "friends_count": 185, "statues_count": 185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Groves, TX", "id": "9af98223aefe3151", "name": "Groves", "place_type": "city", "bounding_box": rectangle("-93.944772,29.924735 -93.890543,29.96429") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48245, "countyName": "Jefferson", "cityID": 4831328, "cityName": "Groves" } }
+{ "create_at": datetime("2015-11-26T10:00:23.000Z"), "id": 669938767961157633, "text": "@RachelELuce @EmilyPereiraaa love you!!!", "in_reply_to_status": 669936689352060928, "in_reply_to_user": 615396645, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 615396645, 501329428 }}, "user": { "id": 598877689, "name": "Kennedy Lord", "screen_name": "klord1997", "lang": "en", "location": "null", "create_at": date("2012-06-03"), "description": "UConn-Avery Point Class of 2019. 03/10/15", "followers_count": 323, "friends_count": 397, "statues_count": 8246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pawcatuck, CT", "id": "c9cdbca03a00179b", "name": "Pawcatuck", "place_type": "city", "bounding_box": rectangle("-71.891122,41.335773 -71.830086,41.408715") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9011, "countyName": "New London", "cityID": 959140, "cityName": "Pawcatuck" } }
+{ "create_at": datetime("2015-11-26T10:00:23.000Z"), "id": 669938768015548416, "text": "@Jack_Septic_Eye oh Jack :P", "in_reply_to_status": 669934534440443904, "in_reply_to_user": 77596200, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 77596200 }}, "user": { "id": 30790987, "name": "Janet LaCava", "screen_name": "JanetLaCava", "lang": "en", "location": "San Mateo, CA", "create_at": date("2009-04-12"), "description": "I enjoy Tinkerbell, Five Nights At Freddys, Markiplier & all his friends, & the Minions! I'm a 43 year old FanWoman extraordinaire! I am a Professional Dumbass!", "followers_count": 1064, "friends_count": 49, "statues_count": 72661 }, "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("2015-11-26T10:00:23.000Z"), "id": 669938768099471360, "text": "Happy turkey day fuck heads. Gobble gobble.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3094332041, "name": "Syko-Clause", "screen_name": "SykoticED", "lang": "en", "location": "null", "create_at": date("2015-03-18"), "description": "Just some internet pleb. Part of Warforged Legion - Emerald Dream (Horde)", "followers_count": 199, "friends_count": 210, "statues_count": 944 }, "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("2015-11-26T10:00:23.000Z"), "id": 669938768296579072, "text": "I 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": 449230130, "name": "d ☯", "screen_name": "ayodeanna", "lang": "en", "location": "az", "create_at": date("2011-12-28"), "description": "I'm 21 // snapchat: ayodeannaa", "followers_count": 249, "friends_count": 188, "statues_count": 14652 }, "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("2015-11-26T10:00:23.000Z"), "id": 669938768359636993, "text": "Oo driving a truck for the first time today ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 37512785, "name": "Dey", "screen_name": "_beanzz", "lang": "en", "location": "JERSEY", "create_at": date("2009-05-03"), "description": "north side of philly #TU18", "followers_count": 2470, "friends_count": 1591, "statues_count": 95898 }, "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("2015-11-26T10:00:23.000Z"), "id": 669938768405618688, "text": "@ItsLuxray LUX WE DID IT HAHAHA https://t.co/9LexCKDq6m", "in_reply_to_status": -1, "in_reply_to_user": 2738776807, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2738776807 }}, "user": { "id": 2817593135, "name": "Aaron_", "screen_name": "Pandr_", "lang": "en", "location": "Mark | Sky | Taylor | Spooky", "create_at": date("2014-10-08"), "description": "null", "followers_count": 97, "friends_count": 97, "statues_count": 5728 }, "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-26T10:00:23.000Z"), "id": 669938768451870724, "text": "white people season their food too", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2165762036, "name": "kelton", "screen_name": "Kjelly_68", "lang": "en", "location": "glory hole", "create_at": date("2013-10-30"), "description": "Kelton Jones// Calumet college wrestling 285/You can catch me with the Devil playin' 21", "followers_count": 535, "friends_count": 589, "statues_count": 15359 }, "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("2015-11-26T10:00:23.000Z"), "id": 669938768456105986, "text": "My oG told me years ago this ����will make you laugh make you cry ..smh get… https://t.co/s0pMtD5m5Z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.15977367,40.04982448"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 169325285, "name": "Uncle Bob", "screen_name": "MaTiCMaN215", "lang": "en", "location": "pe0n fr33", "create_at": date("2010-07-21"), "description": "..stop worrying about what might hold you down and pay attention to why you're standing up. Follow @WyoMusic", "followers_count": 936, "friends_count": 948, "statues_count": 98738 }, "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-26T10:00:23.000Z"), "id": 669938768585986049, "text": "I don't think I need anyone else when I have such an amazing sister! #foreverblessed ����#happythanksgiving https://t.co/RMG0IdDU0n", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "foreverblessed", "happythanksgiving" }}, "user": { "id": 119254692, "name": "isa", "screen_name": "yoo_iSaa", "lang": "en", "location": "reeferside.", "create_at": date("2010-03-02"), "description": "blessed; boba is life", "followers_count": 449, "friends_count": 1127, "statues_count": 52999 }, "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("2015-11-26T10:00:23.000Z"), "id": 669938768611119104, "text": "Morning rain this afternoon, high 58 (14 C). Low 42 (6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119751854, "name": "Chicago Weather", "screen_name": "Chicago_WX", "lang": "en", "location": "Chicago, IL", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Chicago, IL", "followers_count": 1056, "friends_count": 68, "statues_count": 6308 }, "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-26T10:00:23.000Z"), "id": 669938768686800896, "text": "Apparently Mama Neidig doesn't approve of my drug rug+college sweatpants ensemble for Thanksgiving today.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 158863124, "name": "Kat Neidig", "screen_name": "_katherine", "lang": "en", "location": "Long Island, NY", "create_at": date("2010-06-23"), "description": "null", "followers_count": 460, "friends_count": 611, "statues_count": 19400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Islip, NY", "id": "c3940942f7542565", "name": "West Islip", "place_type": "city", "bounding_box": rectangle("-73.31844,40.684315 -73.273577,40.74145") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3680302, "cityName": "West Islip" } }
+{ "create_at": datetime("2015-11-26T10:00:24.000Z"), "id": 669938768812486656, "text": "I have discovered the tastiest protein pancake recipe known to man.. oh what 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": 605571793, "name": "Neil", "screen_name": "rastapastaaa", "lang": "en", "location": "null", "create_at": date("2012-06-11"), "description": "DJ \nASU '19", "followers_count": 466, "friends_count": 309, "statues_count": 14617 }, "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-26T10:00:24.000Z"), "id": 669938769416601601, "text": "...like be happy for me and wait we're still good and we're still a power together it may not be known to the world yet but we know...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 504621083, "name": "clare.", "screen_name": "theclarencerack", "lang": "en", "location": "Chicago, IL", "create_at": date("2012-02-26"), "description": "Miserable In Chicago", "followers_count": 238, "friends_count": 48, "statues_count": 1423 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crown Point, IN", "id": "8890a39a2dfcd43f", "name": "Crown Point", "place_type": "city", "bounding_box": rectangle("-87.402263,41.376611 -87.294472,41.453834") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18089, "countyName": "Lake", "cityID": 1816138, "cityName": "Crown Point" } }
+{ "create_at": datetime("2015-11-26T10:00:24.000Z"), "id": 669938769475293185, "text": "nice win boys ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 544426602, "name": "P. Sawyer", "screen_name": "srosee22", "lang": "en", "location": "Folsom, PA", "create_at": date("2012-04-03"), "description": "#fuckconnie rhs '17", "followers_count": 347, "friends_count": 197, "statues_count": 10751 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Folsom, PA", "id": "c8d23c1f5d6736ea", "name": "Folsom", "place_type": "city", "bounding_box": rectangle("-75.341968,39.882641 -75.29943,39.916604") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42045, "countyName": "Delaware", "cityID": 4226432, "cityName": "Folsom" } }
+{ "create_at": datetime("2015-11-26T10:00:24.000Z"), "id": 669938769886183424, "text": "My three men out for a morning Thanksgiving snorkel. #catalinaisland… https://t.co/MUoaN3PlM0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.33104417,33.35450839"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "catalinaisland" }}, "user": { "id": 300464793, "name": "mlblvr6", "screen_name": "mlblvr6", "lang": "en", "location": "Laguna Niguel, CA", "create_at": date("2011-05-17"), "description": "just a girl who likes to voice her opinion.", "followers_count": 32, "friends_count": 201, "statues_count": 875 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Avalon, CA", "id": "7a9d1044bce5cd3b", "name": "Avalon", "place_type": "city", "bounding_box": rectangle("-118.342496,33.330327 -118.309367,33.357104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 603274, "cityName": "Avalon" } }
+{ "create_at": datetime("2015-11-26T10:00:24.000Z"), "id": 669938769961816064, "text": "@AKBrownLaw @TerrorViewApp @NatSecJosh hope you are enjoying your #Thanksgiving with your family! Great to meet you last night!", "in_reply_to_status": 669726052688752640, "in_reply_to_user": 2613927661, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Thanksgiving" }}, "user_mentions": {{ 2613927661, 3153848417, 1502396388 }}, "user": { "id": 750325616, "name": "Carrie Enright Katz", "screen_name": "lexknitter", "lang": "en", "location": "Alexandria, VA", "create_at": date("2012-08-10"), "description": "wife, mother, bibliophile, laundry and stain removing queen, ouchie fixer and head of the tucking in committee.", "followers_count": 337, "friends_count": 1182, "statues_count": 2143 }, "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("2015-11-26T10:00:24.000Z"), "id": 669938770054004736, "text": "@UrquizaIliana hope you're having a great thanksgiving! ��", "in_reply_to_status": 669924731684958209, "in_reply_to_user": 2764542387, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2764542387 }}, "user": { "id": 342576310, "name": "☽❁♡ℳaryyy♡❁☾", "screen_name": "MaryannHoang1", "lang": "en", "location": "Tempe, AZ", "create_at": date("2011-07-26"), "description": "hi, i'm maryann and i'm a chronic pillow hoarder. ☯ #GoDevils", "followers_count": 1250, "friends_count": 772, "statues_count": 46805 }, "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("2015-11-26T10:00:24.000Z"), "id": 669938770121113600, "text": "when your manager group messages you and your coworkers happy thanksgiving>>>", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 272217454, "name": "sensei sumar", "screen_name": "SumarNguyen", "lang": "en", "location": "null", "create_at": date("2011-03-25"), "description": "is the juice worth the squeeze?", "followers_count": 159, "friends_count": 80, "statues_count": 10183 }, "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("2015-11-26T10:00:24.000Z"), "id": 669938770565865473, "text": "���� https://t.co/Azw1IkJtnj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2302571242, "name": "jamirah.", "screen_name": "jamirahk", "lang": "en", "location": "durham, nc", "create_at": date("2014-01-24"), "description": "#nonchalant", "followers_count": 193, "friends_count": 228, "statues_count": 7880 }, "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("2015-11-26T10:00:24.000Z"), "id": 669938770637008898, "text": "so im watching the dog show and the pug comes on and i start screaming and crying", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 737043320, "name": "sloths ☃", "screen_name": "bridgeope", "lang": "en", "location": "Central America", "create_at": date("2012-08-04"), "description": "i love sloths and pugs.", "followers_count": 581, "friends_count": 141, "statues_count": 40556 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highspire, PA", "id": "e22e2426eb4c3601", "name": "Highspire", "place_type": "city", "bounding_box": rectangle("-76.800266,40.200736 -76.768607,40.215292") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42043, "countyName": "Dauphin", "cityID": 4234664, "cityName": "Highspire" } }
+{ "create_at": datetime("2015-11-26T10:00:24.000Z"), "id": 669938770758602752, "text": "@heyitssammyj every year I wait for it", "in_reply_to_status": 669938718518611970, "in_reply_to_user": 304096913, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 304096913 }}, "user": { "id": 549659473, "name": "Voldemort", "screen_name": "Savannaahkaay", "lang": "en", "location": "null", "create_at": date("2012-04-09"), "description": "♍️♋️ OG Ginge", "followers_count": 240, "friends_count": 187, "statues_count": 25945 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:24.000Z"), "id": 669938770884579329, "text": "������ RT @LRNROSE: RT @Mr1738: damn RT @_KTesq: She's done ���� https://t.co/1hqLJw0IKh", "in_reply_to_status": 669935422798217217, "in_reply_to_user": 33727151, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 33727151, 29065153, 19009407 }}, "user": { "id": 176204947, "name": "Chris DiLalla", "screen_name": "EvryoneH8sChris", "lang": "en", "location": "Rochester, NY", "create_at": date("2010-08-08"), "description": "26 | Barber | A Little Off The Top, 2092 E.Henrietta Rd the dude you see out that's on his phone all night, that's me...", "followers_count": 883, "friends_count": 1982, "statues_count": 80384 }, "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("2015-11-26T10:00:24.000Z"), "id": 669938771056599040, "text": "Omfg @alyssa_gleason �������� https://t.co/5VXGSzhct6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 610990466 }}, "user": { "id": 724307378, "name": "Jerres Annie", "screen_name": "jerresannie", "lang": "en", "location": "Ohio", "create_at": date("2012-07-29"), "description": "brookside senior // 5'3", "followers_count": 354, "friends_count": 96, "statues_count": 3784 }, "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-26T10:00:24.000Z"), "id": 669938771169828864, "text": "5 mfs already came to my house asking for a plate, this is not Halloween. We are not passing out Turkey legs.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 501836396, "name": ".", "screen_name": "Love_Deoniss", "lang": "en", "location": "null", "create_at": date("2012-02-24"), "description": "|Tech High| Seniors | C/o 2016 ✨|", "followers_count": 1209, "friends_count": 1019, "statues_count": 17602 }, "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("2015-11-26T10:00:24.000Z"), "id": 669938771215822848, "text": "Partly cloudy this afternoon, high 73 (23 C). Low 58 (14 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119753180, "name": "Jackson Weather", "screen_name": "JacksonWX", "lang": "en", "location": "Jackson, MS", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Jackson, MS", "followers_count": 294, "friends_count": 68, "statues_count": 6257 }, "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("2015-11-26T10:00:24.000Z"), "id": 669938771668942848, "text": "'Bout time these boots saw Texas @ Ben Wheeler, Texas https://t.co/ngFu8AGGp3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.7036,32.4458"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 291204512, "name": "Cydney Barrick", "screen_name": "cydlou13", "lang": "en", "location": "Pitt State", "create_at": date("2011-05-01"), "description": "You were wild once. Don't let them tame you. Alpha Sigma Alpha", "followers_count": 252, "friends_count": 235, "statues_count": 3153 }, "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": 48467, "countyName": "Van Zandt" } }
+{ "create_at": datetime("2015-11-26T10:00:24.000Z"), "id": 669938771744288768, "text": "starts thanksgiving day off by watching human centipede", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 405203228, "name": "Bri Williams", "screen_name": "srslybri", "lang": "en", "location": "9o'1 ⇒ 'nooga", "create_at": date("2011-11-04"), "description": "i'm just trying to change the color on ur mood ring.《 nobody's baby, everybody's girl 》", "followers_count": 468, "friends_count": 374, "statues_count": 10928 }, "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("2015-11-26T10:00:24.000Z"), "id": 669938771975151616, "text": "#happythanksgiving To My Brothers Enjoy might pop up ���� @fayne_mark40 @LightSkinFusion @_JayDaReal @4674DBLOCK_BOYZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "happythanksgiving" }}, "user_mentions": {{ 569605225, 226273418, 203979747, 352883290 }}, "user": { "id": 4097253441, "name": "✨Pablo Escobar✨", "screen_name": "Godly_Lito", "lang": "en", "location": "Middletown, OH", "create_at": date("2015-11-02"), "description": "Columbus✈️Middletown | IG:Punchanella_100 | SC:BlurryBoyTyquon | Past The Moon & Stars✨", "followers_count": 142, "friends_count": 307, "statues_count": 391 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Middletown, OH", "id": "d417541590918ddf", "name": "Middletown", "place_type": "city", "bounding_box": rectangle("-84.424882,39.463478 -84.298183,39.548729") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39017, "countyName": "Butler", "cityID": 3949840, "cityName": "Middletown" } }
+{ "create_at": datetime("2015-11-26T10:00:24.000Z"), "id": 669938772096606210, "text": "@Stephen6363 @ScintillateMe @herby2121 @jopower666 @FollowYayu yummy ;p I'm sure that I can make you smile!", "in_reply_to_status": 669883848251969536, "in_reply_to_user": 405843511, "favorite_count": 0, "coordinate": point("-116.2630899,43.6475026"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 405843511, 904410547, 2274691870, 297302006, 1659849062 }}, "user": { "id": 37099297, "name": "Rodney M. Warren", "screen_name": "Schlaavey", "lang": "en", "location": "Boise, ID", "create_at": date("2009-05-01"), "description": "Young man searching!", "followers_count": 400, "friends_count": 2026, "statues_count": 3812 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden City, ID", "id": "1119ae872cd8502e", "name": "Garden City", "place_type": "city", "bounding_box": rectangle("-116.314121,43.619007 -116.232073,43.682482") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16001, "countyName": "Ada", "cityID": 1629620, "cityName": "Garden City" } }
+{ "create_at": datetime("2015-11-26T10:00:24.000Z"), "id": 669938772172124160, "text": "Thanksgiving is just more family members than the usual here to insult you ������������������������������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1229124222, "name": ":(:", "screen_name": "ericaxzuniga", "lang": "en", "location": "Williams, CA", "create_at": date("2013-02-28"), "description": "i brake my own heart", "followers_count": 493, "friends_count": 522, "statues_count": 12257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Williams, CA", "id": "95c37fdaf2238a36", "name": "Williams", "place_type": "city", "bounding_box": rectangle("-122.168439,39.130782 -122.132178,39.165799") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6011, "countyName": "Colusa", "cityID": 685586, "cityName": "Williams" } }
+{ "create_at": datetime("2015-11-26T10:00:24.000Z"), "id": 669938772256124928, "text": "@hey_claritza @WeKitties @WeTigersss Happy Thanksgiving! #veganthanksgiving https://t.co/mMeujVvmhn", "in_reply_to_status": -1, "in_reply_to_user": 1736745144, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "veganthanksgiving" }}, "user_mentions": {{ 1736745144, 2209379956, 2510972006 }}, "user": { "id": 567739752, "name": "william ramirez", "screen_name": "wplusr", "lang": "en", "location": "Fort Myers, FL", "create_at": date("2012-04-30"), "description": "연금술사 (Alchemist) @PennageUSA @GoldenKeyFL", "followers_count": 999, "friends_count": 447, "statues_count": 831 }, "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("2015-11-26T10:00:24.000Z"), "id": 669938772289671169, "text": "@kalambalikis I HOPE YOU BECOME A DOG SHOW JISGE", "in_reply_to_status": 669934900863246337, "in_reply_to_user": 629544692, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 629544692 }}, "user": { "id": 1488907591, "name": "sASHaparilla", "screen_name": "ashlei_anderson", "lang": "en", "location": "Flarda ", "create_at": date("2013-06-06"), "description": "22• Turtle enthusiast •Sensitive Pisces• I don't know what I'm doing on here•", "followers_count": 162, "friends_count": 225, "statues_count": 2823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Venice Gardens, FL", "id": "5f079658f0aa29e6", "name": "Venice Gardens", "place_type": "city", "bounding_box": rectangle("-82.425228,27.057244 -82.384742,27.079238") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12115, "countyName": "Sarasota", "cityID": 1273950, "cityName": "Venice Gardens" } }
+{ "create_at": datetime("2015-11-26T10:00:24.000Z"), "id": 669938772360851458, "text": "my running app just gave a notification stating, \"time to gobble up those miles\" ... well Nike I wish I could ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1015530289, "name": "IGGY", "screen_name": "ingrid_gizzelle", "lang": "en", "location": "Long Beach, CA", "create_at": date("2012-12-16"), "description": "MCU XC/T&F 18' | CORINTHIANS 9:24-9:26", "followers_count": 781, "friends_count": 1083, "statues_count": 10177 }, "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("2015-11-26T10:00:24.000Z"), "id": 669938772449083392, "text": "Happppppy birthday to my best friend on her day!!! @haleywittig 24 looks good on ya!!!!�������� https://t.co/5kzfiiBtEd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 137769329 }}, "user": { "id": 105508272, "name": "Mallory Eherenman", "screen_name": "Mal_Eherenman", "lang": "en", "location": "null", "create_at": date("2010-01-16"), "description": "Indiana State, AXOmega❤", "followers_count": 639, "friends_count": 576, "statues_count": 6908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsburg, IN", "id": "00f424d474454ea0", "name": "Brownsburg", "place_type": "city", "bounding_box": rectangle("-86.439701,39.800082 -86.326794,39.912651") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18063, "countyName": "Hendricks", "cityID": 1808416, "cityName": "Brownsburg" } }
+{ "create_at": datetime("2015-11-26T10:00:24.000Z"), "id": 669938772683784192, "text": "I also feel the need to remind everyone every Thanksgiving that my dad exercises by walking laps around our couch. https://t.co/r8eWU8yMvi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 334965755, "name": "madeline kelly", "screen_name": "madelinemkelly", "lang": "en", "location": "NYC • midwest, USA", "create_at": date("2011-07-13"), "description": "assist. art director @goodhousemag. @newhousesu grad. I like lipstick, hip hop, and hugs. my number 1 rule: no insulting kanye west.", "followers_count": 805, "friends_count": 1130, "statues_count": 3694 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clarendon Hills, IL", "id": "47fe07f1c6b38dbd", "name": "Clarendon Hills", "place_type": "city", "bounding_box": rectangle("-87.971039,41.781516 -87.94591,41.810687") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1714572, "cityName": "Clarendon Hills" } }
+{ "create_at": datetime("2015-11-26T10:00:24.000Z"), "id": 669938772708986880, "text": "That's all you need for a Good Thanksgiving! https://t.co/fhsVNsbRPK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 633463409, "name": "Nick Escobar", "screen_name": "TheShizNick44", "lang": "en", "location": "null", "create_at": date("2012-07-11"), "description": "null", "followers_count": 280, "friends_count": 315, "statues_count": 20211 }, "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-26T10:00:24.000Z"), "id": 669938772793012224, "text": "@JessieeBands_ I don't even got yo number lmao", "in_reply_to_status": 669938322949697536, "in_reply_to_user": 71444399, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 71444399 }}, "user": { "id": 239498056, "name": "Dirty Dee", "screen_name": "DessBoogie", "lang": "en", "location": "null", "create_at": date("2011-01-17"), "description": "48216", "followers_count": 2588, "friends_count": 1286, "statues_count": 130917 }, "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-26T10:00:25.000Z"), "id": 669938773019488257, "text": "You thinking yo plate �� gone be the talk of Twitter if yo shit trash ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 45424374, "name": "️", "screen_name": "1rhquvn_", "lang": "en", "location": "Zone 6", "create_at": date("2009-06-07"), "description": "#BlessKing c/o Spring '16 #JournalismMajor", "followers_count": 1007, "friends_count": 462, "statues_count": 46465 }, "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("2015-11-26T10:00:25.000Z"), "id": 669938773136928768, "text": "@Jadeex19 thanks girl��", "in_reply_to_status": 669935323770707970, "in_reply_to_user": 2206187036, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2206187036 }}, "user": { "id": 1255239974, "name": "kay", "screen_name": "kaylamosley22", "lang": "en", "location": "null", "create_at": date("2013-03-09"), "description": "null", "followers_count": 632, "friends_count": 338, "statues_count": 9153 }, "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("2015-11-26T10:00:25.000Z"), "id": 669938773308764160, "text": "my cousin just said \"he's throwing shade\" to my uncle and my grandma.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 616902609, "name": "alyah NOEL", "screen_name": "alyah_noel", "lang": "en", "location": "Moore, OK", "create_at": date("2012-06-23"), "description": "I'm in love with hedgehogs & a dog named shortie.", "followers_count": 601, "friends_count": 481, "statues_count": 25114 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atoka, OK", "id": "741241316108490d", "name": "Atoka", "place_type": "city", "bounding_box": rectangle("-96.149653,34.369508 -96.114853,34.396254") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40005, "countyName": "Atoka", "cityID": 4003300, "cityName": "Atoka" } }
+{ "create_at": datetime("2015-11-26T10:00:25.000Z"), "id": 669938773401214977, "text": "@HegerJenna happy birthday ❤️", "in_reply_to_status": -1, "in_reply_to_user": 2506952968, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2506952968 }}, "user": { "id": 2753959567, "name": "Noah", "screen_name": "BIGM3ACH", "lang": "en", "location": "null", "create_at": date("2014-08-21"), "description": "null", "followers_count": 152, "friends_count": 173, "statues_count": 700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bridgewater, MA", "id": "006b91cb41e4a9cc", "name": "Bridgewater", "place_type": "city", "bounding_box": rectangle("-71.035483,41.929851 -70.898031,42.010968") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25023, "countyName": "Plymouth", "cityID": 2508050, "cityName": "Bridgewater" } }
+{ "create_at": datetime("2015-11-26T10:00:25.000Z"), "id": 669938773489152000, "text": "Watching Sproles turn up on this Thanksgiving Day!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 185837474, "name": "Godiva", "screen_name": "Miss_IshaD", "lang": "en", "location": "Memphis, Tennessee", "create_at": date("2010-09-01"), "description": "Germany born Manhattan, Kansas raised and I Love #Kstate...Graduated a Memphis Tiger...Waiting on my moment! #veteran #army", "followers_count": 527, "friends_count": 748, "statues_count": 21597 }, "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("2015-11-26T10:00:25.000Z"), "id": 669938773573165058, "text": "How many backhanded comments can I take today before I get up and leave", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 721262401, "name": "Madelyn Prevenslik", "screen_name": "mxdelynprevn", "lang": "en", "location": "mt. pleasant", "create_at": date("2012-07-27"), "description": "♊️", "followers_count": 1059, "friends_count": 523, "statues_count": 19024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bessemer, PA", "id": "00b2b56b3644b2ca", "name": "Bessemer", "place_type": "city", "bounding_box": rectangle("-79.559108,40.123672 -79.547379,40.147893") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42073, "countyName": "Lawrence", "cityID": 4205936, "cityName": "Bessemer" } }
+{ "create_at": datetime("2015-11-26T10:00:25.000Z"), "id": 669938773866639360, "text": "Can you recommend anyone for this #Nursing #job? https://t.co/IycnKgGXJ8 #STNA #training #Beachwood, OH #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4899224,41.5011199"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "STNA", "training", "Beachwood", "Hiring" }}, "user": { "id": 214122801, "name": "MenorahParkJobs", "screen_name": "MenorahParkJobs", "lang": "en", "location": "Beachwood, Ohio", "create_at": date("2010-11-10"), "description": "Menorah Park Center for Senior Living is one of the largest senior living communities in the state of Ohio.", "followers_count": 126, "friends_count": 82, "statues_count": 1377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beachwood, OH", "id": "fa5bd676191716ee", "name": "Beachwood", "place_type": "city", "bounding_box": rectangle("-81.532635,41.449669 -81.48037,41.501448") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3904500, "cityName": "Beachwood" } }
+{ "create_at": datetime("2015-11-26T10:00:25.000Z"), "id": 669938774080491520, "text": "Thunderstorms this afternoon, high 73 (23 C). Low 68 (20 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119754993, "name": "Waco Weather", "screen_name": "WacoWeather", "lang": "en", "location": "Waco, TX", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Waco, TX", "followers_count": 809, "friends_count": 68, "statues_count": 6237 }, "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.069786,31.639828") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4876000, "cityName": "Waco" } }
+{ "create_at": datetime("2015-11-26T10:00:25.000Z"), "id": 669938774101504001, "text": "fr https://t.co/Gk0zk9XHSK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2882564096, "name": "☃", "screen_name": "IxcalWendy", "lang": "en", "location": "null", "create_at": date("2014-10-29"), "description": "how do you move on from someone that's on your mind 24/7", "followers_count": 140, "friends_count": 187, "statues_count": 433 }, "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("2015-11-26T10:00:25.000Z"), "id": 669938774189600769, "text": "Madison is the only person who has met all of 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": 343825409, "name": "breesta", "screen_name": "_BreeannMarie__", "lang": "en", "location": "null", "create_at": date("2011-07-27"), "description": "im as real as the streets", "followers_count": 769, "friends_count": 376, "statues_count": 24532 }, "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("2015-11-26T10:00:25.000Z"), "id": 669938774344880128, "text": "Too bad about the chestnuts https://t.co/3KIgMCsx5F", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 863447222, "name": "Sam Wheeler", "screen_name": "ghostbat68", "lang": "en", "location": "null", "create_at": date("2012-10-05"), "description": "2015 NCSTA President, Presidential Award (2005); Albert Einstein Fellow (DOE); NASA NEAT; Physics Instructor; Doctoral Candidate", "followers_count": 177, "friends_count": 351, "statues_count": 1842 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nags Head, NC", "id": "00a9dd516d11c0b7", "name": "Nags Head", "place_type": "city", "bounding_box": rectangle("-75.670276,35.84598 -75.562843,35.995022") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37055, "countyName": "Dare", "cityID": 3745880, "cityName": "Nags Head" } }
+{ "create_at": datetime("2015-11-26T10:00:25.000Z"), "id": 669938774403637248, "text": "Thankful for these and so many other people in my life☺️�� https://t.co/eTWwhnxIrq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 497188470, "name": "madison", "screen_name": "Mkelkis", "lang": "en", "location": "null", "create_at": date("2012-02-19"), "description": "I alone cannot change the world, but I can cast a stone across the waters to create many ripples.", "followers_count": 324, "friends_count": 321, "statues_count": 1287 }, "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("2015-11-26T10:00:25.000Z"), "id": 669938774416203777, "text": "@AnimalSavages @gabbylagunas", "in_reply_to_status": 669366784596705280, "in_reply_to_user": 3148819020, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3148819020, 426126829 }}, "user": { "id": 394361998, "name": "abbey", "screen_name": "SixPackAbss1", "lang": "en", "location": "null", "create_at": date("2011-10-19"), "description": "MERMAIDS ARE REAL", "followers_count": 570, "friends_count": 320, "statues_count": 8402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dry Run, OH", "id": "e8ca10ba56edd187", "name": "Dry Run", "place_type": "city", "bounding_box": rectangle("-84.368089,39.086598 -84.307127,39.12355") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3922674, "cityName": "Dry Run" } }
+{ "create_at": datetime("2015-11-26T10:00:25.000Z"), "id": 669938774806298625, "text": "I can't tell if I was born naturally intuitive, or if psychedelics made me that way.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 346444517, "name": "psychethesia", "screen_name": "MattDuncan666", "lang": "en", "location": "Malkuth", "create_at": date("2011-08-01"), "description": "null", "followers_count": 48, "friends_count": 43, "statues_count": 227 }, "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("2015-11-26T10:00:25.000Z"), "id": 669938775032655872, "text": "Happy Thanksgiving!!!!☺☺☺☺", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2288759330, "name": "$GOLDIE$", "screen_name": "Sunnnnyrae", "lang": "en", "location": "null", "create_at": date("2014-01-12"), "description": "Proud Mommy of Savannah Rae❤", "followers_count": 352, "friends_count": 396, "statues_count": 2569 }, "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("2015-11-26T10:00:25.000Z"), "id": 669938775124934656, "text": "Philly gonna fuck my card up... I 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": 2421518437, "name": "NBA Champs (+250)", "screen_name": "chrispiercreme", "lang": "en", "location": "East Oakland, CA", "create_at": date("2014-03-31"), "description": "cisthirsty", "followers_count": 301, "friends_count": 224, "statues_count": 58794 }, "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("2015-11-26T10:00:25.000Z"), "id": 669938775426994176, "text": "Just three of the many people I'm thankful for today – and always. Wishing all a very Happy Thanksgiving. https://t.co/PicYBwhOpl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 60937618, "name": "Walter Deleon", "screen_name": "WalterDeleonDC", "lang": "en", "location": "Washington, D.C.", "create_at": date("2009-07-28"), "description": "Youngest Latino elected official in DC history. Campaign Manager for @Oberting2016. ANC 5C02 | #OMalley2016 | CUA '18. Tweets are my own opinions.", "followers_count": 720, "friends_count": 187, "statues_count": 6249 }, "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-26T10:00:25.000Z"), "id": 669938775775010817, "text": "@Go_On_Volt @MarcPls @MrGoldenSports @VictionaryHD @MyUnicornIsHigh @MUTCrew @SteelCurtainMUT", "in_reply_to_status": 669938661325082624, "in_reply_to_user": 2878454108, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2353933176, 2976661617, 527605752, 1142656364, 142509856, 2555966789, 361429822 }}, "user": { "id": 2878454108, "name": "golf pro", "screen_name": "AustinDupriest", "lang": "en", "location": "null", "create_at": date("2014-10-26"), "description": "xbox one mut 15 nba, bf, cod, will middle man if needed im as legit as it gets.... I like tacos too. Im 21 years old business major work at shadowride CC", "followers_count": 192, "friends_count": 94, "statues_count": 1177 }, "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("2015-11-26T10:00:25.000Z"), "id": 669938776173604868, "text": "@shlubbbly Update: she's beyond excited for the trip but I think she was happier to have all her kids home ☺️", "in_reply_to_status": 669663741269688320, "in_reply_to_user": 3255180304, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3255180304 }}, "user": { "id": 3255180304, "name": "Shelby Platia", "screen_name": "shlubbbly", "lang": "en", "location": "null", "create_at": date("2015-05-14"), "description": "null", "followers_count": 83, "friends_count": 88, "statues_count": 490 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin Square, NY", "id": "4bfbb13bd35cf5e0", "name": "Franklin Square", "place_type": "city", "bounding_box": rectangle("-73.692411,40.682829 -73.661034,40.717994") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3627309, "cityName": "Franklin Square" } }
+{ "create_at": datetime("2015-11-26T10:00:25.000Z"), "id": 669938776265768960, "text": "https://t.co/DKSXvIORLe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3184374859, "name": "Perry Fedorow", "screen_name": "FedorowPerry", "lang": "en", "location": "null", "create_at": date("2015-05-03"), "description": "Men Anything goes Photography. I love muscular men and good kisser's. The AV picture is me. Adult content be over 18", "followers_count": 2805, "friends_count": 3478, "statues_count": 2336 }, "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-26T10:00:25.000Z"), "id": 669938776265789441, "text": "Can you recommend anyone for this #Healthcare #job? https://t.co/c5bEVeOi2d #MOORESVILLE, NC #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.8100724,35.5848596"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "MOORESVILLE", "Hiring", "CareerArc" }}, "user": { "id": 20824752, "name": "Charlotte Health Job", "screen_name": "tmj_clt_health", "lang": "en", "location": "Charlotte, NC", "create_at": date("2009-02-13"), "description": "Follow this account for geo-targeted Healthcare job tweets in Charlotte, NC. Need help? Tweet us at @CareerArc!", "followers_count": 612, "friends_count": 307, "statues_count": 233 }, "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("2015-11-26T10:00:25.000Z"), "id": 669938776299450368, "text": "This is what happens when you steal an entire slice of turkey off of the table. Bail of $500 https://t.co/hHuYUkZY3x", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2737062760, "name": "Soph", "screen_name": "sophie_dinehart", "lang": "en", "location": "null", "create_at": date("2014-08-08"), "description": "Sophia the 1st", "followers_count": 202, "friends_count": 213, "statues_count": 751 }, "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("2015-11-26T10:00:25.000Z"), "id": 669938776387358720, "text": "6. Jarred Tinordi\n7. #PanXmasWish\n8. Cecilia Walters\n9. Competition Bureau\n10. Derek Saretzky\n\n2015/11/26 11:51 CST https://t.co/SHjd3tkINE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.309,56.9547"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PanXmasWish" }}, "user": { "id": 1270239397, "name": "Trendinalia Canada", "screen_name": "trendinaliaCA", "lang": "es", "location": "Canada", "create_at": date("2013-03-15"), "description": "Information about Twitter's Trending Topics in Canada", "followers_count": 1151, "friends_count": 7, "statues_count": 239336 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Division No. 23, Unorganized, Manitoba", "id": "654fd5617c7361d0", "name": "Division No. 23, Unorganized", "place_type": "city", "bounding_box": rectangle("-102.009382,55.158501 -88.989166,60.000283") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2015-11-26T10:00:25.000Z"), "id": 669938776433553408, "text": "@maaariah_xo you've been here through thick and thin, thank you for putting up with my mean ass, love you so much, happy thanksgiving ����", "in_reply_to_status": -1, "in_reply_to_user": 3302891214, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3302891214 }}, "user": { "id": 2679167875, "name": "Jess", "screen_name": "jessicakrebss", "lang": "en", "location": "maricopa az", "create_at": date("2014-07-25"), "description": "mountains pointe", "followers_count": 157, "friends_count": 66, "statues_count": 334 }, "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("2015-11-26T10:00:25.000Z"), "id": 669938776483868672, "text": "@desibruhh thanks shawty, come visit ��", "in_reply_to_status": 669938591783497728, "in_reply_to_user": 1328002333, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1328002333 }}, "user": { "id": 413529264, "name": "Dab Daddy", "screen_name": "OmarAlcozai1", "lang": "en", "location": "YB's house ", "create_at": date("2011-11-15"), "description": "part-time side nigga", "followers_count": 1327, "friends_count": 933, "statues_count": 46024 }, "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("2015-11-26T10:00:25.000Z"), "id": 669938776588681217, "text": "I'm 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": 1025243185, "name": "ॐ", "screen_name": "_CristianAmadeo", "lang": "en", "location": "#peacemob♡ॐ☮", "create_at": date("2012-12-20"), "description": "My only goal is to be happy ;)", "followers_count": 1122, "friends_count": 897, "statues_count": 24097 }, "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-26T10:00:25.000Z"), "id": 669938776773279744, "text": "@ABCSharkTank Enjoy your family and friends.", "in_reply_to_status": 669915687821950976, "in_reply_to_user": 468583731, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 468583731 }}, "user": { "id": 4034907197, "name": "George Keith", "screen_name": "smoketree345", "lang": "en", "location": "Illinois, USA", "create_at": date("2015-10-24"), "description": "null", "followers_count": 8, "friends_count": 18, "statues_count": 19 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bartlett, IL", "id": "7d7d57e9679ec16b", "name": "Bartlett", "place_type": "city", "bounding_box": rectangle("-88.263058,41.935691 -88.148859,42.016598") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1704013, "cityName": "Bartlett" } }
+{ "create_at": datetime("2015-11-26T10:00:25.000Z"), "id": 669938776802627584, "text": "Haha Happy Thanksgiving https://t.co/Ar3XCq9lyz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1619023974, "name": "Joel H-Train Hunt", "screen_name": "thehuntforgold", "lang": "en", "location": "Denver,Co/ Hometown Kokomo,In", "create_at": date("2013-07-24"), "description": "#Beer30 host an Dj on http://www.MBRadio.us Paralympian 2014 #military Retired #Purple Heart Recipient", "followers_count": 343, "friends_count": 185, "statues_count": 3535 }, "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-26T10:00:25.000Z"), "id": 669938776802762752, "text": "THIS IS MY REAL LIVE ST. JOSEPH'S MAGIC RIGHT HERE.\n\"YOU DO NOT EVEN THIS HOUSE RIGHT HERE SILVIO SILVESTRES!\"\n\ncont.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15329029, "name": "Joseph Alizio Jr.", "screen_name": "joeyalizio1", "lang": "en", "location": "New Hampshire, USA", "create_at": date("2008-07-05"), "description": "47 Published Books. My Income Is Your Wifey Hoe Momma's Paycheck.", "followers_count": 212, "friends_count": 922, "statues_count": 5843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dover, NH", "id": "41fdfd56e9c5e4d3", "name": "Dover", "place_type": "city", "bounding_box": rectangle("-70.96159,43.119192 -70.820896,43.255004") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33017, "countyName": "Strafford", "cityID": 3318820, "cityName": "Dover" } }
+{ "create_at": datetime("2015-11-26T10:00:25.000Z"), "id": 669938776806944769, "text": "exactly https://t.co/FI6Wf5Jr6p", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1855569032, "name": "Σwaggy P", "screen_name": "RemyForShort", "lang": "en", "location": "null", "create_at": date("2013-09-11"), "description": "Intermediate Finesser☆If Ioun Do Nothing; Dammit, I'm Going To Ball!☆ΦΒΣ", "followers_count": 470, "friends_count": 524, "statues_count": 16625 }, "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("2015-11-26T10:00:25.000Z"), "id": 669938776903430145, "text": "Partly cloudy this afternoon, high 67 (19 C). Low 52 (11 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119755728, "name": "Louisville Weather", "screen_name": "Louisville_WX", "lang": "en", "location": "Louisville, KY", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Louisville, KY", "followers_count": 1301, "friends_count": 68, "statues_count": 6264 }, "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-26T10:00:25.000Z"), "id": 669938776953585664, "text": "Happy thanksgiving Twitter fam .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 57871404, "name": "HomeAlone", "screen_name": "thebook_oflevi", "lang": "en", "location": "richmond, Los angeles, sac,CA ", "create_at": date("2009-07-17"), "description": "explaing is like giving a fuck about what others think ... Remember you are self made.", "followers_count": 995, "friends_count": 951, "statues_count": 30174 }, "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-26T10:00:26.000Z"), "id": 669938777222090752, "text": "@chinitashinee @audjgunth @ChristianCrain @kellydee91 @SinnAziza @samanthajoxoxo happy thanking my out of state followers��������", "in_reply_to_status": -1, "in_reply_to_user": 887434904, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 887434904, 25443959, 395658737, 37436891, 2187284372, 26325606 }}, "user": { "id": 387527589, "name": "Michelle A", "screen_name": "ayoaznbeezie", "lang": "en", "location": "Bay Area, CA", "create_at": date("2011-10-08"), "description": "special girl, real good girl, biggest thing in your itty bitty world.", "followers_count": 489, "friends_count": 390, "statues_count": 21405 }, "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("2015-11-26T10:00:26.000Z"), "id": 669938777347989504, "text": "@SethroOnTheTV Brad Pitts Thanksgiving eppy is everything.", "in_reply_to_status": 669937437209174017, "in_reply_to_user": 617155622, "favorite_count": 0, "coordinate": point("-77.72760348,39.63963964"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 617155622 }}, "user": { "id": 830261755, "name": "Mary", "screen_name": "mollymargaret_", "lang": "en", "location": "null", "create_at": date("2012-09-17"), "description": "Love soaps and rock and roll! Oh yeah,and boy bands. Judge me!!!", "followers_count": 725, "friends_count": 1297, "statues_count": 38455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hagerstown, MD", "id": "fb4e19c906cdaa73", "name": "Hagerstown", "place_type": "city", "bounding_box": rectangle("-77.786448,39.592442 -77.671529,39.676503") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24043, "countyName": "Washington", "cityID": 2436075, "cityName": "Hagerstown" } }
+{ "create_at": datetime("2015-11-26T10:00:26.000Z"), "id": 669938777348001792, "text": "@Adotxx @ImperfGod I spent thanksgiving alone once and that shit hurt me so just trying to look after other people ��", "in_reply_to_status": 669938422010798080, "in_reply_to_user": 393117302, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 393117302, 2467195496 }}, "user": { "id": 101309767, "name": "daze", "screen_name": "DaisyCataleya", "lang": "en", "location": "Chicago, IL", "create_at": date("2010-01-02"), "description": "i'm not your type", "followers_count": 1534, "friends_count": 677, "statues_count": 59751 }, "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-26T10:00:26.000Z"), "id": 669938777394163712, "text": "Shot by: @uab28 x #Only4G$\nMakeup: @_legaltender ��\n#O4G$ @ Atlanta, Georgia https://t.co/piOLi8DPEI", "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": {{ "Only4G", "O4G" }}, "user_mentions": {{ 2216719968 }}, "user": { "id": 50809156, "name": "O4G$$", "screen_name": "JassElisa", "lang": "en", "location": "ATL, GA", "create_at": date("2009-06-25"), "description": "#ONLY4G$", "followers_count": 4947, "friends_count": 919, "statues_count": 103003 }, "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("2015-11-26T10:00:26.000Z"), "id": 669938777750540288, "text": "I really shouldn't even feel this bad anymore but 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": 289647683, "name": "Madison", "screen_name": "smweatherman", "lang": "en", "location": "null", "create_at": date("2011-04-28"), "description": "nyb", "followers_count": 393, "friends_count": 266, "statues_count": 18188 }, "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("2015-11-26T10:00:26.000Z"), "id": 669938777851195393, "text": "@allysalynnxoxo thank you bae ������", "in_reply_to_status": 669924038962118656, "in_reply_to_user": 3285579457, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3285579457 }}, "user": { "id": 253875823, "name": "Maddy Johll", "screen_name": "maddymariejohll", "lang": "en", "location": "null", "create_at": date("2011-02-17"), "description": "Jaxen Alexander Clayton 5/23/14❤️ Snapchat: maddyjohll", "followers_count": 351, "friends_count": 959, "statues_count": 7451 }, "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("2015-11-26T10:00:26.000Z"), "id": 669938778044104704, "text": "Skipping breakfast is a must on Thanksgiving", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 593950775, "name": "The Spicy ", "screen_name": "TheSpicy_Pepper", "lang": "en", "location": "Galveston", "create_at": date("2012-05-29"), "description": "Bachelor of Science Maritime Administration", "followers_count": 390, "friends_count": 341, "statues_count": 9628 }, "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("2015-11-26T10:00:26.000Z"), "id": 669938778048299008, "text": "Can't believe I'm actually working today ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3081781501, "name": "Miranda Lienau", "screen_name": "mir_lienau", "lang": "en", "location": "Warren, MI", "create_at": date("2015-03-14"), "description": "18|♌", "followers_count": 536, "friends_count": 445, "statues_count": 4358 }, "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("2015-11-26T10:00:26.000Z"), "id": 669938778077863937, "text": "Their presence is present enough �� #thankful #clique @ Downtown Fort Lauderdale https://t.co/e3nHJVzyzD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.24343181,26.09559987"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thankful", "clique" }}, "user": { "id": 40861237, "name": "Laurs", "screen_name": "laurensab", "lang": "en", "location": "South Florida", "create_at": date("2009-05-18"), "description": "Whatever Forever", "followers_count": 361, "friends_count": 412, "statues_count": 10049 }, "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("2015-11-26T10:00:26.000Z"), "id": 669938778144956417, "text": "That was a great throw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 42025851, "name": "TYUS TRUMBETTI", "screen_name": "ttrum4", "lang": "en", "location": "Burlington, n e w jers", "create_at": date("2009-05-23"), "description": "Caitlin Carey 1.26.13❤️❤️", "followers_count": 909, "friends_count": 557, "statues_count": 33452 }, "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("2015-11-26T10:00:26.000Z"), "id": 669938778362937345, "text": "Dropsigrew", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2272365596, "name": "Keith Shelton", "screen_name": "DPsportseditor", "lang": "en", "location": "Escanaba, MI", "create_at": date("2014-01-01"), "description": "Daily Press Sports Editor, music lover, #libertarian, husband/father, love my life, I have the best job in the world.#DetroitvsEverybody", "followers_count": 657, "friends_count": 696, "statues_count": 11423 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gladstone, MI", "id": "a7f9a85adeee95a7", "name": "Gladstone", "place_type": "city", "bounding_box": rectangle("-87.076642,45.810887 -86.999675,45.87083") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26041, "countyName": "Delta", "cityID": 2632300, "cityName": "Gladstone" } }
+{ "create_at": datetime("2015-11-26T10:00:26.000Z"), "id": 669938778409037824, "text": "All that's left to cook is jambalaya, plantain & gizzard...then turkey should be done in like 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": 37267838, "name": "ℓσvε, Stella Dee", "screen_name": "darkskinbeaut3", "lang": "en", "location": "Washington, DC", "create_at": date("2009-05-02"), "description": "God First. Nigerian. Future PA. HU16", "followers_count": 1197, "friends_count": 992, "statues_count": 63370 }, "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-26T10:00:26.000Z"), "id": 669938778467778560, "text": "@Whois_Tootie where you trying to go", "in_reply_to_status": 669938695563116545, "in_reply_to_user": 132682584, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 132682584 }}, "user": { "id": 296010334, "name": "UNDRAFTED", "screen_name": "_onyourgirlmind", "lang": "en", "location": "null", "create_at": date("2011-05-09"), "description": "SHSU TRACK AND FIELD #FAB5", "followers_count": 1216, "friends_count": 1399, "statues_count": 11669 }, "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("2015-11-26T10:00:26.000Z"), "id": 669938778681683968, "text": "So can we just have like a month off, thanks", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 420082947, "name": "Hay", "screen_name": "HnmxoMarin", "lang": "en", "location": "CA", "create_at": date("2011-11-23"), "description": "um, i like it tha best when ur nice to me", "followers_count": 342, "friends_count": 267, "statues_count": 4758 }, "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("2015-11-26T10:00:26.000Z"), "id": 669938778740428801, "text": "cute boys make me ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 595284890, "name": "nichole :-)", "screen_name": "nikkimcwebb", "lang": "en", "location": "Castle Rock, CO", "create_at": date("2012-05-31"), "description": "i have curly hair and nice eyebrows", "followers_count": 299, "friends_count": 676, "statues_count": 5441 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Castle Rock, CO", "id": "0fd0097acd635907", "name": "Castle Rock", "place_type": "city", "bounding_box": rectangle("-104.92104,39.322269 -104.773048,39.451319") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8035, "countyName": "Douglas", "cityID": 812415, "cityName": "Castle Rock" } }
+{ "create_at": datetime("2015-11-26T10:00:26.000Z"), "id": 669938778883100672, "text": "put salt in your shampoo. RT @Boss__T: It be so hard to get wax out my head EVERY TIME������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 279577153 }}, "user": { "id": 225960428, "name": "Boss.", "screen_name": "TaashNicole", "lang": "en", "location": "continuously moving .", "create_at": date("2010-12-12"), "description": "- don't follow me, i don't follow back.", "followers_count": 892, "friends_count": 222, "statues_count": 153115 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Romulus, MI", "id": "781e991b9f95b37f", "name": "Romulus", "place_type": "city", "bounding_box": rectangle("-83.446302,42.179271 -83.306006,42.268212") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2669420, "cityName": "Romulus" } }
+{ "create_at": datetime("2015-11-26T10:00:26.000Z"), "id": 669938778983800832, "text": "Smoked turkey. #Thanksgiving https://t.co/2I1zd5RO3c", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Thanksgiving" }}, "user": { "id": 19417532, "name": "James Mitchem", "screen_name": "jmitchem", "lang": "en", "location": "Charlotte", "create_at": date("2009-01-23"), "description": "writer. father. husband. @usairforce veteran. INFJ | work https://t.co/UeO31KiHlh | play https://t.co/V3nSzUyGVD | @MinorKingNovel @GoneDogs | raised by wolves", "followers_count": 2604, "friends_count": 661, "statues_count": 34689 }, "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("2015-11-26T10:00:26.000Z"), "id": 669938779201736704, "text": "Retweet Happy Thanksgiving @ashanti @DaOnlySalt @gayleking @oprah @claudiajordan #DrinkUpAshanti @rosariodawson @jlo https://t.co/fhd3H9g7ei", "in_reply_to_status": 669934977929187329, "in_reply_to_user": 2155986174, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DrinkUpAshanti" }}, "user_mentions": {{ 67982898, 65705589, 29546945, 19397785, 24332709, 82939583, 85603854 }}, "user": { "id": 2155986174, "name": "TrendingTweetsNow", "screen_name": "EricScCribb", "lang": "en", "location": "The Battery Charleston ,SC ", "create_at": date("2013-10-25"), "description": "U hit the end of the road and must follow to turn back , Will #Teach u @Twitter one #Tweet at a time€ The Battery is a landmark defensive seawall and promenade", "followers_count": 18964, "friends_count": 20858, "statues_count": 28444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charleston, SC", "id": "7422f29331ca8257", "name": "Charleston", "place_type": "city", "bounding_box": rectangle("-80.134867,32.673231 -79.879867,32.866916") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45019, "countyName": "Charleston", "cityID": 4513330, "cityName": "Charleston" } }
+{ "create_at": datetime("2015-11-26T10:00:26.000Z"), "id": 669938779558293504, "text": "I'm like the only one in my fam that didn't bring a date HAHAHHAH (-,:", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 867300240, "name": "Robyn", "screen_name": "LexieDBarnes_", "lang": "en", "location": "Probs at Bdubbs", "create_at": date("2012-10-07"), "description": "Snapchat: Lexxdawn22✖️Insta: lexiedbarnes_", "followers_count": 415, "friends_count": 231, "statues_count": 7427 }, "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("2015-11-26T10:00:26.000Z"), "id": 669938779566665728, "text": "I'm gone lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 590319928, "name": "✨Amber Marie♓️", "screen_name": "Loveambermariex", "lang": "en", "location": "Clutch City, Tx", "create_at": date("2012-05-25"), "description": "Do you ever wonder if the stars shine out for you?", "followers_count": 572, "friends_count": 483, "statues_count": 9304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:26.000Z"), "id": 669938779734560769, "text": "Well now that the parade is over I should probably get up and get ready", "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": "Broken Arrow, OK", "create_at": date("2012-08-11"), "description": "NYC is the dream. sing/dance/act. ✞ = ♡. Tiger Mystique Show Choir❤️ Broken Arrow, OK. Okla{HOME}a. instagram: abbylaninixoxo", "followers_count": 1324, "friends_count": 1996, "statues_count": 25049 }, "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("2015-11-26T10:00:26.000Z"), "id": 669938779902353408, "text": "Nah man this ain't true is it ���� https://t.co/FpCZWZNNS5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 293546271, "name": "Kristaps", "screen_name": "KrisMena21", "lang": "en", "location": "in my skin", "create_at": date("2011-05-05"), "description": "Ugly and Humble.", "followers_count": 632, "friends_count": 177, "statues_count": 80804 }, "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-26T10:00:26.000Z"), "id": 669938779914833925, "text": "YOU THOUGHT I WAS DONE!!!!! @Angeliiica_Vee @DearSabrina_ I love you guys ❤️❤️❤️", "in_reply_to_status": 669938386270982144, "in_reply_to_user": 339455065, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 718751582, 230272117 }}, "user": { "id": 339455065, "name": "mo ☾", "screen_name": "moeswann", "lang": "en", "location": "CA", "create_at": date("2011-07-20"), "description": "holy inferiority complex, Batman.", "followers_count": 429, "friends_count": 307, "statues_count": 32636 }, "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("2015-11-26T10:00:26.000Z"), "id": 669938780023865344, "text": "@Sarahluv1111 thanksgiving baskets! They have things for thanksgiving for people who can't buy it! People donate and we put it together��", "in_reply_to_status": 669916986453991427, "in_reply_to_user": 383516514, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 383516514 }}, "user": { "id": 2770525703, "name": "Safia", "screen_name": "safiaelsadr", "lang": "en", "location": "null", "create_at": date("2014-09-14"), "description": "Add a bio to your profile", "followers_count": 201, "friends_count": 133, "statues_count": 284 }, "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("2015-11-26T10:00:26.000Z"), "id": 669938780304900097, "text": "gonna go shooting ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2629267407, "name": "NoFvksGiven ✨", "screen_name": "simplyyapril", "lang": "en", "location": "| 8O1 |", "create_at": date("2014-06-21"), "description": "I'm so blunt you could roll my truth.", "followers_count": 596, "friends_count": 921, "statues_count": 2915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buckeye, AZ", "id": "0015cc0d71d49e19", "name": "Buckeye", "place_type": "city", "bounding_box": rectangle("-112.625877,33.355798 -112.461428,33.515442") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 407940, "cityName": "Buckeye" } }
+{ "create_at": datetime("2015-11-26T10:00:26.000Z"), "id": 669938780346802176, "text": "@theellarocco @slvttyspice HAHAHAHAHAHAHHAHA", "in_reply_to_status": 669922782474473472, "in_reply_to_user": 1661490948, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 1661490948, 167214012 }}, "user": { "id": 2894117088, "name": "Sofia Stefanovich", "screen_name": "sofia_clara15", "lang": "en", "location": "null", "create_at": date("2014-11-08"), "description": ":):", "followers_count": 133, "friends_count": 269, "statues_count": 707 }, "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("2015-11-26T10:00:26.000Z"), "id": 669938780363714560, "text": "A dime and that boy dropped it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 743162358, "name": "Mr. Brown", "screen_name": "KzoosReckoning", "lang": "en", "location": "null", "create_at": date("2012-08-07"), "description": "God doesn't let us go through anything we cant handle", "followers_count": 501, "friends_count": 397, "statues_count": 20355 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westwood, MI", "id": "5ef83697018ccdd9", "name": "Westwood", "place_type": "city", "bounding_box": rectangle("-85.649338,42.288789 -85.608758,42.336593") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26077, "countyName": "Kalamazoo", "cityID": 2686380, "cityName": "Westwood" } }
+{ "create_at": datetime("2015-11-26T10:00:26.000Z"), "id": 669938780728504320, "text": "@niiicoleashley tbh gonna download it just for that", "in_reply_to_status": 669937389540896769, "in_reply_to_user": 1544669298, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1544669298 }}, "user": { "id": 1276661418, "name": "Kare-ee not Kar-eee", "screen_name": "kari_holly", "lang": "en", "location": "null", "create_at": date("2013-03-17"), "description": "Missing my home in the World Famous Jungle Cruise. University of Washington ΑΟΠ. Mostly tweet about Disney and Seattle sports.", "followers_count": 234, "friends_count": 340, "statues_count": 6725 }, "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-26T10:00:26.000Z"), "id": 669938781001089024, "text": "So many things and people to be thankful for. Happy Thanksgiving everyone ❤️️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 451635145, "name": "Fetty Waf", "screen_name": "wafademashkiah", "lang": "en", "location": "Club Moody, Texas ", "create_at": date("2011-12-31"), "description": "Enjoying life one Caramel Macciato at a time •BU'19• She is clothed in strength and dignity and she laughs without fear of the future Proverbs 31:25", "followers_count": 337, "friends_count": 462, "statues_count": 5084 }, "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("2015-11-26T10:00:26.000Z"), "id": 669938781236166656, "text": "H��PPY TH��NKSGIVING!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2258029339, "name": "Christopher", "screen_name": "ChrisBair_", "lang": "en", "location": " Flower Mound", "create_at": date("2013-12-22"), "description": "Just know I will succeed. •Salt Life• FMHS'16 • Baptist", "followers_count": 121, "friends_count": 115, "statues_count": 2530 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Collierville, TN", "id": "64ab889e24887e12", "name": "Collierville", "place_type": "city", "bounding_box": rectangle("-89.744463,35.006217 -89.640889,35.110826") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4716420, "cityName": "Collierville" } }
+{ "create_at": datetime("2015-11-26T10:00:27.000Z"), "id": 669938781387145216, "text": "I've been good friends with madison for about a semester now and she still spells my name COMPLETELY wrong", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 368485640, "name": "Michaela", "screen_name": "Chae4015", "lang": "en", "location": "null", "create_at": date("2011-09-05"), "description": "UA '19 -Emily Lesco is life- ΑΔΠ- J♡rdin 8-31-14", "followers_count": 787, "friends_count": 579, "statues_count": 27918 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wickliffe, OH", "id": "37160a0bedf1fb77", "name": "Wickliffe", "place_type": "city", "bounding_box": rectangle("-81.488988,41.588179 -81.444261,41.624057") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39085, "countyName": "Lake", "cityID": 3985036, "cityName": "Wickliffe" } }
+{ "create_at": datetime("2015-11-26T10:00:27.000Z"), "id": 669938781722509312, "text": "Call me a cancer, keep convinced that you're not sick yourself.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160626356, "name": "Lucas", "screen_name": "680South_", "lang": "en", "location": "MI", "create_at": date("2010-06-28"), "description": "null", "followers_count": 554, "friends_count": 511, "statues_count": 68525 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938781781368833, "text": "Thankful for my mom and dad. I realize that im blessed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 870844909, "name": "Samyezzey™", "screen_name": "samanthaavaldez", "lang": "en", "location": "null", "create_at": date("2012-10-09"), "description": "I have Abs..olutely no friends .", "followers_count": 275, "friends_count": 128, "statues_count": 3839 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938781802389505, "text": "Will eat ass for some turkey 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": 243007875, "name": "Jesus Shuttlesworth▲", "screen_name": "SickStory_", "lang": "en", "location": "Dot BostoN", "create_at": date("2011-01-25"), "description": "(Magic, a whole lotta magic)-Professional Idiot-Nice for no reason-Send me nudes-Future Pornstar-Former AGLGod #RIPSJP http://twitch.tv/sickstoryyBz", "followers_count": 1831, "friends_count": 419, "statues_count": 90428 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Abington, MA", "id": "275b91b447f4934a", "name": "Abington", "place_type": "city", "bounding_box": rectangle("-71.002327,42.087622 -70.916474,42.149383") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25023, "countyName": "Plymouth", "cityID": 2500135, "cityName": "Abington" } }
+{ "create_at": datetime("2015-11-26T10:00:27.000Z"), "id": 669938781911412736, "text": "#happythankgivingday let's #celebrateit #djing @juanma00 #midtown #blackfriday #liveit #loveit #sipit… https://t.co/NbRBbsdDJV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.1937256,25.8068905"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "happythankgivingday", "celebrateit", "djing", "midtown", "blackfriday", "liveit", "loveit", "sipit" }}, "user_mentions": {{ 42677326 }}, "user": { "id": 202759904, "name": "bryan peroni", "screen_name": "bryanperoni", "lang": "es", "location": "Miami Beach Fl.", "create_at": date("2010-10-14"), "description": "“Music is my first love, it’s a passion, and it goes beyond liking. it’s about a way of living. Music is essential for my life”.", "followers_count": 2488, "friends_count": 616, "statues_count": 994 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938781961756672, "text": "@ivxxry YAAAASSSS", "in_reply_to_status": 669938232805736448, "in_reply_to_user": 594075034, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 594075034 }}, "user": { "id": 404521088, "name": "shannon", "screen_name": "THEshanmoran", "lang": "en", "location": "Wonderland", "create_at": date("2011-11-03"), "description": "too party for the art kids, too art for the party kids", "followers_count": 365, "friends_count": 300, "statues_count": 17208 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arbutus, MD", "id": "332025cb8fc55ee8", "name": "Arbutus", "place_type": "city", "bounding_box": rectangle("-76.722998,39.212873 -76.659807,39.268135") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2401975, "cityName": "Arbutus" } }
+{ "create_at": datetime("2015-11-26T10:00:27.000Z"), "id": 669938782016180224, "text": "OK YA ITS TIME TO GET READY IM SO EXCITED TO DO MY MAKEUP WOO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3695422272, "name": "the first noëlle", "screen_name": "musicaIhoe", "lang": "en", "location": "arizona ", "create_at": date("2015-09-26"), "description": "☆*:.。 intersectional feminist. vegan. musician. dog enthusiast. environmentalist. be kind to one another. 。.:*☆| enfp | she/her | ♥︎ @filmhoee", "followers_count": 1310, "friends_count": 380, "statues_count": 8183 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938782217498624, "text": "YAY THE DOG SHOW IS UP NEXT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 53060644, "name": "alexia", "screen_name": "SeeWhatLexiSeas", "lang": "en", "location": "San Antonio, Texas", "create_at": date("2009-07-02"), "description": "thick skin, elastic heart.", "followers_count": 1495, "friends_count": 998, "statues_count": 34330 }, "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-26T10:00:27.000Z"), "id": 669938782267965440, "text": "#THANKFUL https://t.co/r5eNaLfEhK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "THANKFUL" }}, "user": { "id": 1343155790, "name": "Kaelea Leinonen", "screen_name": "_kaelein", "lang": "en", "location": "null", "create_at": date("2013-04-10"), "description": "When life gets too hard to stand, kneel.", "followers_count": 494, "friends_count": 441, "statues_count": 11941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hibbing, MN", "id": "a84a01676afdaad4", "name": "Hibbing", "place_type": "city", "bounding_box": rectangle("-92.962962,47.378422 -92.893478,47.445813") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27137, "countyName": "St. Louis", "cityID": 2728790, "cityName": "Hibbing" } }
+{ "create_at": datetime("2015-11-26T10:00:27.000Z"), "id": 669938782301392896, "text": "Want to work at Clemens Food Group? We're #hiring in #Hatfield, PA! Click for details: https://t.co/ijvZHRi0a8 #Manufacturing #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.2993417,40.2798274"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Hatfield", "Manufacturing", "Job", "Jobs" }}, "user": { "id": 22930291, "name": "PA MGMT Jobs", "screen_name": "tmj_pa_mgmt", "lang": "en", "location": "Pennsylvania", "create_at": date("2009-03-05"), "description": "Follow this account for geo-targeted Business/Mgmt. job tweets in Pennsylvania Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 570, "friends_count": 254, "statues_count": 413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hatfield, PA", "id": "96aa2c5d434bc4d7", "name": "Hatfield", "place_type": "city", "bounding_box": rectangle("-75.307834,40.266881 -75.288457,40.287145") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42091, "countyName": "Montgomery", "cityID": 4233112, "cityName": "Hatfield" } }
+{ "create_at": datetime("2015-11-26T10:00:27.000Z"), "id": 669938782397964291, "text": "@RMenavich13 amen", "in_reply_to_status": 669926100504457216, "in_reply_to_user": 368129907, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 368129907 }}, "user": { "id": 2149993861, "name": "Pumpkieran Pie", "screen_name": "kieranisababe", "lang": "en", "location": "Knoxville, TN", "create_at": date("2013-10-22"), "description": "hey hi hello", "followers_count": 277, "friends_count": 194, "statues_count": 2292 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938782817222657, "text": "@LilTunechi I'm ready for no ceilings 2 right now! ��", "in_reply_to_status": -1, "in_reply_to_user": 116362700, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 116362700 }}, "user": { "id": 4093863737, "name": "Nissa", "screen_name": "____nissaa", "lang": "en", "location": "null", "create_at": date("2015-11-01"), "description": "#WMU19", "followers_count": 40, "friends_count": 40, "statues_count": 113 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938783039557632, "text": "@rayantav17 thanks rayan❤️❤️��", "in_reply_to_status": 669936954029404160, "in_reply_to_user": 2236523276, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2236523276 }}, "user": { "id": 603042804, "name": "Muttals", "screen_name": "ASlattum", "lang": "en", "location": "Thousand Oaks, CA", "create_at": date("2012-06-08"), "description": "CA #DMB. Tough Titties", "followers_count": 174, "friends_count": 325, "statues_count": 1224 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938783173775360, "text": "Not the Thanksgiving dinner I imagined, but still thankful. @ John Hopkins Children's Hospital https://t.co/Bb3UOhKse4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.59188291,39.29576829"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 417634927, "name": "Stephen Lyman", "screen_name": "KampaiUS", "lang": "en", "location": "New York, NY, USA", "create_at": date("2011-11-20"), "description": "New Yorker. Baseball fan. Movie buff. Shochu (焼酎) obsessive. Izakaya (居酒屋) lover. Editor of Kampai.US. Oh, and coffee.", "followers_count": 558, "friends_count": 572, "statues_count": 3742 }, "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-26T10:00:27.000Z"), "id": 669938783245238272, "text": "@MORTENofficial Thank you for smiling, the glow, the music and the love for everything you do. It inspires me to go that hard too. ����✌��️✨����", "in_reply_to_status": -1, "in_reply_to_user": 27822439, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27822439 }}, "user": { "id": 20810348, "name": "TiiaNYC", "screen_name": "Tiianyc", "lang": "en", "location": "New York City", "create_at": date("2009-02-13"), "description": "Little girls with dreams become women with vision -unknown", "followers_count": 681, "friends_count": 619, "statues_count": 12749 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938783324803072, "text": "��❤️ https://t.co/sMs01WlerJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 772502989, "name": "gen❥", "screen_name": "genesis_raeanne", "lang": "en", "location": "Denver, CO", "create_at": date("2012-08-21"), "description": "☾ ☼", "followers_count": 1159, "friends_count": 503, "statues_count": 435 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Northglenn, CO", "id": "1fff770504a7d5a5", "name": "Northglenn", "place_type": "city", "bounding_box": rectangle("-105.015875,39.868118 -104.944623,39.921489") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8001, "countyName": "Adams", "cityID": 854330, "cityName": "Northglenn" } }
+{ "create_at": datetime("2015-11-26T10:00:27.000Z"), "id": 669938783543042048, "text": "#thankful for this song lol https://t.co/EyV0yoN3Ny", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thankful" }}, "user": { "id": 48236934, "name": "❥ Lex", "screen_name": "Alexis_Santoro", "lang": "en", "location": "Chattanooga, TN", "create_at": date("2009-06-17"), "description": "I love life", "followers_count": 826, "friends_count": 622, "statues_count": 13154 }, "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.887803 -88.081435,41.942768") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1711332, "cityName": "Carol Stream" } }
+{ "create_at": datetime("2015-11-26T10:00:27.000Z"), "id": 669938783710633984, "text": "Part uno of who I'm thankful for https://t.co/riVVZJBJSg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2346101486, "name": "meleena", "screen_name": "meleeeena", "lang": "en", "location": "null", "create_at": date("2014-02-15"), "description": "c2019", "followers_count": 236, "friends_count": 73, "statues_count": 7714 }, "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-26T10:00:27.000Z"), "id": 669938783916154880, "text": "Black people say white people don't use seasoning, funny. White people insinuate that blacks spend our holidays in jail, not funny.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 65533811, "name": "zee.", "screen_name": "tumpe_", "lang": "en", "location": "trapadoches", "create_at": date("2009-08-13"), "description": "arab; retired model, my fit? yeah, they sent that for the free. #FaithfulBlackWomenOfTwitter SFASU IG: @tumpe__", "followers_count": 1558, "friends_count": 825, "statues_count": 77606 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938783991767040, "text": "Uhhh Wyd.... https://t.co/mZgZKlEVEs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pl", "is_retweet": false, "user": { "id": 2558012349, "name": "Mrs.Tesfaye", "screen_name": "_nonchvlvnt", "lang": "en", "location": "kissland", "create_at": date("2014-05-21"), "description": "XOTWOD", "followers_count": 772, "friends_count": 1432, "statues_count": 38805 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938784079896577, "text": "What's on tap. #beer #cider #craftbeer #craftcider #dccraftbeer #dccraftcider #whatareyoudrinking… https://t.co/Y33z7V1DAx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.98452,38.8803215"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "beer", "cider", "craftbeer", "craftcider", "dccraftbeer", "dccraftcider", "whatareyoudrinking" }}, "user": { "id": 2801007358, "name": "Pipetown Traders", "screen_name": "pipetowntraders", "lang": "en", "location": "Washington D.C.", "create_at": date("2014-10-02"), "description": "Specialty beer ,wine and cider wholesaler . Our selection will make your head spin. We're glass half empty kind of people, come see why...", "followers_count": 323, "friends_count": 724, "statues_count": 510 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938784130076672, "text": "happy thanksgiving:\nin honor of thanksgiving I discovered this excellent tactic for drawing turkeys no patent yet https://t.co/2VzGSKyXWG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467169388, "name": "bahjat", "screen_name": "BahjatMasoud", "lang": "en", "location": "west bank", "create_at": date("2012-01-17"), "description": "keep a watchful eye on the complimentary shampoos when you're with me bud", "followers_count": 93, "friends_count": 93, "statues_count": 400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ruidoso, NM", "id": "6843b30a7503ac93", "name": "Ruidoso", "place_type": "city", "bounding_box": rectangle("-105.703476,33.305314 -105.618997,33.384894") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35027, "countyName": "Lincoln", "cityID": 3565210, "cityName": "Ruidoso" } }
+{ "create_at": datetime("2015-11-26T10:00:27.000Z"), "id": 669938784155402240, "text": "Thankful Meg and Kenz are just as weird 5 years later. https://t.co/sikXqDW9i5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 723058994, "name": "kawaii mom", "screen_name": "fauxy_lady_", "lang": "en", "location": "Williamsburg, VA", "create_at": date("2012-07-28"), "description": "joocy currot", "followers_count": 221, "friends_count": 161, "statues_count": 16160 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938784235073537, "text": "Happy Thanksgiving to all my friends and family. Being together with family is the one thing I love about this holiday. ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1696453830, "name": "Jeremy Maurer", "screen_name": "_maurerpower30", "lang": "en", "location": "PA", "create_at": date("2013-08-24"), "description": "#DCC11, #PSU16, Phillipians 4:13, Basketball.", "followers_count": 594, "friends_count": 1947, "statues_count": 10858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cassandra, PA", "id": "553b071f4abb3af3", "name": "Cassandra", "place_type": "city", "bounding_box": rectangle("-78.643551,40.397952 -78.624344,40.419149") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42021, "countyName": "Cambria", "cityID": 4211616, "cityName": "Cassandra" } }
+{ "create_at": datetime("2015-11-26T10:00:27.000Z"), "id": 669938784423817216, "text": "@1future WE NEED APE SHIT", "in_reply_to_status": -1, "in_reply_to_user": 51742969, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 51742969 }}, "user": { "id": 1387364456, "name": "Jordan Norman", "screen_name": "ThatGuyNorman", "lang": "en", "location": "Tampa, FL", "create_at": date("2013-04-28"), "description": "Black & Italian // St Pete College", "followers_count": 856, "friends_count": 351, "statues_count": 41652 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938784528568320, "text": "Thanks to @kroger for dropping off some #Thanksgiving food. #thankful #HappyTurkeyDay https://t.co/mvoQrURXrn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Thanksgiving", "thankful", "HappyTurkeyDay" }}, "user_mentions": {{ 36359791 }}, "user": { "id": 1851957205, "name": "Oxford Police Dept", "screen_name": "OxfordPolice", "lang": "en", "location": "Oxford, MS", "create_at": date("2013-09-10"), "description": "Official Twitter account of the Oxford Police Department. Not monitored 24/7. Non-Emergencies call 662-232-2400.", "followers_count": 13238, "friends_count": 227, "statues_count": 3273 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938784591478784, "text": "@scotthoying scruff looks good on you", "in_reply_to_status": -1, "in_reply_to_user": 14810162, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14810162 }}, "user": { "id": 30796003, "name": "H̶E̶A̶T̶H̶", "screen_name": "HTrill24", "lang": "en", "location": "Gig Harbor, WA", "create_at": date("2009-04-12"), "description": "this lost boy got fly without peter pan //", "followers_count": 198, "friends_count": 326, "statues_count": 2163 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wollochet, WA", "id": "0095cd340c3aa52a", "name": "Wollochet", "place_type": "city", "bounding_box": rectangle("-122.608907,47.254255 -122.548141,47.318369") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5379555, "cityName": "Wollochet" } }
+{ "create_at": datetime("2015-11-26T10:00:27.000Z"), "id": 669938784709029888, "text": "I'm #Grateful to #God... It's a Struggle... #HappyThanksgiving... :)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Grateful", "God", "HappyThanksgiving" }}, "user": { "id": 27502058, "name": "DJ Ghostly", "screen_name": "DJ_Ghostly", "lang": "en", "location": "Chicago, IL 60641 USA", "create_at": date("2009-03-29"), "description": "Raver/140BPM Jungle DJ/Graffiti Artist/Hardcore Til I Die (HTID)/Peace, Love, Unity, & Respect (PLUR), I also Skate & Snowboard!", "followers_count": 1118, "friends_count": 2043, "statues_count": 5911 }, "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-26T10:00:27.000Z"), "id": 669938785325592576, "text": "Definitely making that kitkat brownie", "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": 997, "friends_count": 747, "statues_count": 215294 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938785413500928, "text": "Watch out I'm coming through! @ Asheville, North Carolina https://t.co/lMkvzGvObC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.5558,35.5799"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 968213888, "name": "Hayden Richau", "screen_name": "haydenrichau", "lang": "en", "location": "Believe in the Possibilities ", "create_at": date("2012-11-24"), "description": "Make it a powerful memory, the happiest you can remember. Allow it to fill you up... Think of the happiest thing you can. - HP", "followers_count": 10967, "friends_count": 354, "statues_count": 6871 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938785484804097, "text": "https://t.co/XlbDLx4crT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1219508407, "name": "Kyle Ryder", "screen_name": "KyRy23", "lang": "en", "location": "null", "create_at": date("2013-02-25"), "description": "null", "followers_count": 520, "friends_count": 435, "statues_count": 8558 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938785484996608, "text": "Thank God for my past, my present, and my future! So much love to all important people in my life. Love you @kristinl24", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 121273415 }}, "user": { "id": 963467215, "name": "Mr.FreshPrince", "screen_name": "242_ChadB10", "lang": "en", "location": "null", "create_at": date("2012-11-21"), "description": "I Love God. I Love Life! It's a Beautiful Gift that shouldn't be wasted. Thus strive to thrive.", "followers_count": 421, "friends_count": 401, "statues_count": 4749 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938785526804481, "text": "@WWE @TitusONeilWWE @VinceMcMahon Get this guy a 1st class seat! Saw him in 28C from Tampa to Minny.", "in_reply_to_status": -1, "in_reply_to_user": 7517222, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 7517222, 151129779, 1222639789 }}, "user": { "id": 215310469, "name": "Rints", "screen_name": "SharpSk8s", "lang": "en", "location": "Twin Cities (by way of Canada)", "create_at": date("2010-11-13"), "description": "resourceful. entrepreneurial. punctual. passionate. motivated. determined. enjoying the journey....", "followers_count": 141, "friends_count": 606, "statues_count": 2323 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938785535193088, "text": "Fr tho. �� https://t.co/ud6cFCKdg5", "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": 594, "friends_count": 416, "statues_count": 15196 }, "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("2015-11-26T10:00:27.000Z"), "id": 669938785573052416, "text": "Happy Thanksgiving from Christian Champion Wrestling! Hope you all have a great day full of family and food! #MuchLove", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MuchLove" }}, "user": { "id": 437727961, "name": "CCW™", "screen_name": "TheOnlyCCW", "lang": "en", "location": "Cary, North Carolina", "create_at": date("2011-12-15"), "description": "This is the official Twitter of the Christian Championship Wrestling foundation! https://m.youtube.com/channel/UCG4ZAScENHBwc0pTwvo7ZUQ", "followers_count": 414, "friends_count": 105, "statues_count": 1906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garner, NC", "id": "0a960df1a2b0abaf", "name": "Garner", "place_type": "city", "bounding_box": rectangle("-78.680992,35.649142 -78.546396,35.731468") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3725480, "cityName": "Garner" } }
+{ "create_at": datetime("2015-11-26T10:00:28.000Z"), "id": 669938785623416832, "text": "Thankful for these people��❤️ https://t.co/374WOzUFXN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2940445989, "name": "Evan Shawver", "screen_name": "EvanShawver1", "lang": "en", "location": "Amherst, OH", "create_at": date("2014-12-25"), "description": "⚾️", "followers_count": 562, "friends_count": 454, "statues_count": 820 }, "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("2015-11-26T10:00:28.000Z"), "id": 669938785950367744, "text": "Canada\nHungary\nAzerbaijan\nNigeria\nIndonesia \nYemen\nAustria https://t.co/efqlnKNiOJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 840527768, "name": "chaniya :-)", "screen_name": "chaniyaok", "lang": "en", "location": "null", "create_at": date("2012-09-22"), "description": "brisha doesnt have service", "followers_count": 1160, "friends_count": 456, "statues_count": 19442 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Comstock Park, MI", "id": "bf89ea14b197cfe6", "name": "Comstock Park", "place_type": "city", "bounding_box": rectangle("-85.699605,43.028969 -85.632777,43.079871") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26081, "countyName": "Kent", "cityID": 2617700, "cityName": "Comstock Park" } }
+{ "create_at": datetime("2015-11-26T10:00:28.000Z"), "id": 669938786218848256, "text": "Crazy girls are in full effect this morning", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 619436162, "name": "Daddy", "screen_name": "lost_qveen", "lang": "en", "location": "Denton, TX", "create_at": date("2012-06-26"), "description": "| UNT | Yea bro, I'm chillin.", "followers_count": 1026, "friends_count": 853, "statues_count": 55193 }, "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("2015-11-26T10:00:28.000Z"), "id": 669938786223169537, "text": "#teamkyle #kyleEdwards #hausofedwards @AlyssaEdwards_1 @LaganjaEstranja @GiaGunn @itsSHANGELA @ViviennePinay https://t.co/lm6doQUweH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "teamkyle", "kyleEdwards", "hausofedwards" }}, "user_mentions": {{ 944835650, 562248311, 594542811, 91441711, 320435426 }}, "user": { "id": 3118993451, "name": "Kyle vest", "screen_name": "blasteffects", "lang": "en", "location": "United States", "create_at": date("2015-03-26"), "description": "A man of many talents such as-Artist(airbrush, spfx, makeup beauty to prosthetic)(murals, flowers, cakes)(skater, stilts, aerialist, model) and way to much more", "followers_count": 599, "friends_count": 652, "statues_count": 1366 }, "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("2015-11-26T10:00:28.000Z"), "id": 669938786420310016, "text": "thankful @GriffinHoffmann ☺️ https://t.co/ZILatd8UV8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2172885328 }}, "user": { "id": 3390422524, "name": "rachel curry", "screen_name": "_rachelcurry_", "lang": "en", "location": "null", "create_at": date("2015-07-23"), "description": "null", "followers_count": 248, "friends_count": 262, "statues_count": 520 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, KY", "id": "788d40e08cf34358", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-85.563842,38.248987 -85.499936,38.297542") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2101504, "cityName": "Anchorage" } }
+{ "create_at": datetime("2015-11-26T10:00:28.000Z"), "id": 669938786554507264, "text": "7min AMRAP and instead of the C&J/ HSPU couple I just did a light Grace (95lbs)!!!! Immediately after… https://t.co/urblbhfdER", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19166667,39.75944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 62334374, "name": "Leondre' Dow", "screen_name": "DesertWiThEagle", "lang": "en", "location": "Dayton, Ohio ", "create_at": date("2009-08-02"), "description": "I rap... Working on my debut mixtape #FOODEHEAD so stay tuned!! iMassage, Military (10yrs), FitLife2014! Anything else tweet me... Eagle", "followers_count": 540, "friends_count": 1189, "statues_count": 14371 }, "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("2015-11-26T10:00:28.000Z"), "id": 669938786894090240, "text": "I'm thankful to be a Millennial.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27551085, "name": "Daniel Dudley", "screen_name": "DDisBORED", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-03-29"), "description": "@RadioDisney Weekend Host. Former PGH Sports Talk Host. Pop Culture Expert. Music Lover. #WVU Grad. Scored 15 pts. on LeBron James. WELLSBURG vs. EVERYBODY", "followers_count": 2833, "friends_count": 434, "statues_count": 69307 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:28.000Z"), "id": 669938787368214528, "text": "few times I agree with Jay Williams. this time he is right about Bama having patience with the retooling of the basketball team #BuckleUp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BuckleUp" }}, "user": { "id": 33363511, "name": "Shane Cooper", "screen_name": "SSCoop80", "lang": "en", "location": "Tuscaloosa, AL", "create_at": date("2009-04-19"), "description": "Sec A Row 21 Seat 2-3 never empty in Coleman #BamaHoops #94FTBOTHWAYS #BuiltByBama #RTR #TheProcessNeverEnds #BBN #WeAreUK #SFGiants #niners365", "followers_count": 211, "friends_count": 697, "statues_count": 5236 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Holt, AL", "id": "00b6f2fa67c8cebb", "name": "Holt", "place_type": "city", "bounding_box": rectangle("-87.505612,33.199755 -87.458301,33.247225") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1125, "countyName": "Tuscaloosa", "cityID": 135704, "cityName": "Holt" } }
+{ "create_at": datetime("2015-11-26T10:00:28.000Z"), "id": 669938787816857600, "text": "@JoseAndresFoods is a sexy sexy man", "in_reply_to_status": -1, "in_reply_to_user": 1286744155, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1286744155 }}, "user": { "id": 29333864, "name": "Jamie Todd", "screen_name": "jamietodd76", "lang": "en", "location": "Amarillo, TX", "create_at": date("2009-04-06"), "description": "Gay as fuck here, but still 100% man... I'll fuck you up...", "followers_count": 31, "friends_count": 66, "statues_count": 566 }, "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("2015-11-26T10:00:28.000Z"), "id": 669938787976282113, "text": "If you're a #BusinessMgmt professional in #NewYork, NY, check out this #job: https://t.co/t9nEyqnhLm #Hiring #CareerArc", "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": {{ "BusinessMgmt", "NewYork", "job", "Hiring", "CareerArc" }}, "user": { "id": 24220809, "name": "NYC Management Jobs", "screen_name": "tmj_nyc_mgmt", "lang": "en", "location": "New York, NY", "create_at": date("2009-03-13"), "description": "Follow this account for geo-targeted Business/Mgmt. job tweets in New York, NY. Need help? Tweet us at @CareerArc!", "followers_count": 614, "friends_count": 308, "statues_count": 765 }, "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("2015-11-26T10:00:28.000Z"), "id": 669938788244811779, "text": "#happythankgiving\nTHE HOLIDAY STANDARD PARTY TONIGHT #THANKSGIVINGNIGHT liegeoakland WE KEEPING IT… https://t.co/cDhuKtKTG7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.274231,37.8009605"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "happythankgiving", "THANKSGIVINGNIGHT" }}, "user": { "id": 27053682, "name": "HOLLAMUSIC", "screen_name": "DJHHolla", "lang": "en", "location": "Richmond (Bay Area California)", "create_at": date("2009-03-27"), "description": "San Francisco KVVF Hot 105.7 - 100.7 Weekend Mix Show Dj l HotAllstardj on Hot 105.7 - 100.7 l Booking Info Hollamusic@gmail.com l Social Networks: @djhholla", "followers_count": 3500, "friends_count": 3659, "statues_count": 69341 }, "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("2015-11-26T10:00:28.000Z"), "id": 669938788789960704, "text": "When your family isn't celebrating thanksgiving until tomorrow so you go to iHop instead", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2575160497, "name": "Rianna James", "screen_name": "rianna_james99", "lang": "en", "location": "null", "create_at": date("2014-06-18"), "description": "no matter what happens in life, be good to people. being good to people is a wonderful legacy to leave behind. -Taylor Swift", "followers_count": 77, "friends_count": 106, "statues_count": 295 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:28.000Z"), "id": 669938788802523136, "text": "Want to work at Department of Veterans Affairs? We're #hiring in #NewOrleans, LA! Click for details: https://t.co/vkP09binaL #Clerical #Job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.0705556,29.9647222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "NewOrleans", "Clerical", "Job" }}, "user": { "id": 31625510, "name": "TMJ-LAN Cleric. Jobs", "screen_name": "tmj_lan_cler", "lang": "en", "location": "New Orleans, LA", "create_at": date("2009-04-15"), "description": "Follow this account for geo-targeted Office/Clerical job tweets in New Orleans, LA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 238, "friends_count": 204, "statues_count": 18 }, "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("2015-11-26T10:00:28.000Z"), "id": 669938788861353989, "text": "@vantetate31 stop playing����", "in_reply_to_status": 669938705973567488, "in_reply_to_user": 2934189231, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2934189231 }}, "user": { "id": 1406438804, "name": "lucy", "screen_name": "lucyaclay", "lang": "en", "location": "shs-penta", "create_at": date("2013-05-05"), "description": "proverbs 31:30", "followers_count": 482, "friends_count": 438, "statues_count": 22545 }, "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("2015-11-26T10:00:28.000Z"), "id": 669938788894806016, "text": "C A L I F O R N I A B U R R I T O <3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 1012207272, "name": "Luis Uroza", "screen_name": "luis_uroza", "lang": "en", "location": "San Diego, CA", "create_at": date("2012-12-14"), "description": "I see you suffer from haterism you should get that checked", "followers_count": 117, "friends_count": 111, "statues_count": 94 }, "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-26T10:00:28.000Z"), "id": 669938788903206914, "text": "So thankful for all the wonderful people 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": 2295380964, "name": "Itzelly", "screen_name": "itzellsoto", "lang": "en", "location": "Casa Grande, AZ", "create_at": date("2014-01-16"), "description": "CGUHS '16 || Mami ✨", "followers_count": 905, "friends_count": 445, "statues_count": 40306 }, "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("2015-11-26T10:00:28.000Z"), "id": 669938789029052416, "text": "@marycateartone ty is the man��", "in_reply_to_status": 669930895525171200, "in_reply_to_user": 383955863, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 383955863 }}, "user": { "id": 506621089, "name": "Colt Boy", "screen_name": "ColtonD_", "lang": "en", "location": "Windermere, FL", "create_at": date("2012-02-27"), "description": "Catch me at Panera", "followers_count": 293, "friends_count": 310, "statues_count": 15471 }, "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("2015-11-26T10:00:28.000Z"), "id": 669938789255540737, "text": "@2chilldyll is my #wcw on thanksgiving Thursday its all good ��", "in_reply_to_status": 669602699571101696, "in_reply_to_user": 2879296126, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wcw" }}, "user_mentions": {{ 2879296126 }}, "user": { "id": 460868968, "name": "erika", "screen_name": "crashthesystem", "lang": "en", "location": "bummerlin", "create_at": date("2012-01-10"), "description": "I trip to make the fall shorter", "followers_count": 426, "friends_count": 160, "statues_count": 8091 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938789830258688, "text": "I'm so thankful to have someone who loves me and wants to be with me forever . I'm so thankful I have you I love you https://t.co/6LvWsCMG7v", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 162850386, "name": "Moe", "screen_name": "chunkiieemonkii", "lang": "en", "location": "null", "create_at": date("2010-07-04"), "description": "#4eva #RIPJAY #4", "followers_count": 850, "friends_count": 723, "statues_count": 39436 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938789926572034, "text": "Love everything about this https://t.co/lclnBc77Wv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 137531592, "name": "Will Leivenberg", "screen_name": "Will_Leivenberg", "lang": "en", "location": "San Francisco, CA", "create_at": date("2010-04-26"), "description": "Bleacher Report's Media Lab Project Manager. So you're sayin there's a chance?", "followers_count": 498, "friends_count": 327, "statues_count": 1216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:29.000Z"), "id": 669938790002237444, "text": "Best Xbone bundle on sale IMO \n(via Newegg)\ncc: @ChumsKnifeblade https://t.co/WfCBZ5kdJl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user_mentions": {{ 25950116 }}, "user": { "id": 83736832, "name": "Bobby Wesley Cruz", "screen_name": "TheBlacktastic", "lang": "en", "location": "Cleveland, OH", "create_at": date("2009-10-19"), "description": "Producer & Host of #eSports podcast @MeterBurnTV | Speedrunner & streamer on @Twitch | Tech/VG Journalist at @Geektasia | Business Inquiries: b.cruz001@live.com", "followers_count": 1358, "friends_count": 466, "statues_count": 66505 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938790018912256, "text": "@48john rather them play against green bay or Vikings", "in_reply_to_status": 669938555498700801, "in_reply_to_user": 30945674, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 30945674 }}, "user": { "id": 1481125982, "name": "Armand D. Tensai", "screen_name": "TENSAINEXTDOOR", "lang": "en", "location": "Shohoku", "create_at": date("2013-06-03"), "description": "Third year at Shohoku. Guy that won inter-high.Blackbear fan. #SpookyBlackHive. I never score I just assist cause I'm ugly", "followers_count": 5980, "friends_count": 1110, "statues_count": 107890 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938790069223424, "text": "Want to work at Ghirardelli? We're #hiring in #Anaheim, CA! Click for details: https://t.co/ahIzul8b0W #BusinessMgmt #management #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.9145036,33.8352932"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Anaheim", "BusinessMgmt", "management", "Job", "Jobs" }}, "user": { "id": 2228943546, "name": "Ghirardelli Careers", "screen_name": "ghirardellijobs", "lang": "en", "location": "null", "create_at": date("2013-12-03"), "description": "Ghirardelli invites you to explore our open #jobs and discover a place where your ideas come to life, as part of our delicious legacy.", "followers_count": 54, "friends_count": 1, "statues_count": 98 }, "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-26T10:00:29.000Z"), "id": 669938790086148097, "text": "@PorterFSU you know what i said mofo lol https://t.co/BAuCQoLOS8", "in_reply_to_status": 669938605020880896, "in_reply_to_user": 43399122, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 43399122 }}, "user": { "id": 1918184400, "name": "Dab Daddy", "screen_name": "chrissingh359", "lang": "en", "location": "fredonia", "create_at": date("2013-09-29"), "description": "null", "followers_count": 330, "friends_count": 405, "statues_count": 4660 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938790098673664, "text": "Just want to thank @getquip for being awesome and getting us exactly what we needed. Can't wait for my brushes! #aboveandbeyond", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "aboveandbeyond" }}, "user_mentions": {{ 2680420820 }}, "user": { "id": 14914565, "name": "Renée Tessier Nunley", "screen_name": "nauset", "lang": "en", "location": "New York", "create_at": date("2008-05-26"), "description": "artist/photographer/graphic designer", "followers_count": 380, "friends_count": 591, "statues_count": 4538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McMinnville, TN", "id": "ad36563abe8783b9", "name": "McMinnville", "place_type": "city", "bounding_box": rectangle("-85.840585,35.631688 -85.72916,35.718131") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47177, "countyName": "Warren", "cityID": 4745100, "cityName": "McMinnville" } }
+{ "create_at": datetime("2015-11-26T10:00:29.000Z"), "id": 669938790203420672, "text": "@I_Am_Mooner10 @wildasadaydream thankful for you brother", "in_reply_to_status": 669908518913204225, "in_reply_to_user": 1337758220, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1337758220, 2408252924 }}, "user": { "id": 436289127, "name": "Garrett Bitton", "screen_name": "garrett_bitton", "lang": "en", "location": "Syracuse, UT", "create_at": date("2011-12-13"), "description": "I drive a Diesel.", "followers_count": 867, "friends_count": 723, "statues_count": 8353 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938790442602497, "text": "Best thanksgiving I could ask for... My newborn son and my beautiful niece both healthy as can be...… https://t.co/n9C2Lj8zaD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.7263031,38.9376297"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 38996924, "name": "AmbitousKing ♌️♌️♌", "screen_name": "HassanTeamDCLU", "lang": "en", "location": " Lawrence Stacking", "create_at": date("2009-05-09"), "description": "Trouble Man #TeamDCLU#TEAMFOLLOWME #teamUCM#TEAMKELLS#TeamLakers#Team24#Darkskin I got me", "followers_count": 1091, "friends_count": 1328, "statues_count": 10687 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938790522298369, "text": "MARIAH!!!! https://t.co/r61tSnVR5A", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 41815073, "name": "Chloe Evans", "screen_name": "HelloChloe_", "lang": "en", "location": "downtown", "create_at": date("2009-05-22"), "description": "on a scale of one to nick miller, what's your relevance to my life", "followers_count": 353, "friends_count": 147, "statues_count": 22291 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938790551556098, "text": "nd then there's this boy ���� my maid of honor. the coolest cat I've ever met, my cuz, my rock, my go to guy. forever. https://t.co/vPgxkVVjpP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3066438966, "name": "☪Elisabeth alyssa", "screen_name": "lisabeth_alyssa", "lang": "en", "location": "Austin, TX", "create_at": date("2015-03-07"), "description": "but, my dear, this is definitely not wonderland, and you are not alice.", "followers_count": 151, "friends_count": 137, "statues_count": 1823 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938790564274176, "text": "@AfricasaCountry he couldn't tell you if he tried it just fell out his mouth.", "in_reply_to_status": 669935150571102208, "in_reply_to_user": 399156475, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 399156475 }}, "user": { "id": 61758499, "name": "Anthony Oseyemi", "screen_name": "AnthonyOseyemi", "lang": "en", "location": "Johannesburg South Africa", "create_at": date("2009-07-31"), "description": "Actor - played #Isidingo bad boy - Hamilton, Writer, Singer, Song-writer, Mentor, Entrepreneur (@4DEntertainMe), FULL ON CREATIVE - 'shout Out to my moms'", "followers_count": 1505, "friends_count": 959, "statues_count": 5883 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938790660636676, "text": "https://t.co/gIrKmENcgE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 15273924, "name": "IHeartNoise / Ilya S", "screen_name": "Iheartnoise", "lang": "en", "location": "Boston", "create_at": date("2008-06-29"), "description": "Trying to grapple w/my Wronglish. Small Bos label (@skyjelly @andavolley @toypianoband).Famed rock critic @bostonhassle + @pixeldeep / https://t.co/LGaL6A09zO", "followers_count": 1868, "friends_count": 1656, "statues_count": 26576 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938790912253952, "text": "In thankful for a great family, a great life, a great job, good healthcare & a country that allows for differing ideas. God bless the USA.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 986712462, "name": "Marc Probst", "screen_name": "probst_marc", "lang": "en", "location": "null", "create_at": date("2012-12-03"), "description": "null", "followers_count": 385, "friends_count": 60, "statues_count": 160 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Holladay, UT", "id": "a77dd27b5fca115c", "name": "Holladay", "place_type": "city", "bounding_box": rectangle("-111.841587,40.62919 -111.789394,40.675376") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4936070, "cityName": "Holladay" } }
+{ "create_at": datetime("2015-11-26T10:00:29.000Z"), "id": 669938790912274433, "text": "waking up on thanksgiving and not being at home doesn't feel right ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30395911, "name": "Em ❤️", "screen_name": "EmFini", "lang": "en", "location": "null", "create_at": date("2009-04-10"), "description": "you live, you learn, you upgrade. @lanceisallred is allll mine ❤️", "followers_count": 336, "friends_count": 207, "statues_count": 6285 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938790979514368, "text": "Turk turk turk turk dat ass ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 734259019, "name": "Keila Fitzpatrick", "screen_name": "KeilaFitzpatric", "lang": "en", "location": "East Lansing, MI", "create_at": date("2012-08-02"), "description": "null", "followers_count": 93, "friends_count": 314, "statues_count": 1938 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Royal Oak, MI", "id": "47cfe29f57708110", "name": "Royal Oak", "place_type": "city", "bounding_box": rectangle("-83.204418,42.474131 -83.111076,42.540305") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2670040, "cityName": "Royal Oak" } }
+{ "create_at": datetime("2015-11-26T10:00:29.000Z"), "id": 669938791121993732, "text": "God bless everyone today, Happy Thanksgiving, go Lions, and more importantly, #GoBlue!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GoBlue" }}, "user": { "id": 68014120, "name": "Jordan Rennolds", "screen_name": "Jtrain023", "lang": "en", "location": "St Peters, MO", "create_at": date("2009-08-22"), "description": "Represent Detroit and St. Louis. I enjoy sports, pooping at work, and a good burrito. #GoBlue", "followers_count": 304, "friends_count": 810, "statues_count": 12601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cottleville, MO", "id": "504502453f3b749d", "name": "Cottleville", "place_type": "city", "bounding_box": rectangle("-90.694611,38.736272 -90.624819,38.788678") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29183, "countyName": "St. Charles", "cityID": 2916678, "cityName": "Cottleville" } }
+{ "create_at": datetime("2015-11-26T10:00:29.000Z"), "id": 669938791428173824, "text": "Registered Nurse RN Travel Cath... - Supplemental Health Care: (#Jackson, MI) https://t.co/NuznuTLFYa #Healthcare https://t.co/L7jj6jGN7P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.4013462,42.245869"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Jackson", "Healthcare" }}, "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": 735, "friends_count": 1, "statues_count": 66233 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jackson, MI", "id": "06ab95cfe367eee7", "name": "Jackson", "place_type": "city", "bounding_box": rectangle("-84.493659,42.176663 -84.364192,42.276421") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26075, "countyName": "Jackson", "cityID": 2641420, "cityName": "Jackson" } }
+{ "create_at": datetime("2015-11-26T10:00:29.000Z"), "id": 669938791436554241, "text": "I'm ready to gain 10 pounds today ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1864868784, "name": "Nikki Guerrero ✨", "screen_name": "GuerreroNikki", "lang": "en", "location": "McAllen, TX", "create_at": date("2013-09-14"), "description": "She is more precious than rubies. Nothing you desire can compare with her. -Proverbs 3:15", "followers_count": 464, "friends_count": 532, "statues_count": 3127 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938791503675392, "text": "CVS Health: Retail Store Shift Supervisor (#Hawthorne, CA) https://t.co/yhvcFQ85KQ #Retail #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.3525748,33.9164032"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hawthorne", "Retail", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 28466494, "name": "TMJ-LAX Retail Jobs", "screen_name": "tmj_lax_retail", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-04-02"), "description": "Follow this account for geo-targeted Retail job tweets in Los Angeles, CA. Need help? Tweet us at @CareerArc!", "followers_count": 471, "friends_count": 314, "statues_count": 1605 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938792187346944, "text": "If you're a #Nursing professional in #Springfield, MA, check out this #job: https://t.co/crDXr56OKQ #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.589811,42.1014831"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "Springfield", "job", "Hiring", "CareerArc" }}, "user": { "id": 36282465, "name": "TMJ-MAW Nursing Jobs", "screen_name": "tmj_maw_nursing", "lang": "en", "location": "Western Area, MA", "create_at": date("2009-04-28"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Western Area, MA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 372, "friends_count": 306, "statues_count": 21 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938792598388736, "text": "Turkey trot with the best��❤️ https://t.co/0KEonQyGEc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2853618896, "name": "Kirsty♔", "screen_name": "Kirsty_710", "lang": "en", "location": "null", "create_at": date("2014-10-12"), "description": "ҜIRST➓", "followers_count": 405, "friends_count": 423, "statues_count": 2156 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fruitland, ID", "id": "106b3189709bdcdb", "name": "Fruitland", "place_type": "city", "bounding_box": rectangle("-116.936332,43.99859 -116.906059,44.041763") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16075, "countyName": "Payette", "cityID": 1628990, "cityName": "Fruitland" } }
+{ "create_at": datetime("2015-11-26T10:00:29.000Z"), "id": 669938792925683712, "text": "@ThatAsianMaddie catch him", "in_reply_to_status": 669937404262920193, "in_reply_to_user": 275802201, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 275802201 }}, "user": { "id": 380563263, "name": "Kyle", "screen_name": "swendaddy", "lang": "en", "location": "Margaritaville", "create_at": date("2011-09-26"), "description": "#UofL17 Tau Kappa Epsilon Rush Chair #RushTKE Official Brand Rep for @RowdyGentleman, @ACKBuckets, @USAPalm, and @BeatboxBevs.", "followers_count": 610, "friends_count": 147, "statues_count": 39932 }, "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-26T10:00:29.000Z"), "id": 669938793101832197, "text": "I love Banana Pudding 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": 609237389, "name": "⚡Сourey⚡", "screen_name": "Courey_Amonte", "lang": "en", "location": " West Baltimore, MD", "create_at": date("2012-06-15"), "description": "Once Upon A Time In Baltimore.", "followers_count": 2902, "friends_count": 2708, "statues_count": 24483 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938793135349760, "text": "Twins & dink �� https://t.co/CyPTjLrgll", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1033396430, "name": "bambi.", "screen_name": "_liljass", "lang": "en", "location": "Huntington, WV", "create_at": date("2012-12-24"), "description": "So many years of education yet nobody ever taught us how to love ourselves & why it's so important.", "followers_count": 1833, "friends_count": 1169, "statues_count": 97257 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938793223352320, "text": "We're #hiring! Read about our latest #job opening here: Pharmacy Technician - https://t.co/ugQYg2ANFX #Raleigh, NC #Healthcare", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.6386145,35.772096"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Raleigh", "Healthcare" }}, "user": { "id": 563319506, "name": "CVS Health Jobs", "screen_name": "CVSHealthJobs", "lang": "en", "location": "null", "create_at": date("2012-04-25"), "description": "null", "followers_count": 430, "friends_count": 3, "statues_count": 52215 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938793328152576, "text": "Interested in a #Sales #job near #Burlington, MA? This could be a great fit: https://t.co/CKjLwJi754 #Hiring #CareerArc", "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", "CareerArc" }}, "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": 686, "friends_count": 319, "statues_count": 889 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938793332379648, "text": "@landonrice100 @sierrambanks good luck getting your puppy back", "in_reply_to_status": -1, "in_reply_to_user": 2331074328, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2331074328, 2341065716 }}, "user": { "id": 1575566868, "name": "kirtie", "screen_name": "krowlllley", "lang": "en", "location": "post falls, id", "create_at": date("2013-07-07"), "description": "I'm not who I used to be, I am redeemed", "followers_count": 686, "friends_count": 233, "statues_count": 5053 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Post Falls, ID", "id": "0e45c1c6e13dff2a", "name": "Post Falls", "place_type": "city", "bounding_box": rectangle("-117.041712,47.692705 -116.854054,47.745105") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16055, "countyName": "Kootenai", "cityID": 1664810, "cityName": "Post Falls" } }
+{ "create_at": datetime("2015-11-26T10:00:29.000Z"), "id": 669938793470930944, "text": "@poweredbycoke0 check my flikr page from the pinned tweet. There's more stuff there.", "in_reply_to_status": 669935623554203648, "in_reply_to_user": 23403866, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23403866 }}, "user": { "id": 249690106, "name": "Dävïdthëdäd |-/", "screen_name": "thedavidthedad", "lang": "en", "location": "Georgia, America ", "create_at": date("2011-02-09"), "description": "I take pictures sometimes and they're available no charge at http://flic.kr/s/aHskkg7jjA. Still not Tyler's dad. Still not Jøsh's dad.", "followers_count": 7374, "friends_count": 6812, "statues_count": 25745 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jefferson, GA", "id": "00835b743f2660bd", "name": "Jefferson", "place_type": "city", "bounding_box": rectangle("-83.63061,34.084144 -83.544854,34.167069") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13157, "countyName": "Jackson", "cityID": 1341988, "cityName": "Jefferson" } }
+{ "create_at": datetime("2015-11-26T10:00:29.000Z"), "id": 669938793491886086, "text": "I make films like a girl! Do you? #womeninfilm #womendirectors https://t.co/XrKsf376Dg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "womeninfilm", "womendirectors" }}, "user": { "id": 2481231044, "name": "Chomp Movie", "screen_name": "ChompMovie", "lang": "en", "location": "Tampa, FL", "create_at": date("2014-05-06"), "description": "An award-winning #horror comedy short by @LynneHansen. Misery w/ #zombies for #indiefilm fans. https://www.facebook.com/ChompMovie", "followers_count": 424, "friends_count": 902, "statues_count": 462 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938793533669376, "text": "Free donte ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2556862811, "name": "BadGirlTay", "screen_name": "PlayaaTickii_", "lang": "en", "location": "null", "create_at": date("2014-05-20"), "description": "sc : PlayaaTickii1 | MoreTickiiLessYou❤️ | DTX✨", "followers_count": 655, "friends_count": 537, "statues_count": 5473 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938793634504704, "text": "@nolechick94 nope. corporate retail. In charge of shipping and distribution for Saks Fifth.", "in_reply_to_status": 669938026936737793, "in_reply_to_user": 14712698, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14712698 }}, "user": { "id": 776413200, "name": "David the Gnome", "screen_name": "haterofgators82", "lang": "en", "location": "A well groomed yard...", "create_at": date("2012-08-23"), "description": "Diehard Nole, that's all that matters... whiskey bent and hell bound.", "followers_count": 841, "friends_count": 758, "statues_count": 31757 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Smyrna, TN", "id": "f9eb63ab4f7dd5db", "name": "Smyrna", "place_type": "city", "bounding_box": rectangle("-86.624429,35.879451 -86.458783,36.029213") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47149, "countyName": "Rutherford", "cityID": 4769420, "cityName": "Smyrna" } }
+{ "create_at": datetime("2015-11-26T10:00:29.000Z"), "id": 669938793663758336, "text": "When Youu tell Your Grandma You ain't Eat yet, so she start Fixing you a BigPlate, But it's Really Your 3rd Plate!�� https://t.co/KaE75pUAqG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 90805864, "name": "_Blackjohnnybravo_", "screen_name": "jarrell27", "lang": "en", "location": "Bottom of da Boot", "create_at": date("2009-11-17"), "description": "#teamSwag #teamEducated #teamDroid #teamEmployed #teamFollowBACK #teamBIG #teamFREAKY .. IF I FWU DEN FWM PLAIN AND SIMPLE!", "followers_count": 191, "friends_count": 206, "statues_count": 865 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938793831464961, "text": "@juli_abes @SidneyBirmele love u ����", "in_reply_to_status": 669936609895186432, "in_reply_to_user": 2649474601, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2649474601, 2232444956 }}, "user": { "id": 3296965622, "name": "gillian", "screen_name": "gillianelson", "lang": "en", "location": "212 ", "create_at": date("2015-07-26"), "description": "I may be young but I'm ready", "followers_count": 110, "friends_count": 86, "statues_count": 495 }, "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("2015-11-26T10:00:29.000Z"), "id": 669938793936510976, "text": "Pretty sure Guinness world records is gonna stop by my house later because i just made the bombest mac n cheese of all time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 419571458, "name": "Emma", "screen_name": "emmaenav", "lang": "en", "location": "null", "create_at": date("2011-11-23"), "description": "cooler than the flipside of your pillow", "followers_count": 401, "friends_count": 364, "statues_count": 19816 }, "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("2015-11-26T10:00:30.000Z"), "id": 669938793974095872, "text": "@TallShipProds @SamHeughan @kricki @caitrionambalfe \nNó b'fhéidir...\nAnything but ginger!\n#zingback", "in_reply_to_status": 669937478929813504, "in_reply_to_user": 226700918, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "zingback" }}, "user_mentions": {{ 226700918, 336445332, 15521834, 175098387 }}, "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": 3654, "friends_count": 3525, "statues_count": 84494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:30.000Z"), "id": 669938794037161984, "text": "thankful for everyone in my life idc if we talk once a year still thankful for u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 548739076, "name": "marissa", "screen_name": "marissacipriano", "lang": "en", "location": "Clinton, MI", "create_at": date("2012-04-08"), "description": "high on life☔️☔️", "followers_count": 378, "friends_count": 131, "statues_count": 14428 }, "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.539274 -82.855854,42.630481") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26091, "countyName": "Lenawee", "cityID": 2616480, "cityName": "Clinton" } }
+{ "create_at": datetime("2015-11-26T10:00:30.000Z"), "id": 669938794058022912, "text": "I didnt know it was my parents 21st anniversary!?!?!?!?!?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2179000873, "name": "Matt", "screen_name": "schoolboy_matt", "lang": "en", "location": "in ariels bed", "create_at": date("2013-11-06"), "description": "kern??? #GigEm #providence2final4 #D1 #CowboysNation", "followers_count": 472, "friends_count": 401, "statues_count": 15216 }, "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("2015-11-26T10:00:30.000Z"), "id": 669938794087522304, "text": "من طبيت شيكاغو قالوا عني اثيوبيا عقب ايطاليا اخر شي الصدمه يقول انتوا يا من اوربا الشرقيه او تل ابيب", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 253613550, "name": "ش", "screen_name": "shayookhat", "lang": "en", "location": "La La land", "create_at": date("2011-02-17"), "description": "Aquarius ♒ Live,Laugh,Dream My family mean the world to me اللهم ارحم عمتي فاطمه و اجعل قبرها روضه من رياض جنتك", "followers_count": 190, "friends_count": 205, "statues_count": 7600 }, "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-26T10:00:30.000Z"), "id": 669938794141913089, "text": "Just waking up! Happy Thanksgiving Everyone! Lowkey getting that Golden Chicken to start 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": 1134909308, "name": "Franco ➰", "screen_name": "1HeroFrank", "lang": "en", "location": "null", "create_at": date("2013-01-30"), "description": "Started in Pallet Town-Born a Firebender-Own a Patamon (Instagram:1herofrank)", "followers_count": 166, "friends_count": 201, "statues_count": 13539 }, "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-26T10:00:30.000Z"), "id": 669938794204762112, "text": "@_RealityVenom_ it happened in the last football game", "in_reply_to_status": 669937892207280128, "in_reply_to_user": 4252503807, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4252503807 }}, "user": { "id": 1090787496, "name": "Matt Kaminski", "screen_name": "mattkaminski51", "lang": "en", "location": "Toledo, Ohio", "create_at": date("2013-01-14"), "description": "comeback kid", "followers_count": 363, "friends_count": 426, "statues_count": 7437 }, "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-26T10:00:30.000Z"), "id": 669938794213150720, "text": "Want to work in #TheWoodlands, TX? View our latest opening: https://t.co/QAqXYDCKh4 #Healthcare #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.4609232,30.1604591"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TheWoodlands", "Healthcare", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 21568228, "name": "TMJ- HOU Health Jobs", "screen_name": "tmj_hou_health", "lang": "en", "location": "Houston, TX", "create_at": date("2009-02-22"), "description": "Follow this account for geo-targeted Healthcare job tweets in Houston, TX from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 509, "friends_count": 311, "statues_count": 1444 }, "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("2015-11-26T10:00:30.000Z"), "id": 669938794490175488, "text": "��PRE BLACK FRIDAY SALE⚫️\n♣️50% OFF on all orders♦️\n♥️Enter \"Blkfriday\" at checkout♠️\nSale end… https://t.co/FAL7IHgrVB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.3022385,25.8415012"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1070309124, "name": "Laz Fresnedo", "screen_name": "LaZFreZno", "lang": "en", "location": "null", "create_at": date("2013-01-08"), "description": "LaZ The Mutant Turtle FreZno // BELIEVE / Hialeah / Capricorn / 28 / VenezCuban / Instagram: @laz_frezno #teamFreZno", "followers_count": 250, "friends_count": 259, "statues_count": 2233 }, "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-26T10:00:30.000Z"), "id": 669938794498494464, "text": "It's sad when you can't even say you have a family.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 345954517, "name": "partygetsmewetter", "screen_name": "dvnihana_", "lang": "en", "location": "free squeak free curt", "create_at": date("2011-07-31"), "description": "keep em all on the need to know. #UMES19 all about loco & rashi ❤️", "followers_count": 5911, "friends_count": 6328, "statues_count": 130881 }, "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("2015-11-26T10:00:30.000Z"), "id": 669938794645336064, "text": "update: im dying!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2671129849, "name": "s(eb)anta//17", "screen_name": "frantaxgrande", "lang": "en", "location": "Miami, FL", "create_at": date("2014-07-22"), "description": "oh my god, you're so cute...i love you too sweetie! -agb 7.18.15", "followers_count": 2758, "friends_count": 1018, "statues_count": 17455 }, "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("2015-11-26T10:00:30.000Z"), "id": 669938794670350336, "text": "With 14 manufacturing centers located in 5 countries, I can ship #Art & #Decor to any destination in the world. https://t.co/goVkXLUWnz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Art", "Decor" }}, "user": { "id": 55303089, "name": "Chuck Staley", "screen_name": "ChuckStaley", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-07-09"), "description": "Art Deco & Geo Designer, Concept Artist, TV Director and Author. See my Art & Designs at my website and my books on Amazon.", "followers_count": 4229, "friends_count": 391, "statues_count": 31132 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-26T10:00:30.000Z"), "id": 669938794670395392, "text": "Couldn't describe today any more accurately ���� #HappyThanksgiving https://t.co/iNp9T28vDD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HappyThanksgiving" }}, "user": { "id": 29632237, "name": "Sara", "screen_name": "SairaLowpez", "lang": "en", "location": "Somewhere in Illinois ", "create_at": date("2009-04-07"), "description": "NCC Alum ♐️ 22", "followers_count": 226, "friends_count": 461, "statues_count": 12834 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warrenville, IL", "id": "74eb8639ee552ba4", "name": "Warrenville", "place_type": "city", "bounding_box": rectangle("-88.224375,41.794976 -88.147535,41.852493") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1778929, "cityName": "Warrenville" } }
+{ "create_at": datetime("2015-11-26T10:00:30.000Z"), "id": 669938795001847809, "text": "That was the most Sim-like I have ever felt.", "in_reply_to_status": 669938426041495553, "in_reply_to_user": 2358774781, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2358774781, "name": "katie jane fernelius", "screen_name": "KatieFernelius", "lang": "en", "location": "null", "create_at": date("2014-02-23"), "description": "student & storyteller | current relationship status: it's complicated with @DukeU", "followers_count": 440, "friends_count": 856, "statues_count": 1609 }, "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("2015-11-26T10:00:30.000Z"), "id": 669938795203182592, "text": "Happy birthday to the most high @djkhaled", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27673684 }}, "user": { "id": 1631267646, "name": "abel", "screen_name": "_abel_rodriguez", "lang": "en", "location": "Orlando, FL", "create_at": date("2013-07-29"), "description": "ucf", "followers_count": 569, "friends_count": 312, "statues_count": 34297 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Golden Gate, FL", "id": "0bc912edeeadc71a", "name": "Golden Gate", "place_type": "city", "bounding_box": rectangle("-81.719513,26.168823 -81.686846,26.19952") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12021, "countyName": "Collier", "cityID": 1226300, "cityName": "Golden Gate" } }
+{ "create_at": datetime("2015-11-26T10:00:30.000Z"), "id": 669938795291127808, "text": "We're #hiring! Click to apply: Pricing Administrator - https://t.co/gt3lIsRCkZ #Sales #Plymouth, MI #Veterans #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.450828,42.357662"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Sales", "Plymouth", "Veterans", "Job", "Jobs" }}, "user": { "id": 1157303605, "name": "Plastipak Jobs", "screen_name": "PlastipakJobs", "lang": "en", "location": "null", "create_at": date("2013-02-07"), "description": "Plastipak Packaging is an industry leader in the design and manufacturing of plastic rigid containers of the highest quality.", "followers_count": 236, "friends_count": 30, "statues_count": 6409 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plymouth Township, MI", "id": "834a0121129de419", "name": "Plymouth Township", "place_type": "city", "bounding_box": rectangle("-83.549674,42.349201 -83.429945,42.396057") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne" } }
+{ "create_at": datetime("2015-11-26T10:00:30.000Z"), "id": 669938795442274305, "text": "Da Greyhound Station #OGGoesToTucson #OGTrip https://t.co/mwwAVyqZwt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "OGGoesToTucson", "OGTrip" }}, "user": { "id": 2938619564, "name": "Gabe", "screen_name": "OGBobbyGabriel", "lang": "en", "location": "ASU", "create_at": date("2014-12-21"), "description": "Honestly I'm a loser", "followers_count": 4434, "friends_count": 4526, "statues_count": 8883 }, "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-26T10:00:30.000Z"), "id": 669938795983171584, "text": "YES YES YES YES YES https://t.co/RT6Lln35sr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 440906491, "name": "Maury Hales", "screen_name": "Ciroc_Obama96", "lang": "en", "location": "Luling, LA", "create_at": date("2011-12-19"), "description": "Striving for greatness while living everyday as its last. LSU '18 ΘΧ", "followers_count": 369, "friends_count": 416, "statues_count": 6460 }, "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("2015-11-26T10:00:30.000Z"), "id": 669938796041867265, "text": "@caaannddiiccee aww thanks, miss you! happy thanksgiving! Have a good one��", "in_reply_to_status": 669938587190689794, "in_reply_to_user": 373810936, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 373810936 }}, "user": { "id": 126991771, "name": "Miss Independent", "screen_name": "kiiiraaraa", "lang": "en", "location": "Laters Baby", "create_at": date("2010-03-27"), "description": "✝ #GOE ✝ • ⚽ • l Pacific University Soccer l #MVT The adventures of me, myself, and I.", "followers_count": 2020, "friends_count": 934, "statues_count": 93916 }, "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("2015-11-26T10:00:30.000Z"), "id": 669938796130123776, "text": "�� https://t.co/pk2wspfYtU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3530529492, "name": "Sc: aangelleeee", "screen_name": "angelle_davis", "lang": "en", "location": "null", "create_at": date("2015-09-11"), "description": "sc: aangelleeee|April 15|Single\n@mollywaterrr_❤", "followers_count": 215, "friends_count": 213, "statues_count": 1332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hahnville, LA", "id": "633de12f5f2d4aac", "name": "Hahnville", "place_type": "city", "bounding_box": rectangle("-90.4326,29.942371 -90.388923,29.988476") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22089, "countyName": "St. Charles", "cityID": 2232510, "cityName": "Hahnville" } }
+{ "create_at": datetime("2015-11-26T10:00:30.000Z"), "id": 669938796214001664, "text": "#THANKSGIVINGDAY #GODBLESSMYFRIENDS @mike52006 @steph93065 @Callisto1947 @JstanleyStanley @JoAnnEileen @SpreadButter @sparksburns", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.7499833,40.7065843"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "THANKSGIVINGDAY", "GODBLESSMYFRIENDS" }}, "user_mentions": {{ 582697898, 261749735, 208083510, 754370958, 33320422, 26643566, 51738464 }}, "user": { "id": 1098165805, "name": "Jose A. Martinez", "screen_name": "lobomundo1", "lang": "en", "location": "NewYork ", "create_at": date("2013-01-17"), "description": "PuertoRican (http://U.S.Army) Conservative , http://Ex.Democrat , Loves Jesus Christ ( Elvis , Frank , Reba , Chet Baker , Louis Armstrong)", "followers_count": 2376, "friends_count": 2383, "statues_count": 4654 }, "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("2015-11-26T10:00:30.000Z"), "id": 669938796251758592, "text": "He always dropping the pass", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2329271801, "name": "HISDREADSSOLAVISH", "screen_name": "chief__steve", "lang": "en", "location": "Ann Arbor, MI", "create_at": date("2014-02-06"), "description": "University of 〽️ichigan class of 18. The least liked nigga in a group of extremely likable niggas. Pick 6️⃣", "followers_count": 471, "friends_count": 492, "statues_count": 37160 }, "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.539274 -82.855854,42.630481") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26091, "countyName": "Lenawee", "cityID": 2616480, "cityName": "Clinton" } }
+{ "create_at": datetime("2015-11-26T10:00:30.000Z"), "id": 669938796797018112, "text": "@beingagrownman Thanks for the follow ��", "in_reply_to_status": -1, "in_reply_to_user": 3433468041, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3433468041 }}, "user": { "id": 3492785296, "name": "John c Laslow", "screen_name": "kronk64", "lang": "en", "location": "Lexington, SC", "create_at": date("2015-08-30"), "description": "Originally from Penn Hills,PA. Like Steelers,Pens, zombies,cats and people with a good sense of humor. Don't take life so seriously!", "followers_count": 549, "friends_count": 890, "statues_count": 6811 }, "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("2015-11-26T10:00:30.000Z"), "id": 669938796855586817, "text": "@Redskins https://t.co/ulIMaHdMKy", "in_reply_to_status": 669878560123940865, "in_reply_to_user": 36375662, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 36375662 }}, "user": { "id": 214610383, "name": "Trey", "screen_name": "T___Rey", "lang": "en", "location": "Oklahoma, USA", "create_at": date("2010-11-11"), "description": "@KState Alumni that Likes: sports,food,news,comedy,movies,hiphop,@Royals♚ @DallasCowboys✭@OKCThunder⚡@Chiefs➳ •Kansan✈️Floridian✈️Texan✈️Oklahoman•", "followers_count": 4936, "friends_count": 472, "statues_count": 42732 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enid, OK", "id": "4ba5405c68ac4670", "name": "Enid", "place_type": "city", "bounding_box": rectangle("-97.981671,36.335631 -97.818889,36.448929") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40047, "countyName": "Garfield", "cityID": 4023950, "cityName": "Enid" } }
+{ "create_at": datetime("2015-11-26T10:00:30.000Z"), "id": 669938796998299648, "text": "Happy day !! ��������✨������❤️Thanksgiving ��✨✨✨���� @ Chicago, Illinois https://t.co/HFZipiSBom", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6844,41.8369"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2167390714, "name": "Linah Meirelles", "screen_name": "29Linah", "lang": "pt", "location": "Sao Paulo, Brazil", "create_at": date("2013-11-03"), "description": "I love my Life !", "followers_count": 5, "friends_count": 42, "statues_count": 3128 }, "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-26T10:00:30.000Z"), "id": 669938797480558592, "text": "If that's your girl I hate to 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": 189596354, "name": "T W E L V E 18♐️❤️", "screen_name": "AvenueB_", "lang": "en", "location": "New Orleans ✈️ Houston ", "create_at": date("2010-09-11"), "description": "You Don't Know Me!!! You Know What I Allow You To See - AveB", "followers_count": 1348, "friends_count": 1073, "statues_count": 82816 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:30.000Z"), "id": 669938797489033216, "text": "Now that's the Lions I want to see today.....better keep this going and not puke it up. Hate the damn eagles", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 180406148, "name": "Christopher Opito", "screen_name": "Opitoc", "lang": "en", "location": "Manchester, CT", "create_at": date("2010-08-19"), "description": "A major weather geek, Enjoying life with my 10 year old daughter, and my amazingly beautiful girlfriend. work as a Warehouse Supervisor. NY Giants-NY Mets-Pen's", "followers_count": 285, "friends_count": 856, "statues_count": 12674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manchester, CT", "id": "f1d134c7fd204d74", "name": "Manchester", "place_type": "city", "bounding_box": rectangle("-72.583489,41.733619 -72.465121,41.820226") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 944690, "cityName": "Manchester" } }
+{ "create_at": datetime("2015-11-26T10:00:30.000Z"), "id": 669938797640032256, "text": "#thanksgiving #brunch (@ Rock Island Arsenal Golf Course in Rock Island, IL) https://t.co/24FiWEndRg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.54808289,41.52101223"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thanksgiving", "brunch" }}, "user": { "id": 2421127980, "name": "Jared Robinson", "screen_name": "jrodgap", "lang": "en", "location": "Rock Island, IL", "create_at": date("2014-03-31"), "description": "null", "followers_count": 41, "friends_count": 91, "statues_count": 5550 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rock Island Arsenal, IL", "id": "8f966c3da836a196", "name": "Rock Island Arsenal", "place_type": "city", "bounding_box": rectangle("-90.570223,41.509992 -90.511303,41.523635") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17161, "countyName": "Rock Island", "cityID": 1765104, "cityName": "Rock Island Arsenal" } }
+{ "create_at": datetime("2015-11-26T10:00:30.000Z"), "id": 669938797694488576, "text": "I'm going to have a panic attack today at work lolz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1078703833, "name": "gerald.", "screen_name": "Dopee_ItsGerald", "lang": "en", "location": "xo | ", "create_at": date("2013-01-10"), "description": "I'm fifty shades of fucked up", "followers_count": 493, "friends_count": 412, "statues_count": 57677 }, "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("2015-11-26T10:00:30.000Z"), "id": 669938797937885184, "text": "Pettigrew continues to have stone hands", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 605832817, "name": "Justin Thompson", "screen_name": "juicy_justin40", "lang": "en", "location": "Chesterfield, MI", "create_at": date("2012-06-11"), "description": "All these times that I had, those will be the best memories. (Ell is perfect❤️)", "followers_count": 828, "friends_count": 668, "statues_count": 35216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tequesta, FL", "id": "e4ea9dc0fe7ff9fd", "name": "Tequesta", "place_type": "city", "bounding_box": rectangle("-80.140709,26.951196 -80.074634,27.000828") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1271525, "cityName": "Tequesta" } }
+{ "create_at": datetime("2015-11-26T10:00:30.000Z"), "id": 669938798034227200, "text": "Blessed to be getting a haircut on ThanksGivings", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3379354647, "name": "Milly Rock", "screen_name": "miltonwaynee", "lang": "en", "location": "IE/LA", "create_at": date("2015-07-16"), "description": "New Twitter", "followers_count": 252, "friends_count": 243, "statues_count": 336 }, "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("2015-11-26T10:00:30.000Z"), "id": 669938798067916800, "text": "@DougMaffia @Ermako bromance", "in_reply_to_status": 669931231417618432, "in_reply_to_user": 2275517580, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2275517580, 1029228998 }}, "user": { "id": 146831835, "name": "Antigoni Filippazzo☯", "screen_name": "AndeeFilippazzo", "lang": "en", "location": "null", "create_at": date("2010-05-22"), "description": "Hogwarts 18 ͛ ☯ ΣΔΤ 413", "followers_count": 480, "friends_count": 641, "statues_count": 21816 }, "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("2015-11-26T10:00:31.000Z"), "id": 669938798176825344, "text": "@alancornett @JamesWithers3 @LSewardKHOU Interesting! I've never had one with a lattice patterned top crust.", "in_reply_to_status": 669920602258604032, "in_reply_to_user": 23682743, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23682743, 80365738, 113674377 }}, "user": { "id": 303592223, "name": "soulfoodscholar", "screen_name": "soulfoodscholar", "lang": "en", "location": "Denver, Colorado", "create_at": date("2011-05-22"), "description": "Passion for African American, BBQ, faith & presidential foodways. Author of 2014 James Beard award-winning book on soul food.", "followers_count": 3617, "friends_count": 2950, "statues_count": 6852 }, "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-26T10:00:31.000Z"), "id": 669938798227247104, "text": "@Rholt234 you are the worst person I know", "in_reply_to_status": 669938363005329408, "in_reply_to_user": 493573405, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 493573405 }}, "user": { "id": 2322983243, "name": "Traenk", "screen_name": "taylor_traenkle", "lang": "en", "location": "null", "create_at": date("2014-02-03"), "description": "Cortland ACHA hockey", "followers_count": 610, "friends_count": 410, "statues_count": 9391 }, "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("2015-11-26T10:00:31.000Z"), "id": 669938798306787328, "text": "#TownBusiness https://t.co/UKgCkP4Xuf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "TownBusiness" }}, "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": 1875, "friends_count": 697, "statues_count": 121505 }, "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("2015-11-26T10:00:31.000Z"), "id": 669938798344712192, "text": "IM SORRY BUT THOSE VEGAN \"TURKEYS\" BE LOOKING WHACK NO THANKS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2269368372, "name": "mija", "screen_name": "lexmentz", "lang": "en", "location": "Chicago, IL", "create_at": date("2013-12-30"), "description": "Pilsen•UIC•20•Latina•Lib", "followers_count": 247, "friends_count": 261, "statues_count": 38 }, "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-26T10:00:31.000Z"), "id": 669938798604742656, "text": "thankful for them ☺️ @ North Reading High School https://t.co/RYoTl4zJj3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.08356111,42.57387778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 37013453, "name": "Hannah Osmani", "screen_name": "_hannnnnnn", "lang": "en", "location": "null", "create_at": date("2009-05-01"), "description": "take care", "followers_count": 595, "friends_count": 377, "statues_count": 11873 }, "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("2015-11-26T10:00:31.000Z"), "id": 669938798940164098, "text": "@Paiggeed @ImMadWySon paige �� so thankful for u, regardless of what relationship we have. ❤️", "in_reply_to_status": 669934381574651904, "in_reply_to_user": 62668769, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 62668769, 252671867 }}, "user": { "id": 149776258, "name": "Brent", "screen_name": "_bptx", "lang": "en", "location": "San Marcos, TX", "create_at": date("2010-05-29"), "description": "khory ross look like a 3rd ward count chocula", "followers_count": 906, "friends_count": 455, "statues_count": 18263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:31.000Z"), "id": 669938798961127424, "text": "Happy Birthday to this Idiot, hope you have a great day love you ❤️���� https://t.co/av0Z0gQ9Wy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3850612580, "name": "♏️add❣e", "screen_name": "madison_ssalas", "lang": "en", "location": "Houston, TX", "create_at": date("2015-10-10"), "description": "greys anatomy n chill", "followers_count": 270, "friends_count": 191, "statues_count": 350 }, "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("2015-11-26T10:00:31.000Z"), "id": 669938799057743872, "text": "Being rich, that shit's a talent being broke that’s a profession See I woke up un-incarcerated, that shit there is a blessing @Jeezy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.2124528,26.0301348"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28683184 }}, "user": { "id": 341259184, "name": "NL4Etwin", "screen_name": "foreignramon32", "lang": "en", "location": "null", "create_at": date("2011-07-23"), "description": "Ain't No Hoe Down...Get your MONEY in watch who you around..YFN", "followers_count": 243, "friends_count": 478, "statues_count": 575 }, "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("2015-11-26T10:00:31.000Z"), "id": 669938799116292096, "text": "who gone let me come get 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": 2850824372, "name": "Drę", "screen_name": "shoesaremyhoes", "lang": "en", "location": "null", "create_at": date("2014-10-10"), "description": "What a Time to be Alive", "followers_count": 359, "friends_count": 600, "statues_count": 4170 }, "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("2015-11-26T10:00:31.000Z"), "id": 669938799158403073, "text": "Another day in the hospital with my my mama , idc if it's thanksgiving . Cancer ain't no joke b ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 375433693, "name": "dennis", "screen_name": "dennis3brown", "lang": "en", "location": "Canada", "create_at": date("2011-09-17"), "description": "null", "followers_count": 2130, "friends_count": 982, "statues_count": 29105 }, "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("2015-11-26T10:00:31.000Z"), "id": 669938799162462208, "text": "Because I'm thankful https://t.co/x6tOvL9XI1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 327812490, "name": "Parker Graves ツ", "screen_name": "ParkGraves", "lang": "en", "location": "@wenszella 2-15-15", "create_at": date("2011-07-01"), "description": "Snapchat: parkgraves - ⚾️ - Joshua 1:9 - music connoisseur - #OU19", "followers_count": 359, "friends_count": 363, "statues_count": 911 }, "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("2015-11-26T10:00:31.000Z"), "id": 669938799326072833, "text": "�������� https://t.co/2J8GcCZ3xs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3401916898, "name": "Viliami", "screen_name": "viliami43", "lang": "en", "location": "Napa, CA", "create_at": date("2015-08-03"), "description": "Ig// Tonganmyponyy Sc// viliami43 http://www.hudl.com/athlete/6268560/highlights/306785547", "followers_count": 175, "friends_count": 150, "statues_count": 295 }, "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("2015-11-26T10:00:31.000Z"), "id": 669938799389097986, "text": "Y'all family must not be making enough to food if y'all running out", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 370302816, "name": "Rico Chavez ✌️", "screen_name": "_dejabajaaa", "lang": "en", "location": "null", "create_at": date("2011-09-08"), "description": "live a life worth living ✌️#NKU19", "followers_count": 884, "friends_count": 657, "statues_count": 49266 }, "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("2015-11-26T10:00:31.000Z"), "id": 669938799477047296, "text": "Eagles D ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1268799224, "name": "Rickyyyyyyyy", "screen_name": "GTRickyyyyy", "lang": "en", "location": "null", "create_at": date("2013-03-14"), "description": "3hunna", "followers_count": 274, "friends_count": 99, "statues_count": 5387 }, "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("2015-11-26T10:00:31.000Z"), "id": 669938799489753089, "text": "Thankful for these people in my life. Lots of love for you all. Miss ya and thanks for putting up with me lol https://t.co/5PkKmeFhp8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 468741863, "name": "pra", "screen_name": "prasamsa1", "lang": "en", "location": "null", "create_at": date("2012-01-19"), "description": "I'll make it snow, I'll make it snow,I'll make it snow, I'll make it thunderstorm \nbhs/mshs'17", "followers_count": 280, "friends_count": 281, "statues_count": 6287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marion, VA", "id": "3437938e5550bb2b", "name": "Marion", "place_type": "city", "bounding_box": rectangle("-81.537368,36.813046 -81.489631,36.859019") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51173, "countyName": "Smyth", "cityID": 5149464, "cityName": "Marion" } }
+{ "create_at": datetime("2015-11-26T10:00:31.000Z"), "id": 669938799544303616, "text": "I'm at McDonalds in Fresno, CA https://t.co/XOOF5iuttb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.69949512,36.79296858"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 88359764, "name": "Brexxie", "screen_name": "brexx_nirpaw", "lang": "en", "location": "Panorama City, Los Angeles", "create_at": date("2009-11-07"), "description": "Hi im Brexx Nirpaw the tri colored Border Collie", "followers_count": 728, "friends_count": 1447, "statues_count": 17911 }, "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("2015-11-26T10:00:31.000Z"), "id": 669938799678386176, "text": "It's too early for my daddy to be getting on my nerves. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 49793392, "name": "Nícole", "screen_name": "ImNicole_Xo", "lang": "en", "location": "Nasty Nac ", "create_at": date("2009-06-22"), "description": "SFA| Hustle Town|Astros. Rockets. Cowboys|", "followers_count": 1794, "friends_count": 1556, "statues_count": 43432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:31.000Z"), "id": 669938799833563136, "text": "What a throw there by Stafford. Excellent throw. #PHIvsDET", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PHIvsDET" }}, "user": { "id": 1677693716, "name": "JAME$", "screen_name": "James_707_cali", "lang": "en", "location": "null", "create_at": date("2013-08-17"), "description": "RaiderNation A's SFGiants Warriors College Football NASCAR Boxing $tocks Politics", "followers_count": 1102, "friends_count": 2005, "statues_count": 23501 }, "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("2015-11-26T10:00:31.000Z"), "id": 669938799892271106, "text": "Want to work at Aurora Health Care? We're #hiring in #MILWAUKEE, WI! Click for details: https://t.co/tgcepqK76O #CustomerService #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.9485796,43.1474462"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "MILWAUKEE", "CustomerService", "Job", "Jobs" }}, "user": { "id": 71948502, "name": "TMJ-MKE CstSrv Jobs", "screen_name": "tmj_mke_cstsrv", "lang": "en", "location": "Milwaukee, WI", "create_at": date("2009-09-05"), "description": "Follow this account for geo-targeted Customer Service job tweets in Milwaukee, WI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 300, "friends_count": 276, "statues_count": 96 }, "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("2015-11-26T10:00:31.000Z"), "id": 669938799946788864, "text": "the holidays are coming and i wanna be cute w/ somebody:((((", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3488919330, "name": "Sam Skrehot", "screen_name": "SkrehotSamantha", "lang": "en", "location": "null", "create_at": date("2015-09-07"), "description": "null", "followers_count": 65, "friends_count": 116, "statues_count": 285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-26T10:00:31.000Z"), "id": 669938799993077761, "text": "Reunited with my brother's from another mother I miss them allot!! �� #homiesforlife @ Brien McMahon… https://t.co/4gJBjadiIv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.43333333,41.08555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "homiesforlife" }}, "user": { "id": 392132514, "name": "Alexis Bonilla", "screen_name": "AlexRocks247", "lang": "en", "location": "Norwalk, CT", "create_at": date("2011-10-16"), "description": "Heroes are remembered, but legends never die. #203 #USA #GodBlessAmerica #YoSoyUnCatracho #honduranpride #AnimalLover #ILoveMarvelAndDc", "followers_count": 708, "friends_count": 990, "statues_count": 63453 }, "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("2015-11-26T10:00:31.000Z"), "id": 669938800127180800, "text": "Boo don't go @PrinceRoyce", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 65186457 }}, "user": { "id": 2796654948, "name": "Mrs_NancyRoyce11", "screen_name": "nancyroyce11", "lang": "en", "location": "null", "create_at": date("2014-09-07"), "description": "Married to Prince Royce09/11/15 ❤️ MET HIM 08/02/14 08/05/15 09/11/15 Day Dream✨Live Life Smiling✨ ROYCENATICA!!", "followers_count": 186, "friends_count": 145, "statues_count": 2246 }, "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-27T10:00:00.000Z"), "id": 670301056014815232, "text": "@REI Rules go outside @ICash1229 @rcash1229 @Columbia1938 @thenorthface @cjtownjc @ImSteveMigs @RTorell #OptOutside #BlackFriday", "in_reply_to_status": -1, "in_reply_to_user": 16583846, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "OptOutside", "BlackFriday" }}, "user_mentions": {{ 16583846, 66581946, 2263887031, 284155407, 7573792, 142872354, 22639627, 1241723959 }}, "user": { "id": 2586401418, "name": "Staples In Carpet", "screen_name": "Staplesincarpet", "lang": "en", "location": "Seattle, WA", "create_at": date("2014-06-24"), "description": "A punk band from the pacific northwest who also makes B movies https://staplesincarpet.bandcamp.com https://vimeo.com/tbrvideo", "followers_count": 521, "friends_count": 820, "statues_count": 1064 }, "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-27T10:00:00.000Z"), "id": 670301056434241536, "text": "time to put on this drake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2746531554, "name": "ang", "screen_name": "an98g", "lang": "en", "location": "flexas", "create_at": date("2014-08-19"), "description": "cozy girlz", "followers_count": 496, "friends_count": 349, "statues_count": 14682 }, "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-27T10:00:00.000Z"), "id": 670301056593747968, "text": "Lol I might get my bangs back ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2358212844, "name": "12/08 ♐️", "screen_name": "nikeishabee", "lang": "en", "location": "somewhere boolin", "create_at": date("2014-02-23"), "description": "don't worry about me!", "followers_count": 846, "friends_count": 617, "statues_count": 21250 }, "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("2015-11-27T10:00:00.000Z"), "id": 670301056706842624, "text": "We are thankful for #LGBT inclusive film! Go see #TheDanishGirl this weekend in LA & NYC! #prideNBCU https://t.co/jPIZQ4dkAL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LGBT", "TheDanishGirl", "prideNBCU" }}, "user": { "id": 285821005, "name": "OUT@NBCUniversal", "screen_name": "OUTNBCUniversal", "lang": "en", "location": "null", "create_at": date("2011-04-21"), "description": "OUT@NBCUniversal is the employee resource group of LGBT & Straight Ally employees at NBCUniversal. Share your pride with #prideNBCU!", "followers_count": 2709, "friends_count": 1225, "statues_count": 15006 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-27T10:00:00.000Z"), "id": 670301056753131520, "text": "BOL, i had a dream I beat ol girl up again ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.7242562,28.0463071"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1232615162, "name": "Donnnnny.", "screen_name": "_QueenDon", "lang": "en", "location": "null", "create_at": date("2013-03-01"), "description": "C/O 2O16.", "followers_count": 995, "friends_count": 387, "statues_count": 29596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winter Haven, FL", "id": "6098c1080dfc7af3", "name": "Winter Haven", "place_type": "city", "bounding_box": rectangle("-81.770135,27.939559 -81.627638,28.095052") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12105, "countyName": "Polk", "cityID": 1278275, "cityName": "Winter Haven" } }
+{ "create_at": datetime("2015-11-27T10:00:00.000Z"), "id": 670301056836853761, "text": "Check out our #listing in #Crestview #FL https://t.co/yfyT3tzAu1 #realestate #realtor https://t.co/Rr0RLkYZo3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.565811,30.776445"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "Crestview", "FL", "realestate", "realtor" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 338, "friends_count": 287, "statues_count": 6572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crestview, FL", "id": "32bb9ace0723336d", "name": "Crestview", "place_type": "city", "bounding_box": rectangle("-86.635021,30.695757 -86.534517,30.805126") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1215475, "cityName": "Crestview" } }
+{ "create_at": datetime("2015-11-27T10:00:00.000Z"), "id": 670301057126440960, "text": "$20 puffer jackets!!! #molliesboutique #blackfriday #pufferjacket @ Mollie's Boutique https://t.co/KAdk6DrPpc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.9916763,29.9498005"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "molliesboutique", "blackfriday", "pufferjacket" }}, "user": { "id": 2286942895, "name": "Mollies Boutique", "screen_name": "ShopMolliesBtq", "lang": "en", "location": "Nederland, TX ", "create_at": date("2014-01-11"), "description": "Trendy fashion boutique in the heart of Nederland, Texas. All sorts of styles to fit all sorts of events. Come and embrace yourself. https://t.co/TKUudczk4f", "followers_count": 265, "friends_count": 284, "statues_count": 3075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nederland, TX", "id": "9bc7a8ad8f4a67b6", "name": "Nederland", "place_type": "city", "bounding_box": rectangle("-94.097964,29.946424 -93.964031,30.012307") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48245, "countyName": "Jefferson", "cityID": 4850580, "cityName": "Nederland" } }
+{ "create_at": datetime("2015-11-27T10:00:00.000Z"), "id": 670301057319202816, "text": "Buffalo Chicken Sandwich! #ashlandproud #kentuckyadventures @ Butter It… https://t.co/x6dY8yAJTw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.6411819,38.4789429"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ashlandproud", "kentuckyadventures" }}, "user": { "id": 236047104, "name": "Alexandra Brian", "screen_name": "grabbaggirl", "lang": "en", "location": "null", "create_at": date("2011-01-09"), "description": "null", "followers_count": 173, "friends_count": 326, "statues_count": 4265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ashland, KY", "id": "4bb94133204a9d69", "name": "Ashland", "place_type": "city", "bounding_box": rectangle("-82.738791,38.40689 -82.605961,38.498424") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21019, "countyName": "Boyd", "cityID": 2102368, "cityName": "Ashland" } }
+{ "create_at": datetime("2015-11-27T10:00:00.000Z"), "id": 670301057646501888, "text": "Take your negativity and go as far away from me as possible ✋��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 338176069, "name": "Emma Spirito ❂", "screen_name": "EmmaSpirito", "lang": "en", "location": "Fredonia, NY", "create_at": date("2011-07-18"), "description": "585 // 716 ✈️ Sunshine Coast of Australia", "followers_count": 495, "friends_count": 421, "statues_count": 12621 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spencerport, NY", "id": "13cc7e0864aba43e", "name": "Spencerport", "place_type": "city", "bounding_box": rectangle("-77.872598,43.166327 -77.753401,43.223419") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3670189, "cityName": "Spencerport" } }
+{ "create_at": datetime("2015-11-27T10:00:00.000Z"), "id": 670301057688432640, "text": "Surely I should be getting old enough to stop crying at cartoons. #TheGoodDinosaur #pixardoesitagain", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TheGoodDinosaur", "pixardoesitagain" }}, "user": { "id": 18082510, "name": "MarkAMeeksCreative", "screen_name": "markameeks", "lang": "en", "location": "Triad of NC", "create_at": date("2008-12-12"), "description": "Graphic Artist, Creative Director, Pastor at Renaissance Road Church", "followers_count": 379, "friends_count": 499, "statues_count": 3398 }, "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("2015-11-27T10:00:00.000Z"), "id": 670301058036404224, "text": "Lmfaooooooo dude https://t.co/oEyxf3x26K", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 412416046, "name": "Drea Juarez", "screen_name": "DreaJ_", "lang": "en", "location": "IE→LA", "create_at": date("2011-11-14"), "description": "find out. USC'17. ig: dreaaaj", "followers_count": 1403, "friends_count": 564, "statues_count": 62293 }, "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-27T10:00:00.000Z"), "id": 670301058133028864, "text": "I'm at @Hooters in Washington, DC https://t.co/qWG2Q8YwpQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.02171862,38.90048101"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16012601 }}, "user": { "id": 48412599, "name": "that guy...", "screen_name": "just2smoothe", "lang": "en", "location": "DC by way of Oxon Hill", "create_at": date("2009-06-18"), "description": "computer nerd, hip hop head and lover of fine art...yea, that's me! Support local artists, especially http://klanlife.tumblr.com/ #RestInParadise Little Bro #AV", "followers_count": 111, "friends_count": 176, "statues_count": 10566 }, "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("2015-11-27T10:00:00.000Z"), "id": 670301058208366592, "text": "Bro I can't deal with this lady till Sunday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 387388301, "name": "S H A E E", "screen_name": "Trvpvrelli_", "lang": "en", "location": "null", "create_at": date("2011-10-08"), "description": "18||Employed||Studewood Texas|Fmoig:@Trvpvrelli", "followers_count": 837, "friends_count": 931, "statues_count": 7620 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-27T10:00:00.000Z"), "id": 670301058309099522, "text": "Any horny men want to fuck my tight hole raw?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.90624448,32.96182261"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2766967035, "name": "UWant2BreedMyAss?", "screen_name": "DFW_BB_BredHole", "lang": "en", "location": "Dallas, TX", "create_at": date("2014-09-11"), "description": "Horny white #teambottom #cumpig that loves getting fucked #bareback and #breedmyhole. Want to #fuckmenext ? #cumslut #DFW #GayDallas #BBBH", "followers_count": 1116, "friends_count": 2047, "statues_count": 3475 }, "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("2015-11-27T10:00:00.000Z"), "id": 670301058380369920, "text": "Tgfad����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3407791450, "name": "Humble", "screen_name": "Wes274Wes", "lang": "en", "location": "Mia✈️Atl", "create_at": date("2015-08-07"), "description": "Georgia Prep Sports Rip Shak Rip Ol Boi new Twitter", "followers_count": 96, "friends_count": 96, "statues_count": 162 }, "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-27T10:00:00.000Z"), "id": 670301058531524609, "text": "Cause people like you take it to the next level with your \"opinions\" https://t.co/vSw7bhfoEW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 214798422, "name": "King Brady", "screen_name": "KenBrady12", "lang": "en", "location": "Columbus, OH", "create_at": date("2010-11-11"), "description": "we are all a single stroke of paint that makes up this canvas of art work called life.", "followers_count": 984, "friends_count": 188, "statues_count": 77899 }, "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("2015-11-27T10:00:00.000Z"), "id": 670301058560839680, "text": "They say I'm handsome cus my bands up������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3437339356, "name": "Kobe...ひ", "screen_name": "ko_stacks", "lang": "en", "location": "Catch Me By Myself", "create_at": date("2015-08-23"), "description": "Loading...", "followers_count": 382, "friends_count": 332, "statues_count": 5614 }, "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.986584") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2416875, "cityName": "Chillum" } }
+{ "create_at": datetime("2015-11-27T10:00:00.000Z"), "id": 670301058602790913, "text": "@ThatsMy_SON I don't tho ��", "in_reply_to_status": 670300999433756672, "in_reply_to_user": 380649899, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 380649899 }}, "user": { "id": 542930060, "name": "Get_thisMoniee✨", "screen_name": "originiality", "lang": "en", "location": "null", "create_at": date("2012-04-01"), "description": "Moniee✨\nIG : kid.sss\nArmy", "followers_count": 670, "friends_count": 640, "statues_count": 2732 }, "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("2015-11-27T10:00:00.000Z"), "id": 670301058657202176, "text": "woke with ah headache . ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.9028572,31.9624839"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 575613052, "name": "dee el w [bee]❤", "screen_name": "btchimmmRACKDUP", "lang": "en", "location": "null", "create_at": date("2012-05-09"), "description": "FMOi: @likedontlurk SC: d_dizzleeee taken by: AB ✨ mrs . brandon 2 you", "followers_count": 1120, "friends_count": 1082, "statues_count": 20168 }, "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": 28021, "countyName": "Claiborne" } }
+{ "create_at": datetime("2015-11-27T10:00:00.000Z"), "id": 670301059001278464, "text": "Bring out the walking canes cause Karly Shittington is officially a… https://t.co/BZOln8SIyX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.56349755,34.36400217"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 446471129, "name": "Sarah Ozzello", "screen_name": "SarahOzzello", "lang": "en", "location": "null", "create_at": date("2011-12-25"), "description": "taco bell • ole miss • ΑΔΠ", "followers_count": 657, "friends_count": 351, "statues_count": 5332 }, "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("2015-11-27T10:00:00.000Z"), "id": 670301059110346752, "text": "My heart �������� https://t.co/pZXxvCwk1P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 237953287, "name": "Muva Orchid Bud", "screen_name": "kjthegawd", "lang": "en", "location": "DMV ", "create_at": date("2011-01-13"), "description": "Hopeless hip-hop romantic ☯ Sometimes I'm Mufasa Sometimes I'm Scar♌️", "followers_count": 2178, "friends_count": 453, "statues_count": 109373 }, "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("2015-11-27T10:00:00.000Z"), "id": 670301059278073857, "text": "I wonder what my \"starter pack\" would look like. Hah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2865333104, "name": "Cassidy Clark", "screen_name": "_CassClark10_", "lang": "en", "location": "null", "create_at": date("2014-10-19"), "description": "null", "followers_count": 427, "friends_count": 379, "statues_count": 3556 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenville, OH", "id": "1707e796e8dd6496", "name": "Greenville", "place_type": "city", "bounding_box": rectangle("-84.647525,40.074038 -84.595954,40.136045") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39037, "countyName": "Darke", "cityID": 3932340, "cityName": "Greenville" } }
+{ "create_at": datetime("2015-11-27T10:00:00.000Z"), "id": 670301059563171841, "text": "Damn wtf my parents stay up all night lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2846151555, "name": "Luisss.", "screen_name": "Luisss7777", "lang": "en", "location": "null", "create_at": date("2014-10-26"), "description": "97 Till Infinity | Promoter Of @WonderCelestial |", "followers_count": 383, "friends_count": 345, "statues_count": 8303 }, "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("2015-11-27T10:00:00.000Z"), "id": 670301059634429952, "text": "������ @ him & tell him happy birthday �� https://t.co/ha6neN2Rn1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2971174081, "name": "Niecccy✨", "screen_name": "niecccycouture_", "lang": "en", "location": "null", "create_at": date("2015-01-09"), "description": "Houston . ⚽️", "followers_count": 1687, "friends_count": 1191, "statues_count": 25917 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-27T10:00:00.000Z"), "id": 670301059752042496, "text": "nike>>>", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2300537055, "name": "kendall", "screen_name": "kensharrell", "lang": "en", "location": "null", "create_at": date("2014-01-23"), "description": "ATL SENIOR", "followers_count": 323, "friends_count": 232, "statues_count": 3815 }, "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-27T10:00:00.000Z"), "id": 670301059873566721, "text": "Check out our #listing in #Crestview #FL https://t.co/8F4laGgX2e #realestate #realtor https://t.co/pECTNLQ7QA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.571611,30.808025"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "Crestview", "FL", "realestate", "realtor" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 338, "friends_count": 287, "statues_count": 6573 }, "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": 12091, "countyName": "Okaloosa" } }
+{ "create_at": datetime("2015-11-27T10:00:00.000Z"), "id": 670301060037120000, "text": "@_UNISecrets I'm looking for a subleaser for my apartment at UV mills for the summer! If anybody is interested please let me know!", "in_reply_to_status": -1, "in_reply_to_user": 2290404470, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2290404470 }}, "user": { "id": 321666534, "name": "Brett From The Trap", "screen_name": "BrettGable_22", "lang": "en", "location": "A Trap House", "create_at": date("2011-06-21"), "description": "|UNI| friend, brother, footballer, husband to @brianajo11, wanna dab?", "followers_count": 1067, "friends_count": 521, "statues_count": 10029 }, "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("2015-11-27T10:00:00.000Z"), "id": 670301060163084289, "text": "why does this never happen when i go to store openings on black friday������ https://t.co/5qhZ5hBXTN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1023486932, "name": "aly(:", "screen_name": "alyrusso_", "lang": "en", "location": "null", "create_at": date("2012-12-19"), "description": "FL", "followers_count": 411, "friends_count": 506, "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("2015-11-27T10:00:01.000Z"), "id": 670301060427329536, "text": "Someone really really wants a Christmas gift", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3542977634, "name": "Victoria Acevedo", "screen_name": "Victwerkiaa", "lang": "en", "location": "null", "create_at": date("2015-09-12"), "description": "truuuuu", "followers_count": 191, "friends_count": 210, "statues_count": 791 }, "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-27T10:00:01.000Z"), "id": 670301060557373441, "text": "burning off all that turkey & pie with an intense workout @TrainAtElevate! #betterforit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "betterforit" }}, "user_mentions": {{ 2788422038 }}, "user": { "id": 36370898, "name": "Taylor Cylke", "screen_name": "taylorcylke", "lang": "en", "location": "Washington, DC", "create_at": date("2009-04-29"), "description": "You only live once, but if you do it right, once is enough.", "followers_count": 510, "friends_count": 655, "statues_count": 5743 }, "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("2015-11-27T10:00:01.000Z"), "id": 670301060620140544, "text": "@_jackiekiller thank ya ��", "in_reply_to_status": 670295921159094272, "in_reply_to_user": 1047130412, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1047130412 }}, "user": { "id": 350684339, "name": "madi", "screen_name": "madiirenaee", "lang": "en", "location": "glenpool | tahlequah", "create_at": date("2011-08-07"), "description": "null", "followers_count": 813, "friends_count": 362, "statues_count": 7761 }, "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("2015-11-27T10:00:01.000Z"), "id": 670301060666273793, "text": "Capoue vs Sanchez pre-match H2H: One betters other in six key areas https://t.co/Nv3t6MKe4z via @HITCsport", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user_mentions": {{ 323186108 }}, "user": { "id": 119195187, "name": "Ken Plows", "screen_name": "peelows", "lang": "en", "location": "null", "create_at": date("2010-03-02"), "description": "null", "followers_count": 916, "friends_count": 2120, "statues_count": 35520 }, "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("2015-11-27T10:00:01.000Z"), "id": 670301060687204352, "text": "@amazinvids @nolanhansen49 @Braden_Hansen44 @jakeh2298 @itaIianoo", "in_reply_to_status": 664206406015291392, "in_reply_to_user": 2493427621, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2493427621, 898946870, 753565261, 2790066699, 2576526964 }}, "user": { "id": 1491514616, "name": "glen hansen", "screen_name": "ghansen921", "lang": "en", "location": "Minnesota, USA", "create_at": date("2013-06-07"), "description": "null", "followers_count": 716, "friends_count": 2042, "statues_count": 10493 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Princeton, MN", "id": "2d99f3e65f91edd5", "name": "Princeton", "place_type": "city", "bounding_box": rectangle("-93.604276,45.537389 -93.555182,45.594597") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27095, "countyName": "Mille Lacs", "cityID": 2752522, "cityName": "Princeton" } }
+{ "create_at": datetime("2015-11-27T10:00:01.000Z"), "id": 670301060855123970, "text": "My new favorite #autumn photograph!\n\n#family #portraits ##FamilyPortrait #BrooklynFamily… https://t.co/GKbT9VFxID", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.97202415,40.66275966"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "autumn", "family", "portraits", "FamilyPortrait", "BrooklynFamily" }}, "user": { "id": 1944596833, "name": "JP Fernandez", "screen_name": "JPFernandez45", "lang": "en", "location": "Brooklyn, New York", "create_at": date("2013-10-07"), "description": "Special Educator, Art Teacher, Photographer, Food Enthusiast, Dog-Lover", "followers_count": 26, "friends_count": 104, "statues_count": 39 }, "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-27T10:00:01.000Z"), "id": 670301060980977664, "text": "@greyskydj We and @SimplyHServants would love to see you midnight Dec 5th for our next Rocky show! https://t.co/caPlTbq9Tu", "in_reply_to_status": 669992032929226752, "in_reply_to_user": 918748969, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 918748969, 1080698203 }}, "user": { "id": 20526745, "name": "Cedar Lee Theatre", "screen_name": "CedarLeeTheatre", "lang": "en", "location": "Cleveland Heights, Ohio", "create_at": date("2009-02-10"), "description": "The Cedar Lee Theatre is Northeast Ohio's premier alternative film venue showing the best indie and foreign films.", "followers_count": 6942, "friends_count": 3791, "statues_count": 22654 }, "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("2015-11-27T10:00:01.000Z"), "id": 670301061026975746, "text": "7 hours Red Nation.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 392955714, "name": "Garvin B Lawless", "screen_name": "DeuceLawless", "lang": "en", "location": "Skiatook", "create_at": date("2011-10-17"), "description": "Living the Perfect Life. The Joker. Red Nation. USA. 2-21", "followers_count": 567, "friends_count": 513, "statues_count": 28125 }, "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("2015-11-27T10:00:01.000Z"), "id": 670301061052174336, "text": "good afternoon mentions? RT/FAV ����✨", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1959579655, "name": "♡☁ 19.8k ♡☁", "screen_name": "itscrewsworldTV", "lang": "en", "location": "Houston, TX", "create_at": date("2013-10-13"), "description": "17 | recording artist | song-writer | actress | promoter | model | booking-business inquires contact: crewloveeemusic2@gmail.com | #TheCrewTeam | #RIPRikoV ♡♕.", "followers_count": 19851, "friends_count": 8801, "statues_count": 111051 }, "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("2015-11-27T10:00:01.000Z"), "id": 670301061253468160, "text": "If you're not mixing sour cream into your #macncheese you're not living life.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "macncheese" }}, "user": { "id": 2990141962, "name": "Jonathan Perrin", "screen_name": "jdzgnx", "lang": "en", "location": "Hammond, La. USA", "create_at": date("2015-01-21"), "description": "Webmaster for http://mylpl.info Livingston Parish Library System in Louisiana.", "followers_count": 50, "friends_count": 68, "statues_count": 515 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-27T10:00:01.000Z"), "id": 670301061291368449, "text": "https://t.co/LfjwDpDOi6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 74933406, "name": "Birthday Boy", "screen_name": "TheHaloCarter", "lang": "en", "location": "Atlanta, Georgia.", "create_at": date("2009-09-16"), "description": "The Profile About Nothing .", "followers_count": 3206, "friends_count": 988, "statues_count": 244162 }, "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("2015-11-27T10:00:01.000Z"), "id": 670301061396234240, "text": "@azblonde2015 @worldisahoax @misterdish69 @CindyBlackwel12 @BarbMuenchen @Carolde @TheTRUMPetts HAPPY HOLIDAYS https://t.co/TnYFXbduMZ", "in_reply_to_status": 670299815079579648, "in_reply_to_user": 3271255423, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3271255423, 482654644, 2953865988, 2371155744, 3124986791, 22936237, 3701618555 }}, "user": { "id": 2985114214, "name": "MIZ MARQUITA.*1", "screen_name": "MarquitaChisho1", "lang": "en", "location": "Eastside ", "create_at": date("2015-01-18"), "description": "Professional Thinker.Poet. Writer. Lyricis, Artist,always a lover never a fighter.Voice of the people! TEAM TRUMP!!!!! #TRUMP2016", "followers_count": 1680, "friends_count": 2339, "statues_count": 15103 }, "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("2015-11-27T10:00:01.000Z"), "id": 670301061433925632, "text": ".@PainterRandy Amazing how U LOVE Jesus & care so little for ppl around U. Do U think Jesus can return THAT kind of love? U said it!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 59673815 }}, "user": { "id": 18085565, "name": "Kleyton Cooper", "screen_name": "kleytoncooper", "lang": "en", "location": "Cleveland, OH", "create_at": date("2008-12-12"), "description": "Politically-left satire! Mech-Engr. Int'l Off-Shore Oil Exploration; Ind Design & Mfr. Foreign Service. Bread baking; auto/sail boat racing. Naturalized US cit.", "followers_count": 1479, "friends_count": 1184, "statues_count": 73939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rocky River, OH", "id": "ad1fd8a17d633e77", "name": "Rocky River", "place_type": "city", "bounding_box": rectangle("-81.874568,41.449742 -81.823152,41.490668") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3968056, "cityName": "Rocky River" } }
+{ "create_at": datetime("2015-11-27T10:00:01.000Z"), "id": 670301061521895426, "text": "I wish it was raining and foggy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 524844037, "name": "єммα ॐ", "screen_name": "emmirreee", "lang": "en", "location": "null", "create_at": date("2012-03-14"), "description": "future MA + phlebotomist", "followers_count": 612, "friends_count": 125, "statues_count": 6682 }, "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("2015-11-27T10:00:01.000Z"), "id": 670301061702287360, "text": "@Lazlovely @Emma_black_cat @BarbaraBpiwetz @FionaKenndal @Mr_Pie Nope. And how much \"stuff\" does u really need?", "in_reply_to_status": 670294256406437888, "in_reply_to_user": 2364568038, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2364568038, 232372624, 1020050376, 2376693973, 219273242 }}, "user": { "id": 496905664, "name": "Linda & Chow", "screen_name": "LBisaillon", "lang": "en", "location": "Casas Adobes, AZ", "create_at": date("2012-02-19"), "description": "null", "followers_count": 946, "friends_count": 659, "statues_count": 37501 }, "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("2015-11-27T10:00:01.000Z"), "id": 670301061731610624, "text": "Temp: 14.2°F\nGust: 7.2mph\nGust Direction: S\nCloud Base: 7690 ft\nPressure: 29.72inHg\nRainfall Rate: 0.00in/hr\nRain Today: 0.00in", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.67972222,38.865"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3232161168, "name": "KCOCOLOR305", "screen_name": "kcocolor305", "lang": "en", "location": "Colorado Springs, CO", "create_at": date("2015-05-31"), "description": "Personal Weather Station.", "followers_count": 10, "friends_count": 10, "statues_count": 13742 }, "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": 8041, "countyName": "El Paso", "cityID": 816000, "cityName": "Colorado Springs" } }
+{ "create_at": datetime("2015-11-27T10:00:01.000Z"), "id": 670301061953925120, "text": "Good morning, Honokaa, HI. #UnitedStates https://t.co/75qPZIJUSN https://t.co/HsUbEyRLcy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-155.4641724,20.07749939"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "UnitedStates" }}, "user": { "id": 17111948, "name": "BigZ", "screen_name": "ioDracy", "lang": "zh-cn", "location": "HangZhou", "create_at": date("2008-11-02"), "description": "@Hangzhou China,Android user,Google fan,Geeker,Blogger", "followers_count": 211, "friends_count": 223, "statues_count": 3803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honokaa, HI", "id": "f9d51873910c1a97", "name": "Honokaa", "place_type": "city", "bounding_box": rectangle("-155.478936,20.063895 -155.435056,20.094307") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15001, "countyName": "Hawaii", "cityID": 1516450, "cityName": "Honokaa" } }
+{ "create_at": datetime("2015-11-27T10:00:01.000Z"), "id": 670301062004387840, "text": "Wind 1.3 mph SSW. Barometer 30.41 in, Steady. Temperature 64.8 °F. Rain today 0.00 in. Humidity 57%", "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": 80, "friends_count": 16, "statues_count": 300945 }, "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("2015-11-27T10:00:01.000Z"), "id": 670301062243438592, "text": "Wishing I was cuddled up to you rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 302891577, "name": "kelsey_penland96", "screen_name": "Kelso205", "lang": "en", "location": "null", "create_at": date("2011-05-21"), "description": "null", "followers_count": 82, "friends_count": 422, "statues_count": 492 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mauldin, SC", "id": "efc250826c29bfea", "name": "Mauldin", "place_type": "city", "bounding_box": rectangle("-82.333778,34.741098 -82.255239,34.826378") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45045, "countyName": "Greenville", "cityID": 4545115, "cityName": "Mauldin" } }
+{ "create_at": datetime("2015-11-27T10:00:01.000Z"), "id": 670301062767644672, "text": "\"Diggin up them holes diggin diggin up them holes oh\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 937859940, "name": "lonewolfshell", "screen_name": "ShellronG", "lang": "en", "location": "null", "create_at": date("2012-11-09"), "description": "Fhs15' 19, Celtics, snapchat: shellron75", "followers_count": 302, "friends_count": 601, "statues_count": 5724 }, "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("2015-11-27T10:00:01.000Z"), "id": 670301063011041280, "text": "Wind 2.9 mph S. Barometer 30.31 in, Falling. Temperature 73.0 °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": 3, "statues_count": 19182 }, "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("2015-11-27T10:00:01.000Z"), "id": 670301063069704193, "text": "You know how this shit goes. \nThis is not four years ago. \nTime escapes me. \nNow forget how it felt when this shit move slow.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 635601379, "name": "Jordan White", "screen_name": "etihWnadroJ", "lang": "en", "location": "null", "create_at": date("2012-07-14"), "description": "Looking forward to the memories of right now.", "followers_count": 217, "friends_count": 206, "statues_count": 3100 }, "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("2015-11-27T10:00:01.000Z"), "id": 670301063229018112, "text": "Wind 0.1 mph WNW. Barometer 30.691 in, Falling slowly. Temperature 17.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.98388889,47.66361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 67603939, "name": "Bemidji Weather", "screen_name": "BemidjiWx", "lang": "en", "location": "Bemidji, MN, USA", "create_at": date("2009-08-21"), "description": "Local Bemidji Weather", "followers_count": 278, "friends_count": 136, "statues_count": 94800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minnesota, USA", "id": "9807c5c5f7a2c6ce", "name": "Minnesota", "place_type": "admin", "bounding_box": rectangle("-97.239256,43.499362 -89.483385,49.384359") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27007, "countyName": "Beltrami" } }
+{ "create_at": datetime("2015-11-27T10:00:01.000Z"), "id": 670301063325421570, "text": "Toni Davis would love to show you the #listing at 203 LADUE Ave #Crestview #FL https://t.co/0Smd3Ey233 #realestate https://t.co/XmipRlqZlt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.563065,30.731449"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "Crestview", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 338, "friends_count": 287, "statues_count": 6574 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crestview, FL", "id": "32bb9ace0723336d", "name": "Crestview", "place_type": "city", "bounding_box": rectangle("-86.635021,30.695757 -86.534517,30.805126") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1215475, "cityName": "Crestview" } }
+{ "create_at": datetime("2015-11-27T10:00:01.000Z"), "id": 670301063350779904, "text": "Wind 5.0 mph SW. Barometer 30.338 in, Falling. Temperature 65.5 °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": 39, "friends_count": 25, "statues_count": 15712 }, "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("2015-11-27T10:00:01.000Z"), "id": 670301063946174464, "text": "From Last Night's Dinner! God bless the wonderful people who welcomed me for dinner ❤️… https://t.co/0RlqG0c0dZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.898,33.6167"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3117340782, "name": "Manna Dabholkar", "screen_name": "mannadabholkar", "lang": "en", "location": "null", "create_at": date("2015-03-30"), "description": "null", "followers_count": 26, "friends_count": 50, "statues_count": 311 }, "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("2015-11-27T10:00:01.000Z"), "id": 670301064340590592, "text": "you deserve better than me..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 278724610, "name": "Ⓜ️UVA", "screen_name": "HoesloveLexus", "lang": "en", "location": "the plug gf㊗️", "create_at": date("2011-04-07"), "description": "only thing popping ..|| dont dm me", "followers_count": 2052, "friends_count": 1255, "statues_count": 24170 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Camp Springs, MD", "id": "4b656a23df9bee4a", "name": "Camp Springs", "place_type": "city", "bounding_box": rectangle("-76.945472,38.768037 -76.890929,38.835708") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2412600, "cityName": "Camp Springs" } }
+{ "create_at": datetime("2015-11-27T10:00:02.000Z"), "id": 670301064537759746, "text": "11/27@13:00 - Temp 63.8F, WC 63.8F. Wind 4.7mph WSW, Gust 10.0mph. Bar 30.460in, Falling. Rain 0.00in. Hum 61%. UV 1.9. SolarRad 360.", "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": 45117 }, "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("2015-11-27T10:00:02.000Z"), "id": 670301064646623232, "text": "@BetYouWontAtMe i was trying to get him the full kit but they only had the revolvers left", "in_reply_to_status": 670300922560446464, "in_reply_to_user": 625280025, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 625280025 }}, "user": { "id": 357198424, "name": "brittneysun", "screen_name": "ArkanSass", "lang": "en", "location": "Texas ", "create_at": date("2011-08-17"), "description": "I belong to Ben. WOO PIG, YALL. WOO PIG. Anya and Archer are my husky fur babies.", "followers_count": 1024, "friends_count": 659, "statues_count": 40727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rogers, AR", "id": "a9e535bc5a50aba9", "name": "Rogers", "place_type": "city", "bounding_box": rectangle("-94.225208,36.265124 -94.070837,36.406101") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton", "cityID": 560410, "cityName": "Rogers" } }
+{ "create_at": datetime("2015-11-27T10:00:02.000Z"), "id": 670301065456300032, "text": "Temp: 81.9°F | Humidity: 45% | Wind: NNE @ 0.7 mph | Barometer: 30.22 in | Dewpoint: 58.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": 24, "friends_count": 1, "statues_count": 163345 }, "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("2015-11-27T10:00:02.000Z"), "id": 670301065749884928, "text": "@PatrikIanPolk @elonjames 100% THIS.", "in_reply_to_status": 670300694721744898, "in_reply_to_user": 23146830, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23146830, 1716581 }}, "user": { "id": 131201772, "name": "Brandon B.", "screen_name": "excuseyou77", "lang": "en", "location": "null", "create_at": date("2010-04-09"), "description": "30. Slash addict. Total multishipper, Kylie worshipper, Doctor Who & SPN fandude, game show admirer, soap lover, classic TV fan. I play by play everything.", "followers_count": 1353, "friends_count": 2027, "statues_count": 142197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Avondale Estates, GA", "id": "599785cb8cd0515d", "name": "Avondale Estates", "place_type": "city", "bounding_box": rectangle("-84.285727,33.754915 -84.250519,33.780942") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1304644, "cityName": "Avondale Estates" } }
+{ "create_at": datetime("2015-11-27T10:00:02.000Z"), "id": 670301065821167616, "text": "What's the point of having a sister if you can't trade pants at sporting events? @kenzzzzz13", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 449195694 }}, "user": { "id": 474372714, "name": "ang", "screen_name": "ang_marie15", "lang": "en", "location": "null", "create_at": date("2012-01-25"), "description": "mind over matter", "followers_count": 541, "friends_count": 525, "statues_count": 7061 }, "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("2015-11-27T10:00:02.000Z"), "id": 670301066001432576, "text": "Temp: 73.8°F Wind:2.2mph 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": 22, "statues_count": 52160 }, "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("2015-11-27T10:00:02.000Z"), "id": 670301066345377792, "text": "left hand is steerin, the other is grippin ya thigh\n\nlight up a spliff n 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": 162142703, "name": "‍ ‍ ☪ ‍ ‍", "screen_name": "ActualPharaoh", "lang": "en", "location": "Bompton, CA", "create_at": date("2010-07-02"), "description": "kentucky • backwoods • balmain • rick owens• maison margiela || #SouthernShit #GetHighRideLow || ATL-HOU-LVILLE-DVILLE-HBURG-MEM || B's & L's up #slime", "followers_count": 19019, "friends_count": 12933, "statues_count": 16612 }, "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("2015-11-27T10:00:02.000Z"), "id": 670301066441854976, "text": "How many blankets do you have \n\"I DONT CARE\"\n����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2929235766, "name": "Ana Rodriguez", "screen_name": "anyyy__", "lang": "en", "location": "null", "create_at": date("2014-12-13"), "description": "null", "followers_count": 301, "friends_count": 285, "statues_count": 7910 }, "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-27T10:00:02.000Z"), "id": 670301066831892481, "text": "#AHIP lobbyists pressuring #Congress to do another bailout unto #insurance cronies.\nLink: https://t.co/RRrj8FeWUc\nKey Word: #ACA\nHT @kksheld", "in_reply_to_status": 668081676795318272, "in_reply_to_user": 418830106, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AHIP", "Congress", "insurance", "ACA" }}, "user_mentions": {{ 418830106 }}, "user": { "id": 309558781, "name": "Kelly Rek", "screen_name": "KellyRek", "lang": "en", "location": "Arizona", "create_at": date("2011-06-02"), "description": "I am a passionate believer in Constitutional Rights.", "followers_count": 3085, "friends_count": 3422, "statues_count": 127819 }, "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("2015-11-27T10:00:02.000Z"), "id": 670301066957856768, "text": "Post-Black Friday shopping adventure to buy a new refrigerator (@ Gourmet Cafe & Pie Company in Los Alamitos, CA) https://t.co/kgNnTxRhYP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.03943539,33.80271133"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 6893212, "name": "Tony Nix", "screen_name": "tonynix", "lang": "en", "location": "SoCal", "create_at": date("2007-06-18"), "description": "One of 10 native Californians still living in California!", "followers_count": 69, "friends_count": 214, "statues_count": 1762 }, "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("2015-11-27T10:00:02.000Z"), "id": 670301067096166400, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 358625728, "name": "CAM❗️", "screen_name": "ORIGINAL_CAMRY", "lang": "en", "location": "southwest applecity alief", "create_at": date("2011-08-19"), "description": "null", "followers_count": 467, "friends_count": 415, "statues_count": 13512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-27T10:00:02.000Z"), "id": 670301067188408320, "text": "See a virtual tour of our listing on 107 CREVE CORE Drive #Crestview #FL https://t.co/ynEo44BrdY #realestate https://t.co/xL5aWSEuui", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.563006,30.732094"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Crestview", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 338, "friends_count": 287, "statues_count": 6575 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crestview, FL", "id": "32bb9ace0723336d", "name": "Crestview", "place_type": "city", "bounding_box": rectangle("-86.635021,30.695757 -86.534517,30.805126") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1215475, "cityName": "Crestview" } }
+{ "create_at": datetime("2015-11-27T10:00:02.000Z"), "id": 670301067297603584, "text": "@jenamiller killing it at TopGolf @ Topgolf Atlanta https://t.co/hz19c80il7", "in_reply_to_status": -1, "in_reply_to_user": 28667925, "favorite_count": 0, "coordinate": point("-84.427124,33.8002548"), "retweet_count": 0, "lang": "nl", "is_retweet": false, "user_mentions": {{ 28667925 }}, "user": { "id": 17351708, "name": "KendrickDisch", "screen_name": "KendrickDisch", "lang": "en", "location": "Atlanta, GA", "create_at": date("2008-11-12"), "description": "Atlanta Corporate Photographer and Video Director, working with companies to create compelling visual stories. [founder of @workshoppers]", "followers_count": 613, "friends_count": 1164, "statues_count": 4357 }, "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("2015-11-27T10:00:02.000Z"), "id": 670301067565928450, "text": "https://t.co/GkNLXxvWjG via YouTube Ron Swanson's laugh. I love @Nick_Offerman", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 502135911 }}, "user": { "id": 26151974, "name": "les", "screen_name": "les_ducote", "lang": "en", "location": "null", "create_at": date("2009-03-23"), "description": "Give me all the bacon & eggs you have.", "followers_count": 11, "friends_count": 108, "statues_count": 4251 }, "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("2015-11-27T10:00:02.000Z"), "id": 670301067570204672, "text": "Wind 2.0 mph NW. Barometer 30.274 in, Steady. Temperature 43.7 °F. Rain today 0.73 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": 116, "friends_count": 0, "statues_count": 109172 }, "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("2015-11-27T10:00:02.000Z"), "id": 670301067670876160, "text": "Everyday is thanksgiving \n#WhenYouLiveLikeThis\n\nGrateful Everyday | not just during the Holidays \n\nA… https://t.co/XydaOOVYgY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.68972222,41.49888889"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WhenYouLiveLikeThis" }}, "user": { "id": 35713609, "name": "Khalil Táriq", "screen_name": "KhalilTCald", "lang": "en", "location": "Cosmos", "create_at": date("2009-04-27"), "description": "I Am ...", "followers_count": 581, "friends_count": 297, "statues_count": 7782 }, "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("2015-11-27T10:00:02.000Z"), "id": 670301067888848896, "text": "Ian Never Gonna Learn To Keep My Phone Ina Case �� .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 459932578, "name": "Bond , James ⚡️", "screen_name": "CashFloww___", "lang": "en", "location": "null", "create_at": date("2012-01-09"), "description": "ᶜᴴᴬᴺᴳᴱ ᴵˢ ᴾᴼˢ ˢ ᴵᴮᴸᴱ", "followers_count": 1144, "friends_count": 977, "statues_count": 55513 }, "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("2015-11-27T10:00:02.000Z"), "id": 670301068094529536, "text": "@camerondallas ayo Mr. Dallas! Where are the @MAGCONTOUR dates???? I'm waiting", "in_reply_to_status": -1, "in_reply_to_user": 405728790, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 405728790, 1949322061 }}, "user": { "id": 713359458, "name": "fangirl | 102 days", "screen_name": "totally_kelsey", "lang": "en", "location": "at a concert probably ", "create_at": date("2012-07-23"), "description": "Janoksianator 4 years| they say we're losers but we're alright with that", "followers_count": 1509, "friends_count": 1886, "statues_count": 25352 }, "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("2015-11-27T10:00:02.000Z"), "id": 670301068169908225, "text": "If I do shop it's gone be on 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": 1602238572, "name": "Nia Ellen", "screen_name": "_Ihateevrybody", "lang": "en", "location": "California, USA", "create_at": date("2013-07-17"), "description": "Nothing worth it comes easy #LongliveMeech", "followers_count": 257, "friends_count": 279, "statues_count": 5345 }, "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("2015-11-27T10:00:03.000Z"), "id": 670301068765429760, "text": "Bra and panties matching nails and toes ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1697420203, "name": "♛ Hanna", "screen_name": "_hannnnnaaa", "lang": "en", "location": "Ponchatoula, LA", "create_at": date("2013-08-24"), "description": "snapchat : @hansim97 ❤️", "followers_count": 421, "friends_count": 257, "statues_count": 17017 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mandeville, LA", "id": "003d28304705c42a", "name": "Mandeville", "place_type": "city", "bounding_box": rectangle("-90.154712,30.348341 -90.003767,30.459998") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22103, "countyName": "St. Tammany", "cityID": 2248225, "cityName": "Mandeville" } }
+{ "create_at": datetime("2015-11-27T10:00:03.000Z"), "id": 670301069088436225, "text": "I dislike the cold ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 151502751, "name": "PARKER™", "screen_name": "whatif31", "lang": "en", "location": "Dallas,Texas ", "create_at": date("2010-06-03"), "description": "#SFA |Dallas Nac| Management/Criminal Justice Live your life and forget your age.", "followers_count": 542, "friends_count": 1076, "statues_count": 37941 }, "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-27T10:00:03.000Z"), "id": 670301069138751493, "text": "Another Black Friday in the books @bebe_stores !! #tistheseason #tobenaughty #bebe #bebegirl #ootd… https://t.co/HyMHFEvHr2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.39108795,40.08728093"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "tistheseason", "tobenaughty", "bebe", "bebegirl", "ootd" }}, "user_mentions": {{ 22995171 }}, "user": { "id": 299810900, "name": "Anita Grossberg", "screen_name": "anita_gberg", "lang": "en", "location": "null", "create_at": date("2011-05-16"), "description": "I'm on that champagne campaign.", "followers_count": 277, "friends_count": 574, "statues_count": 4067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "King of Prussia, PA", "id": "23b12afb6fe7507a", "name": "King of Prussia", "place_type": "city", "bounding_box": rectangle("-75.420016,40.071936 -75.343479,40.117633") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42091, "countyName": "Montgomery", "cityID": 4239736, "cityName": "King of Prussia" } }
+{ "create_at": datetime("2015-11-27T10:00:03.000Z"), "id": 670301069272940544, "text": "Wind 0.0 mph WSW. Barometer 30.369 in, Rising. Temperature 45.8 °F. Rain today 0.00 in. Humidity 54%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-106.69694444,34.80694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14656738, "name": "Andrew Madden", "screen_name": "AbqDrew", "lang": "en", "location": "Los Lunas, New Mexico", "create_at": date("2008-05-04"), "description": "31 year old atheist conservative living in Albuquerque, New Mexico. KCCO", "followers_count": 173, "friends_count": 185, "statues_count": 4910 }, "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("2015-11-27T10:00:03.000Z"), "id": 670301069365284865, "text": "Homework time but first, let me take a nap ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 256026293, "name": "A$AP Stubbs", "screen_name": "TeamQStubbs58", "lang": "en", "location": "Brownsburg, IN ✈️ Hanover, IN", "create_at": date("2011-02-22"), "description": "Hanover '17 #PantherPride #LegacyInTheMaking ΦΔΘ #HitSquad", "followers_count": 799, "friends_count": 761, "statues_count": 33057 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsburg, IN", "id": "00f424d474454ea0", "name": "Brownsburg", "place_type": "city", "bounding_box": rectangle("-86.439701,39.800082 -86.326794,39.912651") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18063, "countyName": "Hendricks", "cityID": 1808416, "cityName": "Brownsburg" } }
+{ "create_at": datetime("2015-11-27T10:00:03.000Z"), "id": 670301069449297920, "text": "@gd21z happy birthday woe ��", "in_reply_to_status": -1, "in_reply_to_user": 2832262879, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2832262879 }}, "user": { "id": 1515125971, "name": "gym", "screen_name": "thejimlantzz4", "lang": "en", "location": "SA❤️", "create_at": date("2013-06-13"), "description": "null", "followers_count": 473, "friends_count": 420, "statues_count": 17177 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Easton, MD", "id": "870410c07f185520", "name": "Easton", "place_type": "city", "bounding_box": rectangle("-76.108363,38.719728 -76.031236,38.820113") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24041, "countyName": "Talbot", "cityID": 2424475, "cityName": "Easton" } }
+{ "create_at": datetime("2015-11-27T10:00:03.000Z"), "id": 670301069856088064, "text": "No saben las ofertas que hay en Victoria's Secret!!!! https://t.co/jn6lVTkIKl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 64263845, "name": "Eowyn", "screen_name": "cams_escallon", "lang": "en", "location": "Neverland", "create_at": date("2009-08-09"), "description": "You think you are going mad so you came to see me to see what a mad person looks like.", "followers_count": 393, "friends_count": 305, "statues_count": 9000 }, "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("2015-11-27T10:00:03.000Z"), "id": 670301070346727424, "text": "@TiaLowkeyDoe ��", "in_reply_to_status": 670300982551547904, "in_reply_to_user": 799427671, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 799427671 }}, "user": { "id": 1546715839, "name": "ari ♈️", "screen_name": "AriTheTrillest", "lang": "en", "location": "409 , Tx", "create_at": date("2013-06-25"), "description": "18 • single • rest easy mommy❤️ • [Lesbian)GBT/ [Femme⚢) • meme was here ❤️", "followers_count": 1119, "friends_count": 773, "statues_count": 62762 }, "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("2015-11-27T10:00:03.000Z"), "id": 670301070363635712, "text": "@mike_is_bored I knew it!", "in_reply_to_status": 670300664988323840, "in_reply_to_user": 162549547, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 162549547 }}, "user": { "id": 191265367, "name": "SkitchP", "screen_name": "skitchP", "lang": "en", "location": "Livonia Michigan", "create_at": date("2010-09-15"), "description": "Great, now what do I have as my twitter bio, PCB saying I'm a weird guy skitch or Iott saying I'm a genius? - @skitchP", "followers_count": 802, "friends_count": 195, "statues_count": 46680 }, "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("2015-11-27T10:00:03.000Z"), "id": 670301070636118017, "text": "Happy Black Friday ❤️❤️ https://t.co/NuE9uNfGoE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 185038458, "name": "The Cat Lady", "screen_name": "klpratt1973", "lang": "en", "location": "United States", "create_at": date("2010-08-30"), "description": "Go Sports", "followers_count": 1147, "friends_count": 1069, "statues_count": 25093 }, "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("2015-11-27T10:00:03.000Z"), "id": 670301070644654081, "text": "She got the primmest body ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 425338194, "name": "MITTNMAL", "screen_name": "Camp_Mall", "lang": "en", "location": "✋", "create_at": date("2011-11-30"), "description": "run it up", "followers_count": 9907, "friends_count": 9497, "statues_count": 16745 }, "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("2015-11-27T10:00:03.000Z"), "id": 670301070736801793, "text": "See a virtual tour of our listing on 4.67 Acres Rochelle Lane #Crestview #FL https://t.co/EFjyi2aQ7R #realestate https://t.co/AtLQNLo4Gl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.455741,30.822359"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Crestview", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 339, "friends_count": 287, "statues_count": 6576 }, "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": 12091, "countyName": "Okaloosa" } }
+{ "create_at": datetime("2015-11-27T10:00:03.000Z"), "id": 670301070959210496, "text": "Ion no why I'm so tired!! Sleep by falling upon me, Catch up on my rest , 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": 2540501041, "name": "CHANICE", "screen_name": "KantonsMother", "lang": "en", "location": "Louisiana, USA", "create_at": date("2014-06-01"), "description": "null", "followers_count": 440, "friends_count": 357, "statues_count": 19438 }, "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("2015-11-27T10:00:03.000Z"), "id": 670301071332388864, "text": "72.7F (Feels: 72.7F) - Humidity: 83% - Wind: 7.6mph SE - Gust: 15.2mph - Pressure: 1018.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": 215708 }, "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("2015-11-27T10:00:03.000Z"), "id": 670301071403782144, "text": "I'm at @ProhibitionChs in Charleston, SC https://t.co/iVeQL6IJq7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.94026421,32.7911638"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1852269307 }}, "user": { "id": 180153463, "name": "J Rocha", "screen_name": "joeljrocha", "lang": "en", "location": "Chicago", "create_at": date("2010-08-18"), "description": "Wine - Food - Golf", "followers_count": 28, "friends_count": 69, "statues_count": 1003 }, "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("2015-11-27T10:00:03.000Z"), "id": 670301071621947392, "text": "@jawaan_taylor74 Welcome Big Man. #GATORNATION #16ators\n#CHOMPCHOMP", "in_reply_to_status": -1, "in_reply_to_user": 2728620982, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GATORNATION", "16ators", "CHOMPCHOMP" }}, "user_mentions": {{ 2728620982 }}, "user": { "id": 480504505, "name": "JON", "screen_name": "theeprog8r", "lang": "en", "location": "null", "create_at": date("2012-02-01"), "description": "null", "followers_count": 58, "friends_count": 415, "statues_count": 605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Titusville, FL", "id": "3b97fb577b7bcfb8", "name": "Titusville", "place_type": "city", "bounding_box": rectangle("-80.86569,28.520735 -80.793083,28.648717") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12009, "countyName": "Brevard", "cityID": 1271900, "cityName": "Titusville" } }
+{ "create_at": datetime("2015-11-27T10:00:03.000Z"), "id": 670301071626080260, "text": "���� Ladies of the Bhutanese Chakra Circle, spinning llama and alpaca wool… https://t.co/Wz2RrUs73W", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.29892967,37.57123366"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1193199552, "name": "Emily LaDouceur", "screen_name": "EmJ_LaDouceur", "lang": "en", "location": "Kentucky, USA", "create_at": date("2013-02-18"), "description": "null", "followers_count": 230, "friends_count": 429, "statues_count": 4343 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berea, KY", "id": "79331dfa19658042", "name": "Berea", "place_type": "city", "bounding_box": rectangle("-84.352921,37.526946 -84.236204,37.657644") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21151, "countyName": "Madison", "cityID": 2105842, "cityName": "Berea" } }
+{ "create_at": datetime("2015-11-27T10:00:03.000Z"), "id": 670301071718424576, "text": "@sportsmommy74 @SportsMom_51 thinking of your families on this gathering weekend,", "in_reply_to_status": -1, "in_reply_to_user": 20609916, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20609916, 528771742 }}, "user": { "id": 41006908, "name": "tyrone crowder", "screen_name": "drtac1", "lang": "en", "location": "null", "create_at": date("2009-05-18"), "description": "null", "followers_count": 231, "friends_count": 986, "statues_count": 13113 }, "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("2015-11-27T10:00:03.000Z"), "id": 670301071818883072, "text": "@karleylindsey aw thanks Karley you're so sweet!��", "in_reply_to_status": 670296457916755968, "in_reply_to_user": 560543976, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 560543976 }}, "user": { "id": 601322828, "name": "n a t a l i e", "screen_name": "nataliealcala45", "lang": "en", "location": "aggieland ", "create_at": date("2012-06-06"), "description": "Matthew Varner is adorable and you should love yourself and your dog", "followers_count": 665, "friends_count": 725, "statues_count": 16296 }, "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("2015-11-27T10:00:03.000Z"), "id": 670301072041316353, "text": "I forgot how annoying Kohl's lines are on Black Friday but hey I'm getting cute stuff ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 171545317, "name": "✧Kayla✧", "screen_name": "AngelFly_13", "lang": "en", "location": "Hilliard, OH", "create_at": date("2010-07-27"), "description": "Harb Darb 16' | Color Guard | Animal Rights Activist | INTP | ♑️", "followers_count": 226, "friends_count": 552, "statues_count": 5021 }, "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("2015-11-27T10:00:03.000Z"), "id": 670301072427106304, "text": "@fuckcats24 same hahaha", "in_reply_to_status": 670300886267113472, "in_reply_to_user": 1607650794, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 1607650794 }}, "user": { "id": 445285566, "name": "Rosie", "screen_name": "rosieposie_WOO", "lang": "en", "location": "Electron Land", "create_at": date("2011-12-23"), "description": "LA 90250 // UC Berkeley - Evolve or go extinct -", "followers_count": 659, "friends_count": 641, "statues_count": 45632 }, "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("2015-11-27T10:00:03.000Z"), "id": 670301072485826560, "text": "I asked mom if she needed any help she said no I'll do everything, okay, you said it! :) off to my room I go ✨��", "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": "Fullerton, CA", "create_at": date("2014-08-09"), "description": "living in my own world", "followers_count": 704, "friends_count": 481, "statues_count": 23621 }, "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("2015-11-27T10:00:04.000Z"), "id": 670301072804720640, "text": "@RealHKsmash @oreeeo @Joe_Fries @ZachWigal @LEGIQN @CouRageJD happy thanksgiving!", "in_reply_to_status": 670055731916201984, "in_reply_to_user": 88591638, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 88591638, 35138072, 91838045, 18101470, 185909682, 246596682 }}, "user": { "id": 38685348, "name": "David Walsh", "screen_name": "Walshy", "lang": "en", "location": "Grand Rapids, MI", "create_at": date("2009-05-08"), "description": "Professional gamer fueled by Red Bull", "followers_count": 47016, "friends_count": 1358, "statues_count": 6857 }, "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("2015-11-27T10:00:04.000Z"), "id": 670301072884432899, "text": "Black Friday shopping ‼️✅", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2744029822, "name": "Angelica", "screen_name": "angelicacuffaro", "lang": "en", "location": "Lutz, FL", "create_at": date("2014-08-15"), "description": "gshs||varsity⚽️", "followers_count": 325, "friends_count": 256, "statues_count": 593 }, "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("2015-11-27T10:00:04.000Z"), "id": 670301072938811393, "text": "#The Pig & Pint / https://t.co/3JTcSiFpZZ Best ribs I've ever had https://t.co/qcf92qn9A8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.17634543,32.33720596"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "The" }}, "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": 4112, "friends_count": 876, "statues_count": 332299 }, "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("2015-11-27T10:00:04.000Z"), "id": 670301073551155200, "text": "@ariel_shealy @_JonathanWhite_ that's one of my \"catchphrases\" lol! I love archer Hahahaha", "in_reply_to_status": 670300843275624448, "in_reply_to_user": 3323903560, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3323903560, 2577188851 }}, "user": { "id": 38620483, "name": "Josh Campbell", "screen_name": "JOSHYCONKER", "lang": "en", "location": "San Bernardino CA", "create_at": date("2009-05-07"), "description": "The #DDT Is The Key 2 my success.. I Love #VideoGames & #ProWrestling Lover Of Old music #Political Activist with a #ComicBook #Fetish #NerdcoreHipHop #TVJunkie", "followers_count": 2165, "friends_count": 2697, "statues_count": 39877 }, "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("2015-11-27T10:00:04.000Z"), "id": 670301073555394560, "text": "@4uego this a terrible news!!!!", "in_reply_to_status": 670300337815719937, "in_reply_to_user": 89027171, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 89027171 }}, "user": { "id": 25153580, "name": "KKB", "screen_name": "theekbradshaw", "lang": "en", "location": "null", "create_at": date("2009-03-18"), "description": "I'm from Compton. I'm extremely smart and I think I know everything! I love shopping and being girly and I'm super passionate about Black rights. Welcome", "followers_count": 266, "friends_count": 315, "statues_count": 10810 }, "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("2015-11-27T10:00:04.000Z"), "id": 670301073559556097, "text": "Light freezing rain/wind this afternoon, high 24 (-4 C). Low 18 (-8 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15953346, "name": "Amarillo Weather", "screen_name": "AmarilloWeather", "lang": "en", "location": "Amarillo, Texas", "create_at": date("2008-08-22"), "description": "Forecasts and advisories for Amarillo, Texas", "followers_count": 3995, "friends_count": 395, "statues_count": 8594 }, "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("2015-11-27T10:00:04.000Z"), "id": 670301073714847745, "text": "I'm so upset they cancelled Arrested Development (9 years ago)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 41247057, "name": "Bonnie", "screen_name": "GinaDiStefano", "lang": "en", "location": "Boston ✈️ Florence ", "create_at": date("2009-05-19"), "description": "and miles to go before i sleep", "followers_count": 504, "friends_count": 440, "statues_count": 24200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Medford, MA", "id": "c8e7273a81fab7c0", "name": "Medford", "place_type": "city", "bounding_box": rectangle("-71.149807,42.396145 -71.071833,42.453759") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2539835, "cityName": "Medford" } }
+{ "create_at": datetime("2015-11-27T10:00:04.000Z"), "id": 670301073844731904, "text": "Still can't stop laughing at this https://t.co/YNGaNRd1Wl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 733326398, "name": "mars", "screen_name": "cherrygasmic", "lang": "en", "location": "Houston, tx", "create_at": date("2012-08-02"), "description": "null", "followers_count": 890, "friends_count": 230, "statues_count": 13443 }, "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("2015-11-27T10:00:04.000Z"), "id": 670301073978974208, "text": "When you get asked if you're by yourself at the airport 10 times :-( #BabyFaceProblems", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BabyFaceProblems" }}, "user": { "id": 3044968025, "name": "bales", "screen_name": "BaleighKathryn", "lang": "en", "location": "Grapevine, TX", "create_at": date("2015-02-19"), "description": "Just your average 18 year old with a makeup addiction & a cool blog. // feminist", "followers_count": 313, "friends_count": 264, "statues_count": 1206 }, "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-27T10:00:04.000Z"), "id": 670301074146856960, "text": "@posifemme it's Black Friday so saleee", "in_reply_to_status": 669661185986400256, "in_reply_to_user": 1061918839, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1061918839 }}, "user": { "id": 1061918839, "name": "super weenie hut jr.", "screen_name": "posifemme", "lang": "en", "location": "Intersectional Feminist", "create_at": date("2013-01-04"), "description": "perpetual perfectionist • ig: italiana_gianna • depop: giannamariec • cis she/her", "followers_count": 2677, "friends_count": 922, "statues_count": 10849 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Niverville, NY", "id": "6ba0dbda3e5521ce", "name": "Niverville", "place_type": "city", "bounding_box": rectangle("-73.695357,42.412236 -73.646751,42.465451") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36021, "countyName": "Columbia", "cityID": 3651297, "cityName": "Niverville" } }
+{ "create_at": datetime("2015-11-27T10:00:04.000Z"), "id": 670301074587283456, "text": "https://t.co/7vaIBIXMW5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 74933406, "name": "Birthday Boy", "screen_name": "TheHaloCarter", "lang": "en", "location": "Atlanta, Georgia.", "create_at": date("2009-09-16"), "description": "The Profile About Nothing .", "followers_count": 3206, "friends_count": 988, "statues_count": 244163 }, "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("2015-11-27T10:00:04.000Z"), "id": 670301074910281728, "text": "Avoiding the precinct ducking my demons", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 371822136, "name": "beck", "screen_name": "yo_beck", "lang": "en", "location": "null", "create_at": date("2011-09-11"), "description": "TEAM SCHEME AFFILIATED", "followers_count": 550, "friends_count": 491, "statues_count": 33358 }, "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("2015-11-27T10:00:04.000Z"), "id": 670301075237445632, "text": "2011 Chevrolet Impala LS\n122k miles ***super clean***\n\nOnly��3⃣0⃣0⃣0⃣ Down Today❗❗ \nAll You Need Is… https://t.co/W9SCPkO5Oc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.4077911,33.6936111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 73210964, "name": "Ritch Yung Da Hu$tla", "screen_name": "ritchyung23", "lang": "en", "location": "ATL,GA", "create_at": date("2009-09-10"), "description": "#PlayboiRitchie #prettymob #DiamondLife #KBC #NWO #TeamLO #LiveRichDieYung #BMM i am not a self-container i do it for every1", "followers_count": 1621, "friends_count": 2051, "statues_count": 10248 }, "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("2015-11-27T10:00:04.000Z"), "id": 670301075417792513, "text": "Wind 5.5 mph SW. Barometer 30.336 in, Falling Rapidly. Temperature 59.8 °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": 6, "friends_count": 43, "statues_count": 4673 }, "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("2015-11-27T10:00:04.000Z"), "id": 670301075841417216, "text": "When it takes you 20 minutes to find a parking spot so ur late to work :-)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 358593713, "name": "Julia Cobernus", "screen_name": "JuliaCobernus", "lang": "en", "location": "North Tonawanda, NY", "create_at": date("2011-08-19"), "description": "life isn't half as bad as they make it out to be", "followers_count": 564, "friends_count": 462, "statues_count": 18123 }, "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-27T10:00:04.000Z"), "id": 670301076331991040, "text": "Afternoon ice this afternoon, high 31 (-1 C). Low 25 (-4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712333, "name": "LubbockWeather", "screen_name": "LubbockWeather", "lang": "en", "location": "Lubbock, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Lubbock, Texas", "followers_count": 1168, "friends_count": 93, "statues_count": 7927 }, "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("2015-11-27T10:00:04.000Z"), "id": 670301076483100672, "text": "Rogers, Over and Out: Stow at the Doorstep of History https://t.co/v8CaUU8Icf @BulldogBackers @Stow_Bulldogs @SMFSports @stowfballalum", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 378336418, 1606915044, 250701636, 1692170474 }}, "user": { "id": 95916597, "name": "Eric Frantz", "screen_name": "jjhuddlenews", "lang": "en", "location": "null", "create_at": date("2009-12-10"), "description": "Managing Editor at http://JJHuddle.com, Ohio's SOURCE for High School Sports", "followers_count": 6105, "friends_count": 2729, "statues_count": 37619 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Centerville, OH", "id": "edacd050f402302c", "name": "Centerville", "place_type": "city", "bounding_box": rectangle("-84.21092,39.579164 -84.098555,39.674261") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39113, "countyName": "Montgomery", "cityID": 3913190, "cityName": "Centerville" } }
+{ "create_at": datetime("2015-11-27T10:00:04.000Z"), "id": 670301076587876354, "text": "@Boringstein @GayRodney Donald, why are you not listening to Badu. There is a new Andre 3000 verse in the year 2015", "in_reply_to_status": 670300857750130692, "in_reply_to_user": 24469899, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24469899, 1306549052 }}, "user": { "id": 93430666, "name": "Fake as fuck", "screen_name": "WestFortyNinth", "lang": "en", "location": "Sarah Lawrence College", "create_at": date("2009-11-29"), "description": "Thinking about either Future or Blade Runner", "followers_count": 1320, "friends_count": 1441, "statues_count": 102147 }, "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("2015-11-27T10:00:04.000Z"), "id": 670301076596383744, "text": "Unconditional love is not letting someone hurt you over and over. Jesus had an inner circle of intimate friends that did not include Judas.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4165317549, "name": "Stacy Blizzard", "screen_name": "StacyBlizzard12", "lang": "en", "location": "null", "create_at": date("2015-11-11"), "description": "null", "followers_count": 5, "friends_count": 14, "statues_count": 42 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brices Creek, NC", "id": "7c903eeaa49580c1", "name": "Brices Creek", "place_type": "city", "bounding_box": rectangle("-77.113115,35.052235 -77.052005,35.074174") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37049, "countyName": "Craven", "cityID": 3707765, "cityName": "Brices Creek" } }
+{ "create_at": datetime("2015-11-27T10:00:05.000Z"), "id": 670301077154234368, "text": "Drinking a Guinness Draught by @GuinnessIreland at @fadoaustin — https://t.co/aUQyOjF9vu #photo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.7455,30.2669"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "photo" }}, "user_mentions": {{ 188036336, 18162694 }}, "user": { "id": 764292415, "name": "Bruery Boys", "screen_name": "TheBrueryBoys", "lang": "en", "location": "null", "create_at": date("2012-08-17"), "description": "Beer is good!!", "followers_count": 225, "friends_count": 238, "statues_count": 5768 }, "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-27T10:00:05.000Z"), "id": 670301077342785536, "text": "#speakeasybarber Joe doing his thing. #tattoo #barber @ St. George, Utah https://t.co/kZu7tMVgfo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.583,37.1042"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "speakeasybarber", "tattoo", "barber" }}, "user": { "id": 340365048, "name": "Chris Deome", "screen_name": "ChrisDeome", "lang": "en", "location": "Las Vegas", "create_at": date("2011-07-22"), "description": "Living the Grok lifestyle, on a fitness journey for life. I am married to the best woman in the world, and I have two awesome kids. #100kReasons", "followers_count": 354, "friends_count": 984, "statues_count": 3247 }, "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("2015-11-27T10:00:05.000Z"), "id": 670301077439295489, "text": "@nicolex666 is the realist tho", "in_reply_to_status": -1, "in_reply_to_user": 230690894, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 230690894 }}, "user": { "id": 958815740, "name": "sarah gerken", "screen_name": "sargeliza", "lang": "en", "location": "rva ", "create_at": date("2012-11-19"), "description": "the baddest ♠️", "followers_count": 278, "friends_count": 331, "statues_count": 2988 }, "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("2015-11-27T10:00:05.000Z"), "id": 670301077669941248, "text": "I work go to school & come home ... Or ride around ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2773213402, "name": "Janiqua.", "screen_name": "WorldOfJayy_", "lang": "en", "location": "New Orleans ", "create_at": date("2014-09-16"), "description": "06/26♋️ Free My Big Bro Nerro", "followers_count": 318, "friends_count": 391, "statues_count": 6482 }, "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("2015-11-27T10:00:05.000Z"), "id": 670301077682556928, "text": "Where y'all at https://t.co/Own5WA7o3P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 58535764, "name": "SC: chriscanto", "screen_name": "_ChrisCanto", "lang": "en", "location": "Dallas, TX /New Mexico", "create_at": date("2009-07-20"), "description": "Tough times never last but tough people do. US AIRFORCE ✈️", "followers_count": 1716, "friends_count": 649, "statues_count": 538 }, "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-27T10:00:05.000Z"), "id": 670301077825277952, "text": "Nothing more nothing less. https://t.co/61aS48Zxh7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 512765977, "name": "Jamaican Goddess", "screen_name": "envyme_binks", "lang": "en", "location": "Brazil ", "create_at": date("2012-03-02"), "description": "Product of God's grace & mercy", "followers_count": 2507, "friends_count": 1649, "statues_count": 122025 }, "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("2015-11-27T10:00:05.000Z"), "id": 670301077833560064, "text": "@erikaudio Thanks very much!", "in_reply_to_status": 667768624472440832, "in_reply_to_user": 1134179844, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1134179844 }}, "user": { "id": 350341023, "name": "Escape Plan", "screen_name": "EscapePlanFilms", "lang": "en", "location": "Austin, TX", "create_at": date("2011-08-07"), "description": "Video & film production by cinematographic engineers. We offer a range of services, unique videos, & creative vision.", "followers_count": 934, "friends_count": 2085, "statues_count": 926 }, "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("2015-11-27T10:00:05.000Z"), "id": 670301077896429569, "text": "Do people not understand the way things are arranged?!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 45849313, "name": "Uriel Perez", "screen_name": "BlindNerd", "lang": "en", "location": "El Paso, TX/Austin, TX", "create_at": date("2009-06-09"), "description": "Junior at the University of Texas at Austin. History Major.", "followers_count": 439, "friends_count": 415, "statues_count": 52903 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bee Cave, TX", "id": "01761a81117e20df", "name": "Bee Cave", "place_type": "city", "bounding_box": rectangle("-98.002152,30.288613 -97.91652,30.328124") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4807156, "cityName": "Bee Cave" } }
+{ "create_at": datetime("2015-11-27T10:00:05.000Z"), "id": 670301077938511872, "text": "@erictruthy #myself \"sucks when the people you look to for encouragement give you only discouraging ideals\"", "in_reply_to_status": -1, "in_reply_to_user": 2745058307, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "myself" }}, "user_mentions": {{ 2745058307 }}, "user": { "id": 2745058307, "name": "eric cartagena", "screen_name": "erictruthy", "lang": "en", "location": "n y c ", "create_at": date("2014-08-16"), "description": "deals with applied sciences in imperfect worlds, jockey application analyst.", "followers_count": 43, "friends_count": 455, "statues_count": 1940 }, "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-27T10:00:05.000Z"), "id": 670301078076964864, "text": "@_55stacks @FBIsLife ayye ������", "in_reply_to_status": 670297826589458432, "in_reply_to_user": 2891552403, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "user_mentions": {{ 2891552403, 2292180146 }}, "user": { "id": 987892056, "name": "Mike Tiernan", "screen_name": "mtier74", "lang": "en", "location": "null", "create_at": date("2012-12-03"), "description": "Never Die Easy -Walter Payton | St. Rita Football | #K121", "followers_count": 483, "friends_count": 425, "statues_count": 5087 }, "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("2015-11-27T10:00:05.000Z"), "id": 670301078081183744, "text": "Put off my 14-miler as long as I could today. Yes, it's Nov 27 & 68°. But I'm just not feeling it.… https://t.co/3yi5hiBblE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.18271421,34.95466225"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 188591961, "name": "Angela Amick", "screen_name": "MommyAngela3", "lang": "en", "location": "Chattanooga, TN", "create_at": date("2010-09-08"), "description": "Married to best friend @RunningAmick w/3 young boys, 1 w/Down syndrome. (13.1 x 16) MARATHON FINISHER!! Addictions: @WaltDisneyWorld & @DonnieWahlberg", "followers_count": 707, "friends_count": 1022, "statues_count": 21847 }, "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": 13295, "countyName": "Walker" } }
+{ "create_at": datetime("2015-11-27T10:00:05.000Z"), "id": 670301078727041024, "text": "I'm not ready for the frost on 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": 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": 375, "friends_count": 339, "statues_count": 60434 }, "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("2015-11-27T10:00:05.000Z"), "id": 670301078781497344, "text": "HAHAHA LINDA LISTEN https://t.co/rRNYyxyYNG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 1489210910, "name": "Rusty Hook", "screen_name": "TannerWatsonn", "lang": "en", "location": "Huntington Beach, CA", "create_at": date("2013-06-06"), "description": "Love life.. Do you be you", "followers_count": 246, "friends_count": 260, "statues_count": 2518 }, "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("2015-11-27T10:00:05.000Z"), "id": 670301078966013954, "text": "Afternoon showers this afternoon, high 62 (17 C). Low 39 (4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712562, "name": "ElPasoWX", "screen_name": "ElPasoWX", "lang": "en", "location": "El Paso, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for El Paso, Texas", "followers_count": 525, "friends_count": 93, "statues_count": 7985 }, "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("2015-11-27T10:00:05.000Z"), "id": 670301079641329664, "text": "I'm a sweetheart with a dash of a fiery attitude lol.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3387684732, "name": "Morgyn Narisse Lindo", "screen_name": "RastafariCurls", "lang": "en", "location": "Sacramento, CA", "create_at": date("2015-08-29"), "description": "Just vibe with me.", "followers_count": 22, "friends_count": 16, "statues_count": 350 }, "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-27T10:00:05.000Z"), "id": 670301079649783810, "text": "@morgaan_niicole like hitting your grandpa �� that's so sad . Be happy they are alive now , yanno", "in_reply_to_status": 670300810690076677, "in_reply_to_user": 3702270317, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3702270317 }}, "user": { "id": 546383956, "name": "k gates", "screen_name": "K_Mariiiieee", "lang": "en", "location": "snapchat kaliiiprincess", "create_at": date("2012-04-05"), "description": "RHS'18", "followers_count": 1339, "friends_count": 359, "statues_count": 57045 }, "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("2015-11-27T10:00:05.000Z"), "id": 670301079649800193, "text": "In case you forgot how white people ruin 5 things at once https://t.co/oVcJqcLTi2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2785148132, "name": "Maureen Biology", "screen_name": "MaureenLeigh_", "lang": "en", "location": "Florida ", "create_at": date("2014-09-01"), "description": "SLHS Senior // @G_Eazy //", "followers_count": 375, "friends_count": 369, "statues_count": 2278 }, "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("2015-11-27T10:00:05.000Z"), "id": 670301079817609217, "text": "I forgot to tweet about how thankful I am for Spotify premium on thanksgiving :///", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 807467942, "name": "Andrew Johnston", "screen_name": "ajjbrah98", "lang": "en", "location": "lol", "create_at": date("2012-09-06"), "description": "AJ, ahj, a bae, gay j, aye j, andrew I mean pick one I hear them all", "followers_count": 210, "friends_count": 159, "statues_count": 2138 }, "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("2015-11-27T10:00:05.000Z"), "id": 670301079851020288, "text": "This one of those dreams you don't wake up from\nThen again, you don't sleep if you come from where I 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": 1352813551, "name": "Bulklahoma® Sooner", "screen_name": "BossBrax", "lang": "en", "location": "Norman, Oklahoma", "create_at": date("2013-04-14"), "description": "22. Iron sharpens iron, so one man sharpens another Prov. 27:17 OU '17 #Sooners #RavensNation #Thunderup #NeverEverQuit", "followers_count": 1413, "friends_count": 1065, "statues_count": 29260 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Duncan, OK", "id": "98ff6aa065874b12", "name": "Duncan", "place_type": "city", "bounding_box": rectangle("-97.998608,34.460573 -97.921111,34.565247") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40137, "countyName": "Stephens", "cityID": 4021900, "cityName": "Duncan" } }
+{ "create_at": datetime("2015-11-27T10:00:05.000Z"), "id": 670301079968440320, "text": "We're #hiring! Click to apply: Clinic LPN - https://t.co/8T3O6Ky2JC #Nursing #Puyallup, WA #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.2928974,47.1853785"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Nursing", "Puyallup", "Job", "Jobs", "CareerArc" }}, "user": { "id": 22868585, "name": "WA Nursing Jobs", "screen_name": "tmj_wa_nursing", "lang": "en", "location": "Washington", "create_at": date("2009-03-04"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Washington Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 714, "friends_count": 305, "statues_count": 457 }, "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("2015-11-27T10:00:05.000Z"), "id": 670301080031354880, "text": "Nice #ALDUBStaySTRONG https://t.co/c7l8qswgRx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ALDUBStaySTRONG" }}, "user": { "id": 3444257786, "name": "hemmi", "screen_name": "skylerhemmi", "lang": "en", "location": "null", "create_at": date("2015-09-03"), "description": "null", "followers_count": 157, "friends_count": 183, "statues_count": 16029 }, "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-27T10:00:05.000Z"), "id": 670301080052301824, "text": "Who's trying to go to this Future concert with Hayden and I tonight??", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 634845182, "name": "Chadillac", "screen_name": "Chad_Stegall", "lang": "en", "location": "Charlotte, NC", "create_at": date("2012-07-13"), "description": "null", "followers_count": 268, "friends_count": 246, "statues_count": 4042 }, "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("2015-11-27T10:00:05.000Z"), "id": 670301080065044481, "text": "Just passed 100k plays of \"Out For You\" on Soundcloud! Thank you for the love ✌��️ @ Pappy & Harriet's https://t.co/MrdphiwHas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-116.4932022,34.1562538"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2316360649, "name": "Colyer", "screen_name": "StevenColyer", "lang": "en", "location": "Los Angeles", "create_at": date("2014-01-28"), "description": "null", "followers_count": 125, "friends_count": 37, "statues_count": 408 }, "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": 687056, "cityName": "Yucca Valley" } }
+{ "create_at": datetime("2015-11-27T10:00:05.000Z"), "id": 670301080316608512, "text": "@clarkson37 fight me", "in_reply_to_status": 670271991337742336, "in_reply_to_user": 2739598745, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2739598745 }}, "user": { "id": 3074065199, "name": "Tasia Gilinsky", "screen_name": "tasibugg", "lang": "en", "location": "Price, UT ", "create_at": date("2015-03-05"), "description": "• Cheryl & I are in love • Booty Guru • USUE • • You change the M to W • Taken by Jack Gilinsky •", "followers_count": 236, "friends_count": 233, "statues_count": 4165 }, "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("2015-11-27T10:00:05.000Z"), "id": 670301080505425920, "text": "@ahaggerty66 thank you so much lex I love you����", "in_reply_to_status": 670299516487262208, "in_reply_to_user": 2276993830, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2276993830 }}, "user": { "id": 2162300973, "name": "ju", "screen_name": "julia_mendess1", "lang": "en", "location": "null", "create_at": date("2013-10-31"), "description": "do more of what makes you happy | eos '16", "followers_count": 454, "friends_count": 424, "statues_count": 2750 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Falmouth, MA", "id": "01f6465e1150bb6e", "name": "Falmouth", "place_type": "city", "bounding_box": rectangle("-70.657005,41.53445 -70.498041,41.640072") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25001, "countyName": "Barnstable", "cityID": 2523070, "cityName": "Falmouth" } }
+{ "create_at": datetime("2015-11-27T10:00:05.000Z"), "id": 670301080811659264, "text": "Of course my dad finds SELENAS birth control in 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": 54369309, "name": "bea fontano", "screen_name": "crazyfilipinooo", "lang": "en", "location": "302", "create_at": date("2009-07-06"), "description": "Instagram: thecrazyfilipino SHU 18' Nursing", "followers_count": 8765, "friends_count": 4408, "statues_count": 33158 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glasgow, DE", "id": "9fd15f668b562e2d", "name": "Glasgow", "place_type": "city", "bounding_box": rectangle("-75.78638,39.57775 -75.691599,39.626433") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle", "cityID": 1029350, "cityName": "Glasgow" } }
+{ "create_at": datetime("2015-11-27T10:00:05.000Z"), "id": 670301080903774208, "text": "I'm just going to online shop until my shift 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": 1276536595, "name": "jazmin", "screen_name": "jazmintenorio35", "lang": "en", "location": "AZ", "create_at": date("2013-03-17"), "description": "null", "followers_count": 521, "friends_count": 434, "statues_count": 11298 }, "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("2015-11-27T10:00:05.000Z"), "id": 670301081004584960, "text": "Your annual reminder that Juicy J has more Oscars than Dicaprio, Fincher, Thomas Newman and Deakins combined.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27257661, "name": "John Rivera", "screen_name": "Thejohnrivera", "lang": "en", "location": "Chicago Illinois ", "create_at": date("2009-03-28"), "description": "Film Critic, Reality Tv Junkie, Sports Fanatic, Party Animal I RT a lot of stuff as well", "followers_count": 428, "friends_count": 2001, "statues_count": 37294 }, "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-27T10:00:06.000Z"), "id": 670301081251901440, "text": "what type of movie should I watch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2560517046, "name": "ken for the win", "screen_name": "kennedyadamsha", "lang": "en", "location": "null", "create_at": date("2014-06-10"), "description": "I had to restart my potatoes", "followers_count": 405, "friends_count": 641, "statues_count": 2968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mont Belvieu, TX", "id": "3df077e12571b025", "name": "Mont Belvieu", "place_type": "city", "bounding_box": rectangle("-94.872936,29.860827 -94.84694,29.879989") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48071, "countyName": "Chambers", "cityID": 4849068, "cityName": "Mont Belvieu" } }
+{ "create_at": datetime("2015-11-27T10:00:06.000Z"), "id": 670301081277075456, "text": "I know that's why I had to chop mine https://t.co/voIqQOMyNo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1121514170, "name": "$$$ Meem", "screen_name": "Sylver__24", "lang": "en", "location": "The Bay", "create_at": date("2013-01-26"), "description": "#LongLiveLeel | #GOERISM | Trying to find a baddie to wife so my tweets could suck | Y'all don't really hear me tho", "followers_count": 1168, "friends_count": 756, "statues_count": 162785 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301081604231169, "text": "Check out our #listing in #Baker #FL https://t.co/CBlNOQZKcJ #realestate #realtor https://t.co/Xac3cR2KZ5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.661922,30.885057"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "Baker", "FL", "realestate", "realtor" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 339, "friends_count": 287, "statues_count": 6577 }, "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": 12091, "countyName": "Okaloosa" } }
+{ "create_at": datetime("2015-11-27T10:00:06.000Z"), "id": 670301081637777409, "text": "Rain this afternoon, high 35 (2 C). Low 28 (-2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712512, "name": "AbileneWeather", "screen_name": "AbileneWeather", "lang": "en", "location": "Abilene, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Abilene, Texas", "followers_count": 916, "friends_count": 93, "statues_count": 8025 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301081793118209, "text": "i hate pretending to be nice to someone just bc they're nice 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": 394249137, "name": "rupac shakur", "screen_name": "youngsusgod", "lang": "en", "location": "chapel hill", "create_at": date("2011-10-19"), "description": "KURBBOYZ2015", "followers_count": 188, "friends_count": 241, "statues_count": 5060 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301081977692160, "text": "@kyr95 Russell gave Moffat the theory on how to explain it.\nDammit Russell.\nCursing us with tears even after you've left the show", "in_reply_to_status": 670300887852654593, "in_reply_to_user": 2844702656, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2227941692 }}, "user": { "id": 2844702656, "name": "Noah Benjamin", "screen_name": "EmperorBenjie", "lang": "en", "location": "null", "create_at": date("2014-10-07"), "description": "WORLD'S BIGGEST ZOOM AND KYLO REN FANBOY. #DCEUSupporter #StarWarsFanboy #FlashFanboy", "followers_count": 551, "friends_count": 1106, "statues_count": 19720 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301082103468032, "text": "Ooouu if someone snatched something from my child they're getting Sparta kicked. https://t.co/fkhzNADuC5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 31577691, "name": "Queen Ynobe", "screen_name": "ebonyspencer_", "lang": "en", "location": "greenville", "create_at": date("2009-04-15"), "description": "Ya-No-Be | 21 | East Carolina University | F.A.M.E. Modeling Troupe | #RIP JL & MB ❤️", "followers_count": 1661, "friends_count": 643, "statues_count": 88721 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301082174644224, "text": "@emo_samoan bye", "in_reply_to_status": 670300970039947264, "in_reply_to_user": 954814152, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 954814152 }}, "user": { "id": 2874169064, "name": "aericka", "screen_name": "_aericka_21", "lang": "en", "location": "Oxnard, CA", "create_at": date("2014-10-23"), "description": "groovy", "followers_count": 311, "friends_count": 226, "statues_count": 18670 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301082526965760, "text": "lmao yesterday saw everybody from allstar", "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 pooh", "screen_name": "niaamyaa", "lang": "en", "location": "null", "create_at": date("2014-05-05"), "description": "null", "followers_count": 1179, "friends_count": 972, "statues_count": 70505 }, "place": { "country": "United States", "country_code": "United States", "full_name": "River Ridge, LA", "id": "c32a444362096e3a", "name": "River Ridge", "place_type": "city", "bounding_box": rectangle("-90.23943,29.937959 -90.202855,29.97557") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2265150, "cityName": "River Ridge" } }
+{ "create_at": datetime("2015-11-27T10:00:06.000Z"), "id": 670301082548072448, "text": "@patters0n__ I Had To ����", "in_reply_to_status": 670300815366557696, "in_reply_to_user": 876856628, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 876856628 }}, "user": { "id": 3293284095, "name": "Ajai ❤", "screen_name": "naateeyaa16", "lang": "en", "location": "Stuart, FL", "create_at": date("2015-05-21"), "description": "X-XXIX-MMXV --- Aracelis ❤", "followers_count": 310, "friends_count": 247, "statues_count": 7324 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Salerno, FL", "id": "97e2346184350535", "name": "Port Salerno", "place_type": "city", "bounding_box": rectangle("-80.237333,27.111576 -80.151866,27.172504") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12085, "countyName": "Martin", "cityID": 1258727, "cityName": "Port Salerno" } }
+{ "create_at": datetime("2015-11-27T10:00:06.000Z"), "id": 670301082568921089, "text": "Local artist in Idaho and Utah are interested in booking shows DM me or links in 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": 719302034, "name": "Lifer Music", "screen_name": "LiferMusic208", "lang": "en", "location": "We are all Earthlings ", "create_at": date("2012-07-26"), "description": "18+ It's bigger than music it's a movement. @JRoyal208 @JNutty274. Lifer Music Management Team #LMG lifermanagement@gmail.com https://soundcloud.com/lifermusic", "followers_count": 938, "friends_count": 609, "statues_count": 28744 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301082623586304, "text": "https://t.co/HeiWFrAuEm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2614318089, "name": "Nqg", "screen_name": "nqg0214", "lang": "en", "location": "null", "create_at": date("2014-06-16"), "description": "♒ aquarius♒", "followers_count": 52, "friends_count": 116, "statues_count": 693 }, "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-27T10:00:06.000Z"), "id": 670301082812215296, "text": "Wind 0.0 mph E. Barometer 30.01 in, Rising slowly. Temperature 53.2 °F. Rain today 0.07 in. Humidity 57%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.13472222,33.81666667"), "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": 20, "friends_count": 92, "statues_count": 12853 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301082979930112, "text": "@KatrinaNation @thenation Americans need journalists to really act like the 4th Estate in this odd prez campaign full of lies & extremism.", "in_reply_to_status": 670298420364578816, "in_reply_to_user": 59159771, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 59159771, 1947301 }}, "user": { "id": 25003156, "name": "J. Lang Wood", "screen_name": "woodjl68", "lang": "en", "location": "On a planet far, far away", "create_at": date("2009-03-17"), "description": "Writer", "followers_count": 357, "friends_count": 110, "statues_count": 46694 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301083386945538, "text": "sooooo who's gonna come w me to this & the river of lights ??? https://t.co/6mBsSZGXoP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2720821294, "name": "lece.", "screen_name": "lovelylece12", "lang": "en", "location": "null", "create_at": date("2014-07-22"), "description": "null", "followers_count": 533, "friends_count": 223, "statues_count": 10650 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301083756060673, "text": "Here at the thanksgiving thunder in Georgia on this Friday fixing to start qualifying ran pretty good practice https://t.co/LNMRf8qiDz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.246301,34.38459"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3561907335, "name": "Corey Sheppard", "screen_name": "LilBuck94", "lang": "en", "location": "null", "create_at": date("2015-09-05"), "description": "Corey Reed Sheppard im 21 and like to raise hell and party to the sun comes up", "followers_count": 20, "friends_count": 84, "statues_count": 9 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia Karting Complex", "id": "07d9f45988c88000", "name": "Georgia Karting Complex", "place_type": "poi", "bounding_box": rectangle("-83.2463011,34.3845899 -83.246301,34.38459") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13119, "countyName": "Franklin", "cityID": 1313352, "cityName": "Carnesville" } }
+{ "create_at": datetime("2015-11-27T10:00:06.000Z"), "id": 670301083948875777, "text": "We're #hiring! Click to apply: CAE in Cyber Operations Summer Intern Program - https://t.co/5nGfXa6j4C #internship #Honolulu, HI #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "internship", "Honolulu", "Job", "Jobs" }}, "user": { "id": 154601054, "name": "TMJ-HON Intern Jobs", "screen_name": "tmj_HON_intern", "lang": "en", "location": "Honolulu, HI", "create_at": date("2010-06-11"), "description": "Follow this account for geo-targeted Internships job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 263, "friends_count": 256, "statues_count": 3 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301083999141888, "text": "definition of Texas weather 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": 2152072110, "name": "tubberson", "screen_name": "BethelBlaine", "lang": "en", "location": "null", "create_at": date("2013-10-23"), "description": "Blaine Tubs Bethel", "followers_count": 491, "friends_count": 479, "statues_count": 1976 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301084313780224, "text": "Try https://t.co/uhh6IamsGf goat cheese, oven roasted tomato, pesto, no tomato sauce #goat #menu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.33691638,33.95118693"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "goat", "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": 4112, "friends_count": 876, "statues_count": 332300 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301084490027010, "text": "N4TRQ-8 Passing , , https://t.co/Jebzp2ND2R #APRS 13b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.229,36.7365"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "APRS" }}, "user": { "id": 56001919, "name": "Arthur", "screen_name": "n4trqjedi", "lang": "en", "location": "APRS N of Chesapeake, VA", "create_at": date("2009-07-11"), "description": "Electronics Technician and Amateur Radio Operator N4TRQ\r\nhttp://n4trq.wordpress.com", "followers_count": 423, "friends_count": 681, "statues_count": 46532 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301084552818688, "text": "Wind 0.0 mph ---. Barometer 1008.6 hPa, Falling. Temperature 86.2 °F. Rain today 0.00 in. Humidity 18%", "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": 91, "friends_count": 264, "statues_count": 135704 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301084687011840, "text": "Afternoon rain this afternoon, high 36 (2 C). Low 30 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712530, "name": "MidlandWeather", "screen_name": "MidlandWeather", "lang": "en", "location": "Midland/Odessa, TX", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Midland/Odessa, TX", "followers_count": 864, "friends_count": 93, "statues_count": 8008 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301084884201476, "text": "@jcastro_69 https://t.co/JWewkifEoK", "in_reply_to_status": -1, "in_reply_to_user": 2446371343, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2446371343 }}, "user": { "id": 2777316277, "name": "Amanda Valdez", "screen_name": "radmanda_", "lang": "en", "location": "null", "create_at": date("2014-08-28"), "description": "happy", "followers_count": 325, "friends_count": 246, "statues_count": 15366 }, "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-27T10:00:06.000Z"), "id": 670301084892553217, "text": "#Infosec #security UK immigration rules fly in the face of cyber security skills shortage -… https://t.co/Q8RBBI5toR #cybersecurity", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.79259811,36.07372228"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Infosec", "security", "cybersecurity" }}, "user": { "id": 28918980, "name": "#SocialMedia NC", "screen_name": "greensboro_nc", "lang": "en", "location": "Greensboro, NC", "create_at": date("2009-04-04"), "description": "Twitter★LinkedIn★Facebook★ Pinterest★Google★Instagram★ Tech★SEO★Business★Marketing★ Leadership★Social Media Influencer★ IG:@greensboro_nc♥️NC /@_North_Carolina.", "followers_count": 262777, "friends_count": 231540, "statues_count": 898099 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301084989132800, "text": "#Gratitude @RocioGarciaM1 @CaribbeanFitOne @3rdrockhome @positive4life @UnleashedNyOrg @Hacoooooooooooo @abcollar https://t.co/HqyWSBfaYg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Gratitude" }}, "user_mentions": {{ 2607392729, 95777654, 149296998, 989374110, 180764863, 68116569, 40769584 }}, "user": { "id": 29997206, "name": "Lauri Meizler", "screen_name": "LauriMeizler", "lang": "en", "location": "Boston area", "create_at": date("2009-04-09"), "description": "#Entrepreneur #healthcoach #speaker #blogger supporting you to realize strength #love innerpeace, combat #DomesticAbuse #Crohns mom #WomenWhoInspire #Glutenfree", "followers_count": 44497, "friends_count": 43624, "statues_count": 36910 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301085064556544, "text": "My girl https://t.co/chzjsbfRCn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2211940507, "name": "daaaanniii", "screen_name": "danimariecherry", "lang": "en", "location": "drhs '17", "create_at": date("2013-11-23"), "description": "•happy• right?", "followers_count": 440, "friends_count": 446, "statues_count": 841 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301085236633602, "text": "Temp 63.9°F RH 55% Wind 0.5 E Gust 6.0 E SLP 30.456 in Falling Rain 0.00 in Solar 489 UV 2.8 #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": 103, "friends_count": 65, "statues_count": 31375 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301085253238785, "text": "�� #FRIYAY @ Pop's Poboys https://t.co/etDKDq6mVM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.0188904,30.2218208"), "retweet_count": 0, "lang": "hr", "is_retweet": false, "hashtags": {{ "FRIYAY" }}, "user": { "id": 39944204, "name": "Kim Dupuis", "screen_name": "KiiimiiiD", "lang": "en", "location": "New Orleans, LA", "create_at": date("2009-05-13"), "description": "Lover of life, contributor of geekness. Tweeting music on the reg. // Marketing Mgr @thebukuproject #BUKU2016", "followers_count": 553, "friends_count": 1457, "statues_count": 4420 }, "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("2015-11-27T10:00:06.000Z"), "id": 670301085282643968, "text": "Big bear bound.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2258555964, "name": "fuck off sandwich", "screen_name": "ali_juliano9", "lang": "en", "location": "null", "create_at": date("2013-12-22"), "description": "so good at being in trouble", "followers_count": 166, "friends_count": 148, "statues_count": 1037 }, "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("2015-11-27T10:00:07.000Z"), "id": 670301085743976448, "text": "@Isabel__Monreal awh thank you meg! ☺️��❤️", "in_reply_to_status": 670300927664852992, "in_reply_to_user": 3344079855, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3344079855 }}, "user": { "id": 1689042566, "name": "Evez (:", "screen_name": "evez99", "lang": "en", "location": "Texas, USA", "create_at": date("2013-08-21"), "description": "zulu♡", "followers_count": 350, "friends_count": 324, "statues_count": 5540 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakeport, TX", "id": "001efdc165225e22", "name": "Lakeport", "place_type": "city", "bounding_box": rectangle("-94.726067,32.393632 -94.696885,32.417034") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48183, "countyName": "Gregg", "cityID": 4840696, "cityName": "Lakeport" } }
+{ "create_at": datetime("2015-11-27T10:00:07.000Z"), "id": 670301085907595265, "text": "seen him! hehe but I would love to see him again�� https://t.co/zAw0duuJe9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 318671326, "name": "Hayley Douglas △⃒⃘", "screen_name": "hayleyalinee", "lang": "en", "location": "null", "create_at": date("2011-06-16"), "description": "ovo/freebandz @L_DeLaune is my bitch", "followers_count": 849, "friends_count": 251, "statues_count": 2746 }, "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("2015-11-27T10:00:07.000Z"), "id": 670301085916004352, "text": "I drop my comb so many times when I flat iron 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": 2881704781, "name": "Hail to the V", "screen_name": "VersaceLaGail", "lang": "en", "location": "Dal, Tx☀️✈️Gram", "create_at": date("2014-10-28"), "description": "Be a catalyst to your happiness. Call me Sace. Grambling State Softball #20 ⚾️❤️", "followers_count": 540, "friends_count": 479, "statues_count": 5504 }, "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("2015-11-27T10:00:07.000Z"), "id": 670301086528487424, "text": "@TaylorHicks It's lovely. Thank you, Taylor.", "in_reply_to_status": 670289954543661056, "in_reply_to_user": 27755192, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27755192 }}, "user": { "id": 167120259, "name": "margaret louise", "screen_name": "somersette", "lang": "en", "location": "shadow of the blue ridge", "create_at": date("2010-07-15"), "description": "Taylor Hicks fan. Play the blues......... play the blues...Born in 1927,lived thru 7 wars,13 presidents, lived in 22 towns & 29 houses. Life's good", "followers_count": 90, "friends_count": 74, "statues_count": 3569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lyman, SC", "id": "010557952781abf0", "name": "Lyman", "place_type": "city", "bounding_box": rectangle("-82.174289,34.927826 -82.097952,35.028296") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45083, "countyName": "Spartanburg", "cityID": 4543315, "cityName": "Lyman" } }
+{ "create_at": datetime("2015-11-27T10:00:07.000Z"), "id": 670301086570401793, "text": "#blackfriday \n#buyonegetonefree #supersale #docsmouthguards #dmgtakeover #inthelab \n#tagsomeone… https://t.co/Gn9XkQYir4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.3439825,32.7322572"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "blackfriday", "buyonegetonefree", "supersale", "docsmouthguards", "dmgtakeover", "inthelab", "tagsomeone" }}, "user": { "id": 3178429021, "name": "Docs Mouthguards", "screen_name": "docsmouthguards", "lang": "en", "location": "Fort Worth", "create_at": date("2015-04-28"), "description": "#DocsMouthGuards is based out of Fort Worth Tx.\nCustom professional grade protection for all athletes.\n#FollowUs #Instagram #snapchat @docsmouthguards", "followers_count": 38, "friends_count": 157, "statues_count": 340 }, "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("2015-11-27T10:00:07.000Z"), "id": 670301086801141760, "text": "One Direction just came on the forever 21 radio . ���� for once I don't wanna kill myself while in here", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 200730025, "name": "white phandom trash", "screen_name": "mythicallyphan", "lang": "en", "location": "ran off to japan", "create_at": date("2010-10-09"), "description": "need to go tell my mom my only life goal is to get spit roasted by Dan and Phil. \nbut I hate Dan howell", "followers_count": 1229, "friends_count": 2154, "statues_count": 36377 }, "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("2015-11-27T10:00:07.000Z"), "id": 670301086872301568, "text": "Temp: 32.9°F - Dew Point: 30.1° - Wind: 6.2 mph - Gust: 13.9 - Rain Today: 0.52in. - Pressure: 30.43in, - 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": 9749 }, "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("2015-11-27T10:00:07.000Z"), "id": 670301087249772546, "text": "I'm bored so https://t.co/lqfUJ02RIi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1213635487, "name": "idk", "screen_name": "OsbournNick", "lang": "en", "location": "null", "create_at": date("2013-02-23"), "description": "18. Heavy music, Tx.", "followers_count": 343, "friends_count": 142, "statues_count": 3456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Knollwood, TX", "id": "a4557ab92995ce9e", "name": "Knollwood", "place_type": "city", "bounding_box": rectangle("-96.623364,33.682684 -96.614402,33.694415") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48181, "countyName": "Grayson", "cityID": 4839680, "cityName": "Knollwood" } }
+{ "create_at": datetime("2015-11-27T10:00:07.000Z"), "id": 670301087623057409, "text": "3 months ago. this hair day started off horrible but ended up being 1 of my best ���� https://t.co/pc9KEdB9Do", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 309876251, "name": "dharmaaray", "screen_name": "dharmaaray", "lang": "en", "location": "null", "create_at": date("2011-06-02"), "description": "null", "followers_count": 974, "friends_count": 601, "statues_count": 25494 }, "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("2015-11-27T10:00:07.000Z"), "id": 670301087820156929, "text": "Cloudy this afternoon, high 77 (25 C). Low 69 (21 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712593, "name": "CorpusChristiWX", "screen_name": "CorpusChristiWX", "lang": "en", "location": "Corpus Christi, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Corpus Christi, Texas", "followers_count": 1004, "friends_count": 93, "statues_count": 8000 }, "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("2015-11-27T10:00:07.000Z"), "id": 670301087988101121, "text": "Id drop everything 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": 588029513, "name": "Bobby 婊子", "screen_name": "bobbykennedyy", "lang": "en", "location": "District Of Columbia, USA", "create_at": date("2012-05-22"), "description": "#ZW7", "followers_count": 868, "friends_count": 490, "statues_count": 12801 }, "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-27T10:00:07.000Z"), "id": 670301088013230081, "text": "Wind 2.0 mph S. Barometer 30.283 in, Falling. Temperature 56.6 °F. Rain today 0.03 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": 363, "friends_count": 286, "statues_count": 6286 }, "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("2015-11-27T10:00:07.000Z"), "id": 670301088193605632, "text": "#linebrawl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "linebrawl" }}, "user": { "id": 2323294760, "name": "Trevor Suhr", "screen_name": "trevorsuhr13", "lang": "en", "location": "null", "create_at": date("2014-02-01"), "description": "|Chicago Bruins CSDHL #13| |Lake Park Highschool|", "followers_count": 286, "friends_count": 232, "statues_count": 615 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Romulus, MI", "id": "781e991b9f95b37f", "name": "Romulus", "place_type": "city", "bounding_box": rectangle("-83.446302,42.179271 -83.306006,42.268212") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2669420, "cityName": "Romulus" } }
+{ "create_at": datetime("2015-11-27T10:00:07.000Z"), "id": 670301088273182721, "text": "Mission Payloads Radiation Systems Engineer - General Dynamics - AIS: (#Scottsdale, AZ) https://t.co/8Qx2EsuRnv #Engineering #Veterans #Job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.9260519,33.4941704"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Scottsdale", "Engineering", "Veterans", "Job" }}, "user": { "id": 2348856350, "name": "GD-AIS Careers", "screen_name": "GDAISCareers", "lang": "en", "location": "null", "create_at": date("2014-02-17"), "description": "#nowhiring Build your career at General Dynamics today. #openjobs Follow this account to see all our job postings or visit us at", "followers_count": 185, "friends_count": 9, "statues_count": 615 }, "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("2015-11-27T10:00:07.000Z"), "id": 670301088336248832, "text": "Looks like she took a pic of herself from the back too but nope it's me https://t.co/NROaghfkyT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 357824362, "name": "Samantha Goldfarb ♡", "screen_name": "sammigoldfarb", "lang": "en", "location": "Orlando, FL", "create_at": date("2011-08-18"), "description": "If you see someone without a smile, give them one of yours", "followers_count": 1407, "friends_count": 949, "statues_count": 29802 }, "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("2015-11-27T10:00:07.000Z"), "id": 670301088650629120, "text": "We are looking for a buyer on 6244 BULLET Dr #Crestview #FL https://t.co/njmYF54hkd #realestate https://t.co/EybXJ7yT4W", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.572742,30.825736"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Crestview", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6578 }, "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": 12091, "countyName": "Okaloosa" } }
+{ "create_at": datetime("2015-11-27T10:00:07.000Z"), "id": 670301089217015808, "text": "Haaaa�������������� https://t.co/nQUMFmulaQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 236120790, "name": "G.L.W", "screen_name": "genejalave", "lang": "en", "location": "null", "create_at": date("2011-01-09"), "description": "snapchat|| genejalave", "followers_count": 777, "friends_count": 535, "statues_count": 27247 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Darby, PA", "id": "442f8732107cab6d", "name": "Darby", "place_type": "city", "bounding_box": rectangle("-75.275294,39.912157 -75.247034,39.930325") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4218152, "cityName": "Darby" } }
+{ "create_at": datetime("2015-11-27T10:00:07.000Z"), "id": 670301089242071040, "text": "Pro tip: never trust a white hair stylist with the tracks of her extensions showing.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 279288934, "name": "P. Schreif", "screen_name": "pschreif", "lang": "en", "location": "Minneapolis, MN", "create_at": date("2011-04-08"), "description": "// underneath this pretty face is something complicated. i come with a side of trouble but i know that's why you're staying //", "followers_count": 209, "friends_count": 375, "statues_count": 7133 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maple Grove, MN", "id": "6390886d39468948", "name": "Maple Grove", "place_type": "city", "bounding_box": rectangle("-93.528854,45.064772 -93.400913,45.152293") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2740166, "cityName": "Maple Grove" } }
+{ "create_at": datetime("2015-11-27T10:00:08.000Z"), "id": 670301089678266368, "text": "@todiyah kendall is so freaking pretty omg watch out todiyah she's gonna break some hearts", "in_reply_to_status": 670143961931997184, "in_reply_to_user": 2279923362, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2279923362 }}, "user": { "id": 1228959859, "name": "yliza jasmine", "screen_name": "ylizajasmine", "lang": "en", "location": "null", "create_at": date("2013-02-28"), "description": "dwight you ignorant slut", "followers_count": 526, "friends_count": 270, "statues_count": 32413 }, "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("2015-11-27T10:00:08.000Z"), "id": 670301089804128257, "text": "@settern Congratulations! (was your spelling of \"merry me\" Freudian?)", "in_reply_to_status": 670298282082570241, "in_reply_to_user": 14957252, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14957252 }}, "user": { "id": 28211240, "name": "Jerry Knaus", "screen_name": "jerryknaus", "lang": "en", "location": "null", "create_at": date("2009-04-01"), "description": "A Little Lebowski Urban Achiever", "followers_count": 52, "friends_count": 106, "statues_count": 1381 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Farmington, UT", "id": "005390255b84c40b", "name": "Farmington", "place_type": "city", "bounding_box": rectangle("-111.945782,40.952071 -111.872385,41.016875") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49011, "countyName": "Davis", "cityID": 4924740, "cityName": "Farmington" } }
+{ "create_at": datetime("2015-11-27T10:00:08.000Z"), "id": 670301089820860417, "text": "does anyone know where I can find some really good quality turtlenecks for the winter? trying to do some online shopping", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2646800011, "name": "dog mom", "screen_name": "trashybae", "lang": "en", "location": "tx", "create_at": date("2014-07-14"), "description": "intersectional feminist • virgo • art • 18\n#blacklivesmatter #translivesmatter // ig: trashybae", "followers_count": 598, "friends_count": 346, "statues_count": 7274 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alamo, TX", "id": "3edaa460eefb4857", "name": "Alamo", "place_type": "city", "bounding_box": rectangle("-98.144602,26.135228 -98.083938,26.239135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4801576, "cityName": "Alamo" } }
+{ "create_at": datetime("2015-11-27T10:00:08.000Z"), "id": 670301089917353984, "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": 326376917, "name": "Mohammed Alshayeb", "screen_name": "alshayeb_91", "lang": "en", "location": "U.S.A ", "create_at": date("2011-06-29"), "description": "off !", "followers_count": 339, "friends_count": 372, "statues_count": 14360 }, "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-27T10:00:08.000Z"), "id": 670301090106052608, "text": "Diary of Alicia Keys or Baduism\nThe Black Album\nCarter 2\nLate Registration\nGet Rich of Die Trying or Man on the Moon https://t.co/EhEF4uic3N", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 83458108, "name": "Ya Tshisuaka-Patríck", "screen_name": "KingWaka_", "lang": "en", "location": "Kinshasa, DRC ✈ Fort Worth, TX", "create_at": date("2009-10-18"), "description": "My words are only worth as much as the value you give them. I'm just a intellectual who loves to act stupid. Your 6'7 friendly neighborhood African #UNT #243", "followers_count": 4753, "friends_count": 2931, "statues_count": 144304 }, "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("2015-11-27T10:00:08.000Z"), "id": 670301090718466048, "text": "Rain this afternoon, high 38 (3 C). Low 32 (0 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712706, "name": "TulsaWX", "screen_name": "TulsaWX", "lang": "en", "location": "Tulsa, Oklahoma", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Tulsa, Oklahoma", "followers_count": 511, "friends_count": 93, "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("2015-11-27T10:00:08.000Z"), "id": 670301090756210688, "text": "Join the Starbucks team! See our latest #Hospitality #job opening here: https://t.co/b39SI7RyNs #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.7044918,34.4193212"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 99812776, "name": "TMJ-CAA HRTA Jobs", "screen_name": "tmj_CAA_HRTA", "lang": "en", "location": "Santa Barbara, CA", "create_at": date("2009-12-27"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Santa Barbara, CA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 278, "friends_count": 268, "statues_count": 43 }, "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("2015-11-27T10:00:08.000Z"), "id": 670301090777137152, "text": "@TheLakeShOwKinG @GamerBabeJess you gotta ignore the hecklers", "in_reply_to_status": 670300859821981696, "in_reply_to_user": 368735686, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 368735686, 120828505 }}, "user": { "id": 2915933862, "name": "#UnleashTheKraken", "screen_name": "Marcus_Mims213", "lang": "en", "location": "Killeen, TX", "create_at": date("2014-12-01"), "description": "null", "followers_count": 360, "friends_count": 598, "statues_count": 22133 }, "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("2015-11-27T10:00:08.000Z"), "id": 670301090831802369, "text": "Bus https://t.co/1F2s13POmD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2693402965, "name": "1️⃣day", "screen_name": "_MIRAcleland", "lang": "en", "location": "null", "create_at": date("2014-07-30"), "description": "peace love and happiness", "followers_count": 1650, "friends_count": 1318, "statues_count": 23033 }, "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-27T10:00:08.000Z"), "id": 670301091213496320, "text": "Drinking The Judge by @taftsalehouse at @taftsalehouse — https://t.co/USrCfkERhA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.5175,39.1113"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3070167095, 3070167095 }}, "user": { "id": 2275844509, "name": "AndTis4", "screen_name": "CincinnaTisfor", "lang": "en", "location": "Cincinnati, Ohio", "create_at": date("2014-01-04"), "description": "Just moved to Cincinnati. From England via Georgia. Love music and craft beer.", "followers_count": 35, "friends_count": 170, "statues_count": 703 }, "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("2015-11-27T10:00:08.000Z"), "id": 670301091314159618, "text": "I feel like an asshole when people notice me in public and it takes me awhile to notice who they are.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 235776550, "name": "Mr. Sherman", "screen_name": "JustTooFocused_", "lang": "en", "location": "null", "create_at": date("2011-01-08"), "description": "You can play your heart out everyone don't get a ring though. #860", "followers_count": 925, "friends_count": 892, "statues_count": 68275 }, "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("2015-11-27T10:00:08.000Z"), "id": 670301091423236096, "text": "Peanuts movie! (@ AMC Loews Boston Common 19 - @amctheatres for The Peanuts Movie 3D in Boston, MA) https://t.co/HtgnVnPLE4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.0637951,42.35327609"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 40245758 }}, "user": { "id": 78218693, "name": "Elana Kahn", "screen_name": "emk808", "lang": "en", "location": "Brighton, MA", "create_at": date("2009-09-28"), "description": "Registered nurse, student midwife, and budding actress. @HeavynDragon, @healingmachine and @Rattify are my girls.", "followers_count": 1003, "friends_count": 839, "statues_count": 27514 }, "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("2015-11-27T10:00:08.000Z"), "id": 670301091741966336, "text": "LCC 17, Linton-Stockton 0. TVs all over the state switch to UCF.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 182040472, "name": "Gary Jewell", "screen_name": "funrunr1", "lang": "en", "location": "Kokomo, IN", "create_at": date("2010-08-23"), "description": "Ex-engineer, former Boilermaker runner, high school XC & track coach. USATF Level II Endurance. Muckraker. Advertising account exec. #WesternXC", "followers_count": 180, "friends_count": 225, "statues_count": 4283 }, "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("2015-11-27T10:00:08.000Z"), "id": 670301092182294528, "text": "Ahahaha slaying so many lives we all know a couple of these though ���� https://t.co/Eju2u91yuO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 142845322, "name": "Mr. Rodriguez ⚾️", "screen_name": "stumpy_danny", "lang": "en", "location": "Moreno Valley", "create_at": date("2010-05-11"), "description": "Ball is life. Yankees and Los Doyers. My girl is hotter than yours. Shortstop for the Sons Of Pitches. CSUF class of 2017", "followers_count": 456, "friends_count": 430, "statues_count": 29724 }, "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("2015-11-27T10:00:08.000Z"), "id": 670301092379549696, "text": "@OzikTrey @TheRealWiseOne O_o we are doing it! Hope you are feeling better buddy!", "in_reply_to_status": 670300055396446209, "in_reply_to_user": 2209451809, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2209451809, 2794592434 }}, "user": { "id": 309338567, "name": "Linyb ", "screen_name": "Linyb55", "lang": "en", "location": "Degenning Hard", "create_at": date("2011-06-01"), "description": "sports and other nonsense", "followers_count": 232, "friends_count": 358, "statues_count": 1434 }, "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("2015-11-27T10:00:08.000Z"), "id": 670301092417138692, "text": "Now I know why we had those weird ass games in P.E https://t.co/Gn4jixvmPG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 853667875, "name": "ripper", "screen_name": "billy_swisher", "lang": "en", "location": "rollin with the lench mob ", "create_at": date("2012-09-29"), "description": "I pilot a datsun 280z • BMX", "followers_count": 937, "friends_count": 358, "statues_count": 21916 }, "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("2015-11-27T10:00:08.000Z"), "id": 670301092530544642, "text": "My mom must not understand the words \"I'm not cooking\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 247579361, "name": "shanti ❣", "screen_name": "_shanticashee", "lang": "en", "location": "in my skin", "create_at": date("2011-02-04"), "description": "⠀⠀⠀⠀⠀⠀⠀⠀⠀happy ⠀⠀⠀⠀⠀⠀⠀⠀ @augustalsina✨ Delonta Nakyrus Hart 12:16 ❤️ & Tierra Hall 3:27 ❤️ forever in our hearts", "followers_count": 893, "friends_count": 782, "statues_count": 50974 }, "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("2015-11-27T10:00:08.000Z"), "id": 670301092798836736, "text": "See a virtual tour of our listing on 702 E Chestnut Ave #Crestview #FL https://t.co/T6f1e3DADf #realestate https://t.co/vT8JsUTOVX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.561696,30.757915"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Crestview", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crestview, FL", "id": "32bb9ace0723336d", "name": "Crestview", "place_type": "city", "bounding_box": rectangle("-86.635021,30.695757 -86.534517,30.805126") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1215475, "cityName": "Crestview" } }
+{ "create_at": datetime("2015-11-27T10:00:08.000Z"), "id": 670301093050630145, "text": "When your sister ruins every damn family vacation you take, which is why you stopped going on them in the first place. #imjustgonnadrink", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "imjustgonnadrink" }}, "user": { "id": 1884205862, "name": "Meg Hall", "screen_name": "meg_hall_", "lang": "en", "location": "| IG: meg__hall | future nurse", "create_at": date("2013-09-19"), "description": "if i upset you, don't stress. never forget, god hasn't finished with me yet.", "followers_count": 289, "friends_count": 365, "statues_count": 1647 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Myers Beach, FL", "id": "6553e146f836a023", "name": "Fort Myers Beach", "place_type": "city", "bounding_box": rectangle("-81.968676,26.402222 -81.882625,26.466423") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12071, "countyName": "Lee", "cityID": 1224150, "cityName": "Fort Myers Beach" } }
+{ "create_at": datetime("2015-11-27T10:00:08.000Z"), "id": 670301093121789953, "text": "Smh....hoodlums https://t.co/f3Qq3wqQAD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 745741638, "name": "cool ass .", "screen_name": "YoGrannyKno", "lang": "en", "location": "waveyland, surfing", "create_at": date("2012-08-08"), "description": "By humility and the fear of the Lord are riches, and honour, and life |(Proverbs 22:4 KJV) | booking: Gatlin.Tyren@Gmail.com | #SU19", "followers_count": 3806, "friends_count": 3094, "statues_count": 106520 }, "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("2015-11-27T10:00:08.000Z"), "id": 670301093314695169, "text": "@stepbcrowder https://t.co/ctSQ5Gz20C", "in_reply_to_status": 670276774681550848, "in_reply_to_user": 1235420178, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1235420178 }}, "user": { "id": 1890682148, "name": "CSW", "screen_name": "BigDCSW", "lang": "en", "location": "Dallas, TX", "create_at": date("2013-09-21"), "description": "Just a PROUD & liberal Texas man observing the world & raising Heaven & hell along the way. I may be blunt but tough love is sometimes the best medicine. ✌️", "followers_count": 956, "friends_count": 1764, "statues_count": 3769 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paloma Creek, TX", "id": "0123115a0c1b23ed", "name": "Paloma Creek", "place_type": "city", "bounding_box": rectangle("-96.946655,33.220749 -96.929913,33.22994") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4854868, "cityName": "Paloma Creek" } }
+{ "create_at": datetime("2015-11-27T10:00:08.000Z"), "id": 670301093373435904, "text": "Wind 7.0 mph E. Barometer 30.133 in, Falling slowly. Temperature 74.3 °F. Rain today 0.16in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.11555556,29.6725"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25210383, "name": "New Braunfels WX", "screen_name": "jeffreycentexwx", "lang": "en", "location": "New Braunfels, TX", "create_at": date("2009-03-18"), "description": "Weather Station located on County Line Road in New Braunfels, TX.\r\n\r\nRealtime Weather @ http://www.nbtx-wx.com\r\nFollow me @ http://twitter.com/jeffreycentex", "followers_count": 370, "friends_count": 22, "statues_count": 156402 }, "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("2015-11-27T10:00:08.000Z"), "id": 670301093390258176, "text": "Lmao and its probably that same exact price any other day. https://t.co/f7BPm0nbqK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 623562069, "name": "Steven", "screen_name": "ILikeMusicYes", "lang": "en", "location": "null", "create_at": date("2012-06-30"), "description": "mitchy, katniss and lifting heavy stuff", "followers_count": 280, "friends_count": 245, "statues_count": 41454 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-27T10:00:08.000Z"), "id": 670301093637713920, "text": "@princesss_aly @fitnessfre ����", "in_reply_to_status": 670300312796721152, "in_reply_to_user": 1321510429, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1321510429, 440604772 }}, "user": { "id": 3017761147, "name": "M.", "screen_name": "_mcast", "lang": "en", "location": "Houston,Tx ", "create_at": date("2015-02-12"), "description": "18. JSH❤️", "followers_count": 336, "friends_count": 272, "statues_count": 4303 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-27T10:00:09.000Z"), "id": 670301093784498176, "text": "Me and my sister in law LORI got Starbucks https://t.co/ec8SejIDqg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 288395058, "name": "maricona", "screen_name": "katyhasley", "lang": "en", "location": "null", "create_at": date("2011-04-26"), "description": "this life is more than ordinary", "followers_count": 156, "friends_count": 72, "statues_count": 20186 }, "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("2015-11-27T10:00:09.000Z"), "id": 670301093792841728, "text": "Idk what to wear to work today help", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1481181889, "name": "Victor", "screen_name": "TheThiccest", "lang": "en", "location": "Puyallup, WA", "create_at": date("2013-06-03"), "description": "With this hand I will lift your sorrows.", "followers_count": 1066, "friends_count": 575, "statues_count": 30848 }, "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("2015-11-27T10:00:09.000Z"), "id": 670301093822267392, "text": "No @GregCapullo #DKIII covers at the local shop ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DKIII" }}, "user_mentions": {{ 410089108 }}, "user": { "id": 48159920, "name": "K", "screen_name": "KarlAlden", "lang": "en", "location": "An abandoned theater", "create_at": date("2009-06-17"), "description": "I directed The Matrix.", "followers_count": 317, "friends_count": 109, "statues_count": 13715 }, "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("2015-11-27T10:00:09.000Z"), "id": 670301094074011648, "text": "@ATrizzay bitch", "in_reply_to_status": 670135067369435136, "in_reply_to_user": 116044081, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 116044081 }}, "user": { "id": 709556695, "name": "YUNG Breeze", "screen_name": "AaronRies1", "lang": "en", "location": "cincinnati, OH", "create_at": date("2012-07-21"), "description": "my name is YUNG Breeze/Memorize your Memories/Im a 17 year old rap artist and I #skateeverydamnday!", "followers_count": 340, "friends_count": 213, "statues_count": 4622 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Villa Hills, KY", "id": "b7bd48b75b8a55d6", "name": "Villa Hills", "place_type": "city", "bounding_box": rectangle("-84.623004,39.045091 -84.561637,39.080773") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21117, "countyName": "Kenton", "cityID": 2179698, "cityName": "Villa Hills" } }
+{ "create_at": datetime("2015-11-27T10:00:09.000Z"), "id": 670301094120034304, "text": "I'd like Forte on the Cowboys", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1177146896, "name": "Ryan D. Fort", "screen_name": "RyanDFort", "lang": "en", "location": "Garland Texas", "create_at": date("2013-02-13"), "description": "Sports Head.... FortOnSports owner & founder. All about the art of journalism & sports writing.", "followers_count": 886, "friends_count": 778, "statues_count": 55456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kilgore, TX", "id": "96bc1d9f5b8fe538", "name": "Kilgore", "place_type": "city", "bounding_box": rectangle("-94.916194,32.353319 -94.806957,32.425799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48183, "countyName": "Gregg", "cityID": 4839124, "cityName": "Kilgore" } }
+{ "create_at": datetime("2015-11-27T10:00:09.000Z"), "id": 670301094132715521, "text": "My name is Abby and I'm the worst person alive ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 347896489, "name": "Abby", "screen_name": "abbyabrams_", "lang": "en", "location": "Geneva, NY", "create_at": date("2011-08-03"), "description": "HWS 2019", "followers_count": 317, "friends_count": 386, "statues_count": 5456 }, "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("2015-11-27T10:00:09.000Z"), "id": 670301094145323008, "text": "Wind 2.0 mph SW. Barometer 30.424 in, Falling Rapidly. Temperature 66.1F. Rain today 0.01 in. Humidity 61% | 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": 566, "friends_count": 762, "statues_count": 34177 }, "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("2015-11-27T10:00:09.000Z"), "id": 670301094296195073, "text": "Still laughing about @pkristin74 buying 10 wake up wraps from Dunkin Donuts last night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2781659236 }}, "user": { "id": 347351892, "name": "Shannon Coyle", "screen_name": "ShanShanCoyle", "lang": "en", "location": "Oak Forest, IL", "create_at": date("2011-08-02"), "description": "Dubuque, IA • Loras College '19", "followers_count": 775, "friends_count": 540, "statues_count": 13562 }, "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("2015-11-27T10:00:09.000Z"), "id": 670301094447325185, "text": "it's now or never\nif I wait will I forgive myself?\nto offer it to you would be cruel\ncause all I want to do is use you,\nuse you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 592167771, "name": "Michelle Caldera", "screen_name": "MichelleCalde17", "lang": "en", "location": "Miami, FL", "create_at": date("2012-05-27"), "description": "| 21 | FIU | ♋️", "followers_count": 266, "friends_count": 139, "statues_count": 17977 }, "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("2015-11-27T10:00:09.000Z"), "id": 670301094807924736, "text": "Join the Oracle team! See our latest #Sales #job opening here: https://t.co/RrssawFGsb #SantaClara, CA #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.9552356,37.3541079"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "job", "SantaClara", "Hiring", "CareerArc" }}, "user": { "id": 21712562, "name": "TMJ - SJC Sales Jobs", "screen_name": "tmj_sjc_sales", "lang": "en", "location": "San Jose, CA", "create_at": date("2009-02-23"), "description": "Follow this account for geo-targeted Sales job tweets in San Jose, CA. Need help? Tweet us at @CareerArc!", "followers_count": 447, "friends_count": 312, "statues_count": 466 }, "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("2015-11-27T10:00:09.000Z"), "id": 670301095676129280, "text": "Now open! #BlackFridaySale! #Buy1Get1 #BlackAndGrayTattoo #TribalTattoo #LetteringTattoo #BlackFridayTattooSpecial https://t.co/YFFmBgA55e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BlackFridaySale", "Buy1Get1", "BlackAndGrayTattoo", "TribalTattoo", "LetteringTattoo", "BlackFridayTattooSpecial" }}, "user": { "id": 2243535896, "name": "Mystical Ink Tattoo", "screen_name": "Mystical_Ink", "lang": "en", "location": "2544 E. Charleston, Las Vegas", "create_at": date("2013-12-12"), "description": "Tattoo and Piercing shop, Las Vegas, Nevada", "followers_count": 20, "friends_count": 35, "statues_count": 124 }, "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("2015-11-27T10:00:09.000Z"), "id": 670301095810482177, "text": "Never �� @Nanarosario_", "in_reply_to_status": 670300961282347008, "in_reply_to_user": 195110483, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 195110483 }}, "user": { "id": 435415983, "name": "D", "screen_name": "Fly_kidd_11", "lang": "en", "location": "The 7 . Pennsylvania ", "create_at": date("2011-12-12"), "description": "#SHIPU⛵️ | #OTG | #RIPBIGBRO", "followers_count": 1919, "friends_count": 957, "statues_count": 45254 }, "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("2015-11-27T10:00:09.000Z"), "id": 670301096028442624, "text": "alright lettuce head", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 952971193, "name": "briaNNa", "screen_name": "briannaaav", "lang": "en", "location": "San Antonio, TX", "create_at": date("2012-11-16"), "description": "@kendalljenner & @maiamitchell 's love child.", "followers_count": 2861, "friends_count": 492, "statues_count": 26391 }, "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-27T10:00:09.000Z"), "id": 670301096103931904, "text": "no ceilings 2 >", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 330555461, "name": "PHïL", "screen_name": "_PimpP", "lang": "en", "location": "Orange, Tx", "create_at": date("2011-07-06"), "description": "RTC ❤️", "followers_count": 2644, "friends_count": 943, "statues_count": 79826 }, "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("2015-11-27T10:00:09.000Z"), "id": 670301096213131265, "text": "impulsively bought a camera for $400 haha rip bank account :-)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 762726354, "name": "Kate Agnew", "screen_name": "katexvx", "lang": "en", "location": "philadelphia", "create_at": date("2012-08-16"), "description": "42, drives a dodge stratus, manager at the local smoke shop", "followers_count": 80, "friends_count": 171, "statues_count": 1080 }, "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("2015-11-27T10:00:09.000Z"), "id": 670301096305229824, "text": "Now that Thanksgiving is over....������ @ Glendale Springs, North Carolina https://t.co/Uo0SNqs96T", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3806,36.345"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 232936334, "name": "Kendra Stowe", "screen_name": "KendraStowe", "lang": "en", "location": "null", "create_at": date("2011-01-01"), "description": "Cam Newton Foundation. All things NC State. Freelancer for http://ReadThePoint.com. Trying to afford rent one day soon.", "followers_count": 641, "friends_count": 529, "statues_count": 6976 }, "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.32187,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37193, "countyName": "Wilkes" } }
+{ "create_at": datetime("2015-11-27T10:00:09.000Z"), "id": 670301096368193537, "text": "@10chavezsuzy my baby �� miss you", "in_reply_to_status": 670158858044633088, "in_reply_to_user": 3184616898, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3184616898 }}, "user": { "id": 2937752238, "name": "leslie", "screen_name": "lpslie28", "lang": "en", "location": "Pico Rivera, CA", "create_at": date("2014-12-20"), "description": "null", "followers_count": 330, "friends_count": 248, "statues_count": 5023 }, "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-27T10:00:09.000Z"), "id": 670301096519315457, "text": "Tyrus is still sleeping. 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": 2413974143, "name": "taelur", "screen_name": "TaelurVargo", "lang": "en", "location": "Pennsylvania", "create_at": date("2014-03-15"), "description": "God grant me the Serenity to accept the things I cannot change the Courage to change the things I can Wisdom to know the difference.\n11.7.14♡", "followers_count": 355, "friends_count": 415, "statues_count": 6166 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin, PA", "id": "6fedd7dab30dc35f", "name": "Franklin", "place_type": "city", "bounding_box": rectangle("-78.895516,40.333391 -78.874885,40.353763") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42121, "countyName": "Venango", "cityID": 4227456, "cityName": "Franklin" } }
+{ "create_at": datetime("2015-11-27T10:00:09.000Z"), "id": 670301096653533184, "text": "When will someone like me for my personality and not my hot bod ;-(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 218676744, "name": "sam", "screen_name": "sam0ns", "lang": "en", "location": "strawberry fields", "create_at": date("2010-11-22"), "description": "⚡️", "followers_count": 941, "friends_count": 438, "statues_count": 28777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bayonet Point, FL", "id": "2d1b0aee7297f004", "name": "Bayonet Point", "place_type": "city", "bounding_box": rectangle("-82.733975,28.299909 -82.642106,28.350175") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12101, "countyName": "Pasco", "cityID": 1204162, "cityName": "Bayonet Point" } }
+{ "create_at": datetime("2015-11-27T10:00:09.000Z"), "id": 670301097039237120, "text": "Check out our #listing in #DefuniakSprings #FL https://t.co/1x40AWF0GH #realestate #realtor https://t.co/zqDVbJXDQK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.249291,30.830953"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "DefuniakSprings", "FL", "realestate", "realtor" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6580 }, "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": 12131, "countyName": "Walton" } }
+{ "create_at": datetime("2015-11-27T10:00:09.000Z"), "id": 670301097039302656, "text": "Me every day lol https://t.co/auJuBH8VoK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 262457353, "name": "KP2Trill", "screen_name": "kaylapilant", "lang": "en", "location": "SaTX", "create_at": date("2011-03-07"), "description": "Stay gold college lyfe ulta 360 with the wrist boi Texas", "followers_count": 945, "friends_count": 866, "statues_count": 31130 }, "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-27T10:00:09.000Z"), "id": 670301097097981952, "text": "reading Drop The Rock rn and loving it~ https://t.co/4mmG1cNLYe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1173667878, "name": "Audrey Cathleen", "screen_name": "apearlpicnic", "lang": "en", "location": "San Antonio, TX", "create_at": date("2013-02-12"), "description": "Oh just cosmic dust and other stuff. I'm a collector, like you. I like jokes, so tell me jokes. Bee Aware~ #tellmetuesday", "followers_count": 195, "friends_count": 596, "statues_count": 6601 }, "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-27T10:00:09.000Z"), "id": 670301097165123584, "text": "We're #hiring! Click to apply: Advanced Manager - CMV - https://t.co/ybLlQyxVDV #BusinessMgmt #cintasjobs #Lawton, OK #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.3903305,34.6086854"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "BusinessMgmt", "cintasjobs", "Lawton", "Job", "Jobs" }}, "user": { "id": 78995297, "name": "Cintas Careers", "screen_name": "CintasCareers", "lang": "en", "location": "null", "create_at": date("2009-10-01"), "description": "Looking for a career, and not just another job? Consider Cintas, where the spirit is the difference. Follow us for our latest career opportunities.", "followers_count": 239, "friends_count": 2, "statues_count": 9736 }, "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("2015-11-27T10:00:09.000Z"), "id": 670301097202987008, "text": "Love watching movies that are actually good", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3515152213, "name": "Childish Camdino", "screen_name": "camden_melo", "lang": "en", "location": "Dartmouth, MA", "create_at": date("2015-09-10"), "description": "Everyday is a new day", "followers_count": 54, "friends_count": 160, "statues_count": 118 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Smith Mills, MA", "id": "db0dcea533e078a2", "name": "Smith Mills", "place_type": "city", "bounding_box": rectangle("-71.047745,41.619498 -70.958622,41.680953") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25005, "countyName": "Bristol", "cityID": 2562300, "cityName": "Smith Mills" } }
+{ "create_at": datetime("2015-11-27T10:00:09.000Z"), "id": 670301097223782400, "text": "Circus by Britney Spears will always be great��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1476278106, "name": "Hannah", "screen_name": "annahhmarie", "lang": "en", "location": "null", "create_at": date("2013-06-01"), "description": "I once was lost but now I am found", "followers_count": 338, "friends_count": 264, "statues_count": 4011 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodway, TX", "id": "be425fec510e15e5", "name": "Woodway", "place_type": "city", "bounding_box": rectangle("-97.271603,31.481764 -97.18932,31.52128") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4880224, "cityName": "Woodway" } }
+{ "create_at": datetime("2015-11-27T10:00:09.000Z"), "id": 670301097416916992, "text": "@RickyPDillon @andrewlx lmao", "in_reply_to_status": 670300618167160832, "in_reply_to_user": 73171449, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 73171449, 238834602 }}, "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": 1315, "friends_count": 2557, "statues_count": 104458 }, "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("2015-11-27T10:00:09.000Z"), "id": 670301097462882304, "text": "@th3olivesbranch https://t.co/Qp66isWorQ Thinly sliced chicken breast and thigh marinated in shawarma spices #sliced #menu", "in_reply_to_status": -1, "in_reply_to_user": 824007054, "favorite_count": 0, "coordinate": point("-117.65943289,33.50905311"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sliced", "menu" }}, "user_mentions": {{ 824007054 }}, "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": 4112, "friends_count": 876, "statues_count": 332301 }, "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("2015-11-27T10:00:09.000Z"), "id": 670301097559478272, "text": "when r u gonna tell me y ur happy??", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2898001207, "name": "Orion", "screen_name": "Obonghi_", "lang": "en", "location": "buffalo ny", "create_at": date("2014-11-12"), "description": "snapchat//obonghi", "followers_count": 126, "friends_count": 76, "statues_count": 513 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clarence Center, NY", "id": "01369daa5fd253f9", "name": "Clarence Center", "place_type": "city", "bounding_box": rectangle("-78.696944,42.988257 -78.616849,43.034615") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3615836, "cityName": "Clarence Center" } }
+{ "create_at": datetime("2015-11-27T10:00:09.000Z"), "id": 670301097576263681, "text": "@s_fidazzy The Office", "in_reply_to_status": 670145713775509506, "in_reply_to_user": 760816567, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 760816567 }}, "user": { "id": 717605994, "name": "Josh Armstrong", "screen_name": "Josh_Armstrong2", "lang": "en", "location": "Greensburg, Pa", "create_at": date("2012-07-26"), "description": "In squad we trust. #D3Wrestling #A1Academics", "followers_count": 747, "friends_count": 529, "statues_count": 8147 }, "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("2015-11-27T10:00:10.000Z"), "id": 670301097987194880, "text": "Who Will Make It This Sunday in The Mid-Season Finale: #TheWalkingDead \n@WalkingDead_AMC https://t.co/8tSp70J0ch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TheWalkingDead" }}, "user_mentions": {{ 65184105 }}, "user": { "id": 276452562, "name": "~Marietta", "screen_name": "LilAssKickerAMC", "lang": "en", "location": "East Tennessee ", "create_at": date("2011-04-03"), "description": "#TWDFamily ❤️ #TheWalkingDead \nI Can tell people are judgmental just by looking at them.", "followers_count": 19716, "friends_count": 19848, "statues_count": 1866 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clinton, TN", "id": "00682d3a69270e5f", "name": "Clinton", "place_type": "city", "bounding_box": rectangle("-84.185072,36.060303 -84.094339,36.141741") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47001, "countyName": "Anderson", "cityID": 4715580, "cityName": "Clinton" } }
+{ "create_at": datetime("2015-11-27T10:00:10.000Z"), "id": 670301098046062592, "text": "@amoneylaviero youre annoying", "in_reply_to_status": 670290315492855809, "in_reply_to_user": 3010633468, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3010633468 }}, "user": { "id": 2359571720, "name": "Harry Pacheco", "screen_name": "pacheco1235", "lang": "en", "location": "Belleville IL", "create_at": date("2014-02-24"), "description": "What happens when an unstopable force meets an immovable object? \n\nFUTURE SOLDIER", "followers_count": 408, "friends_count": 361, "statues_count": 4005 }, "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("2015-11-27T10:00:10.000Z"), "id": 670301098566135808, "text": "�� noooooo https://t.co/2cO1GWJksd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3376463052, "name": ". . . .❤️", "screen_name": "AshlanFields", "lang": "en", "location": "null", "create_at": date("2015-08-28"), "description": "null", "followers_count": 467, "friends_count": 414, "statues_count": 4960 }, "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("2015-11-27T10:00:10.000Z"), "id": 670301098763243521, "text": "@ikilaliki �� im so serious", "in_reply_to_status": 670299814760984576, "in_reply_to_user": 103189426, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 103189426 }}, "user": { "id": 247127855, "name": "c➿lBuckz!", "screen_name": "Coooliee", "lang": "en", "location": "Virginia Beach, VA", "create_at": date("2011-02-03"), "description": "21 ⚠️ rest in peace grandpa, ricky, vell & free prince", "followers_count": 988, "friends_count": 494, "statues_count": 42632 }, "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("2015-11-27T10:00:10.000Z"), "id": 670301098851233792, "text": "that feeling when you come home ���� https://t.co/Dr4GwKF8Rt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 336677056, "name": "mckenna genyn", "screen_name": "heymckky", "lang": "en", "location": "Portland, OR", "create_at": date("2011-07-16"), "description": "Alumni of the University of Oregon @UOCinema | feel goods: Jesus, storytelling, coffee+donuts, Parks&Rec, #YLUO, fresh flowers, and sharpening my wit", "followers_count": 216, "friends_count": 169, "statues_count": 4288 }, "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-27T10:00:10.000Z"), "id": 670301099048353792, "text": "@teenagernotes @nissacampos @MarisolVasquez_ @vgabrielax us", "in_reply_to_status": 670290308312260608, "in_reply_to_user": 148779151, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 148779151, 1059355962, 1184414106, 3076500177 }}, "user": { "id": 2285572260, "name": "kate", "screen_name": "katieberryberry", "lang": "en", "location": "null", "create_at": date("2014-01-10"), "description": "Enrique Tomas Castillo is my baby", "followers_count": 618, "friends_count": 581, "statues_count": 17638 }, "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-27T10:00:10.000Z"), "id": 670301099098832896, "text": "I count my blessings but I suck at math.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 429279467, "name": "Chris", "screen_name": "ChaseMeChris", "lang": "en", "location": "Massachusetts, USA ", "create_at": date("2011-12-05"), "description": "Vibes", "followers_count": 2266, "friends_count": 1441, "statues_count": 7221 }, "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("2015-11-27T10:00:10.000Z"), "id": 670301099543392256, "text": "Wind 6 mph ENE. Barometer 30.13 in, Falling. Temperature 79.0 °F. Rain today 0.00 in. Humidity 41%", "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": 50, "friends_count": 67, "statues_count": 23876 }, "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("2015-11-27T10:00:10.000Z"), "id": 670301099585241088, "text": "Join the Accountable Healthcare Staffing team! See our latest #Nursing #job opening here: https://t.co/2wDbsu7GU3 #Murrieta, CA #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.2139232,33.5539143"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "Murrieta", "Hiring" }}, "user": { "id": 22533900, "name": "TMJ-CA-US Nursing", "screen_name": "tmj_CA_NURSING", "lang": "en", "location": "California", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 465, "friends_count": 308, "statues_count": 2748 }, "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("2015-11-27T10:00:10.000Z"), "id": 670301099794939906, "text": "Mack Brown calling a UH game, the irony", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.18957783,29.919771"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2151630596, "name": "D-R-E", "screen_name": "DRDREon290", "lang": "en", "location": "Houston", "create_at": date("2013-10-23"), "description": "null", "followers_count": 238, "friends_count": 208, "statues_count": 15588 }, "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("2015-11-27T10:00:10.000Z"), "id": 670301100038311937, "text": "I'm at The Gun Range in San Diego, CA https://t.co/Wzi0px7Lv4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.15365518,32.82142551"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 292960093, "name": "Craig Davis", "screen_name": "Salty619", "lang": "en", "location": "San Diego", "create_at": date("2011-05-04"), "description": "null", "followers_count": 75, "friends_count": 219, "statues_count": 1607 }, "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-27T10:00:10.000Z"), "id": 670301100059181056, "text": "@MarketPlunger copious amounts of ass.", "in_reply_to_status": 670301010599010304, "in_reply_to_user": 346257282, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 346257282 }}, "user": { "id": 1928213371, "name": "PsychoOnWallst", "screen_name": "PsychoOnWallSt", "lang": "en", "location": "The BatCave", "create_at": date("2013-10-02"), "description": "People should be ruled by dogs.\nTweeter is not real life.\nI eat, therefore I am big boneded.", "followers_count": 10014, "friends_count": 227, "statues_count": 29882 }, "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-27T10:00:10.000Z"), "id": 670301100340150273, "text": "If you're a #Finance professional in #Covington, LA, check out this #job: https://t.co/hTKBnpJrgT #regions #regionsbank #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.1009108,30.4754702"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Finance", "Covington", "job", "regions", "regionsbank", "Hiring" }}, "user": { "id": 72653327, "name": "Regions Bank", "screen_name": "regionsjobs", "lang": "en", "location": "Birmingham, Alabama", "create_at": date("2009-09-08"), "description": "At Regions, we believe associates deserve more than just a job. We believe in offering performance-driven individuals a place where they can build a career", "followers_count": 773, "friends_count": 45, "statues_count": 11942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covington, LA", "id": "dd239a4b0de5d7b2", "name": "Covington", "place_type": "city", "bounding_box": rectangle("-90.182377,30.440165 -90.080751,30.506931") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22103, "countyName": "St. Tammany", "cityID": 2218125, "cityName": "Covington" } }
+{ "create_at": datetime("2015-11-27T10:00:10.000Z"), "id": 670301100407287808, "text": "@byPresto only got a size 13 for 750", "in_reply_to_status": 670127088712753152, "in_reply_to_user": 2932765186, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2932765186 }}, "user": { "id": 30811334, "name": "CABRINIGREEN", "screen_name": "RonMacFrmCG", "lang": "en", "location": "Wild End Cabrini Green", "create_at": date("2009-04-12"), "description": "*MUSIC ADVISOR* Let Me Write Yo Next Hit! RIP TROYMAC RIP S1 #˩αҒαмιℓια #07042009 IG:RonMacFrmCG", "followers_count": 1224, "friends_count": 2044, "statues_count": 13597 }, "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-27T10:00:10.000Z"), "id": 670301100835123200, "text": "See a virtual tour of our listing on 125 Schipper Rd #DefuniakSprings #FL https://t.co/7UuTnhQX1O #realestate https://t.co/B37A2t7Z7c", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.32683,30.747306"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DefuniakSprings", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6581 }, "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": 12131, "countyName": "Walton" } }
+{ "create_at": datetime("2015-11-27T10:00:10.000Z"), "id": 670301100851916800, "text": "The students from The School of Communication presents: Eye of the Tigers \nCome out Wednesday… https://t.co/CkTPkiajyK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.36114151,29.72203362"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 52456415, "name": "Jade♉️", "screen_name": "JayyThaTruthh", "lang": "en", "location": "Houston, TX", "create_at": date("2009-06-30"), "description": "College Senior. #TxSU16 #FemaleDJ", "followers_count": 1081, "friends_count": 1685, "statues_count": 10706 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-27T10:00:10.000Z"), "id": 670301101191639040, "text": "So today I've spun out, had pop explode in my car and scrapped the side of my front bumper. Today is fun��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 480958904, "name": "kathleen", "screen_name": "kathleen_boggs", "lang": "en", "location": "dinkytown", "create_at": date("2012-02-01"), "description": "mn // university of minnesota // dog enthusiast // coffee junkie", "followers_count": 523, "friends_count": 358, "statues_count": 7759 }, "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("2015-11-27T10:00:10.000Z"), "id": 670301101220990976, "text": "When I sleep in until 12 I feel like I've wasted the whole day ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 369898473, "name": "Briggette", "screen_name": "Briggette121", "lang": "en", "location": "Omaha, NE", "create_at": date("2011-09-07"), "description": "Snapchat: briggette12", "followers_count": 437, "friends_count": 380, "statues_count": 9282 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Island, NE", "id": "df97a7d96847d923", "name": "Grand Island", "place_type": "city", "bounding_box": rectangle("-98.435794,40.873321 -98.263776,40.988846") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31079, "countyName": "Hall", "cityID": 3119595, "cityName": "Grand Island" } }
+{ "create_at": datetime("2015-11-27T10:00:10.000Z"), "id": 670301101548138497, "text": "Its the busiest shopping day of the year come be a part of it and join us. #shopsmall #shoplocal @… https://t.co/POPaKlOnRg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.86441451,21.31122072"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "shopsmall", "shoplocal" }}, "user": { "id": 244857048, "name": "H&Q", "screen_name": "HOUNDandQUAIL", "lang": "en", "location": "Honolulu", "create_at": date("2011-01-30"), "description": "null", "followers_count": 131, "friends_count": 22, "statues_count": 978 }, "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("2015-11-27T10:00:10.000Z"), "id": 670301101686558721, "text": "Nobody know that I'm from Compton", "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": "HHS || SC: love_brookiiee || Packers", "followers_count": 399, "friends_count": 371, "statues_count": 4080 }, "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("2015-11-27T10:00:10.000Z"), "id": 670301102017896448, "text": "Freezing rain/wind this afternoon, high 31 (-1 C). Low 28 (-2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712679, "name": "OKCWX", "screen_name": "OKCWX", "lang": "en", "location": "Oklahoma City, Oklahoma", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Oklahoma City, Oklahoma", "followers_count": 175, "friends_count": 93, "statues_count": 8089 }, "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-27T10:00:11.000Z"), "id": 670301102282252290, "text": "So I went Black Friday shopping... https://t.co/C085MQpC9n", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2329350532, "name": "Elaine", "screen_name": "00elaine000", "lang": "en", "location": "null", "create_at": date("2014-02-06"), "description": "null", "followers_count": 243, "friends_count": 333, "statues_count": 5656 }, "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("2015-11-27T10:00:11.000Z"), "id": 670301102382927872, "text": "*uses fake id to play w puppies bc u must be 21 to take into play room*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 352829017, "name": "Peyton! At The Disco", "screen_name": "peytongavlick", "lang": "en", "location": "CULVER | BGSU | ΑΞΔ", "create_at": date("2011-08-10"), "description": "everything in life is just for a while", "followers_count": 607, "friends_count": 474, "statues_count": 16414 }, "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("2015-11-27T10:00:11.000Z"), "id": 670301102856736769, "text": "@lunalizama1 wyd ?!", "in_reply_to_status": 670300931708194816, "in_reply_to_user": 2829168468, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2829168468 }}, "user": { "id": 330655954, "name": "SNOWMANtha⛄️", "screen_name": "SamanthaVirgen", "lang": "en", "location": "South Sioux City, NE", "create_at": date("2011-07-06"), "description": "@drake | snapchat & insta: samantha_virgen", "followers_count": 1868, "friends_count": 674, "statues_count": 84481 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Sioux City, NE", "id": "4539fa8c44fc9f3f", "name": "South Sioux City", "place_type": "city", "bounding_box": rectangle("-96.451509,42.429592 -96.381282,42.48898") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31043, "countyName": "Dakota", "cityID": 3146030, "cityName": "South Sioux City" } }
+{ "create_at": datetime("2015-11-27T10:00:11.000Z"), "id": 670301102928211968, "text": "@karlaarnau se ve que si, se ve bonito así natural. habla con profesionales y mete mano total al final crece de nuevo!", "in_reply_to_status": 670300382812241920, "in_reply_to_user": 729229634, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 729229634 }}, "user": { "id": 112638738, "name": "Inmoral ♏️", "screen_name": "elbigboss22", "lang": "en", "location": "Unmillonario.com ", "create_at": date("2010-02-08"), "description": "Me da igual todo... De Puerto Rico para el mundo entero... Puedo ser el mejor o el peor. \n\nkik: jrmelendez SC: joshuamj IG: elbigboss22", "followers_count": 260, "friends_count": 191, "statues_count": 29474 }, "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("2015-11-27T10:00:11.000Z"), "id": 670301102999375872, "text": "Join the Hair Cuttery Salon team! See our latest #Cosmetology #job opening here: https://t.co/fTSkXJ66OV #STUART, FL #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.2264149,27.1646635"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Cosmetology", "job", "STUART", "Hiring" }}, "user": { "id": 4183786094, "name": "@haircutteryjob", "screen_name": "haircutteryjob", "lang": "en", "location": "null", "create_at": date("2015-11-09"), "description": "Hair Cuttery was founded with a simple idea – to create an environment that celebrates stylists’ talents while encouraging whole-person career development.", "followers_count": 5, "friends_count": 1, "statues_count": 728 }, "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("2015-11-27T10:00:11.000Z"), "id": 670301103225970688, "text": "#flashbackfriday to the time that I spoke to the future beauty professionals at #empirebeautyschool… https://t.co/NjlRclfKdd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.04263413,40.78821837"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "flashbackfriday", "empirebeautyschool" }}, "user": { "id": 17487015, "name": "Tamara Delbridge", "screen_name": "tamaradelbridge", "lang": "en", "location": "ÜT: 40.708468,-74.07273", "create_at": date("2008-11-19"), "description": "Tamara Delbridge- Celebrity makeup artist, member of makeup artist union I.A.T.S.E Local 798", "followers_count": 2496, "friends_count": 1765, "statues_count": 4459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Secaucus, NJ", "id": "c5e2e95296cb121d", "name": "Secaucus", "place_type": "city", "bounding_box": rectangle("-74.093886,40.750391 -74.033688,40.806587") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3466570, "cityName": "Secaucus" } }
+{ "create_at": datetime("2015-11-27T10:00:11.000Z"), "id": 670301103297294336, "text": "Just light up https://t.co/V9QBP4hyzO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 311076102, "name": "chef buddah", "screen_name": "YoBuddahSparkUP", "lang": "en", "location": "lost", "create_at": date("2011-06-04"), "description": "BU$HLeauge | certified chef | personal chef | party caterer | Le Cordon Bleu Culinary Arts Institute Graduate", "followers_count": 2095, "friends_count": 411, "statues_count": 182364 }, "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("2015-11-27T10:00:11.000Z"), "id": 670301103402016768, "text": "Listening to The Smiths (certain songs tho) The Cure and John Maus with the old lady. Appreciating the day #BlackFriday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BlackFriday" }}, "user": { "id": 29619901, "name": "A. Partí", "screen_name": "PartiPants27", "lang": "en", "location": "Long Beach, CA", "create_at": date("2009-04-07"), "description": "Just an extra ordinary dude tweeting up a storm of extra ordinary things like Music,Boxing,Burritos etc. Instagram & Snapchat: PartiPants27", "followers_count": 379, "friends_count": 300, "statues_count": 37562 }, "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("2015-11-27T10:00:11.000Z"), "id": 670301103590887428, "text": "Day 2 without coffee. This situation is getting a little dangerous ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 400419092, "name": "Katie Meredith", "screen_name": "kaymurdiff", "lang": "en", "location": "Spartanburg, SC", "create_at": date("2011-10-28"), "description": "You are my hiding place and my shield; I hope in Your Word. Psalm 119:114", "followers_count": 379, "friends_count": 342, "statues_count": 9894 }, "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("2015-11-27T10:00:11.000Z"), "id": 670301103800590337, "text": "Dumb ass question.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1122442465, "name": "BABYGIRL", "screen_name": "yung_belleezy", "lang": "en", "location": "Dubai, United Arab Emirates", "create_at": date("2013-01-26"), "description": "null", "followers_count": 1512, "friends_count": 989, "statues_count": 40060 }, "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-27T10:00:11.000Z"), "id": 670301103817256960, "text": "update: im barely alive and it's only 11", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 547963391, "name": "Kel-C", "screen_name": "kelsey__fitz", "lang": "en", "location": "new mexico", "create_at": date("2012-04-07"), "description": "hi im kelsey and im just a mess of person who can't get their life together.", "followers_count": 553, "friends_count": 890, "statues_count": 9767 }, "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("2015-11-27T10:00:11.000Z"), "id": 670301104253480960, "text": "@rinconsalma lol go high more often then :p", "in_reply_to_status": 670298937366966272, "in_reply_to_user": 3359217978, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3359217978 }}, "user": { "id": 1036683686, "name": "Dario El Buchon", "screen_name": "NegroMagikk", "lang": "en", "location": "Desert Hot Springs, CA", "create_at": date("2012-12-26"), "description": "20 | Soccer is life ⚽ | Follow me ✌", "followers_count": 970, "friends_count": 1065, "statues_count": 56758 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Desert Hot Springs, CA", "id": "4ead8537112abb01", "name": "Desert Hot Springs", "place_type": "city", "bounding_box": rectangle("-116.57552,33.917616 -116.457506,33.991559") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 618996, "cityName": "Desert Hot Springs" } }
+{ "create_at": datetime("2015-11-27T10:00:11.000Z"), "id": 670301104396218369, "text": "Truth be told God been looking out so much that I feel guilty when I ask him for more", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 342861037, "name": "LoneWolf...", "screen_name": "ViccasoGenius_", "lang": "en", "location": "Jersey ✈️ Maryland ", "create_at": date("2011-07-26"), "description": "Here Physically...", "followers_count": 2321, "friends_count": 732, "statues_count": 131280 }, "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("2015-11-27T10:00:11.000Z"), "id": 670301104488382464, "text": "so basically i need today to go 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": 3395348173, "name": "AshwCa$h", "screen_name": "Ash_inabottle", "lang": "en", "location": "South Los Angeles, Los Angeles", "create_at": date("2015-08-30"), "description": "Anthropologist • Forever young • Sunset lover", "followers_count": 197, "friends_count": 155, "statues_count": 3054 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-27T10:00:11.000Z"), "id": 670301104505155586, "text": "Drum set of the day... https://t.co/WXpjGPEt6I", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 40764296, "name": "Daniel Krajeski", "screen_name": "DanielKrajeski", "lang": "en", "location": "Portland, OR", "create_at": date("2009-05-17"), "description": "Enjoy playing my drums, golf, and classic cars.", "followers_count": 162, "friends_count": 316, "statues_count": 27433 }, "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("2015-11-27T10:00:11.000Z"), "id": 670301104614207488, "text": "@Shannon_Oatman just called me caramel love. Not even mad.", "in_reply_to_status": -1, "in_reply_to_user": 2183607895, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2183607895 }}, "user": { "id": 419070539, "name": "becca ♡", "screen_name": "beckyboo_18", "lang": "en", "location": "KU", "create_at": date("2011-11-22"), "description": "remember to love yourself.", "followers_count": 391, "friends_count": 343, "statues_count": 4402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newton, KS", "id": "fa8690b2c9ddb059", "name": "Newton", "place_type": "city", "bounding_box": rectangle("-97.377877,37.999177 -97.309268,38.071764") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20079, "countyName": "Harvey", "cityID": 2050475, "cityName": "Newton" } }
+{ "create_at": datetime("2015-11-27T10:00:11.000Z"), "id": 670301104643641344, "text": "#Repost britnieturner \n\nMy father the #SithLord \"Do Ittt\" @ Fernandina… https://t.co/eK856EBzI1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.46540836,30.67117055"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Repost", "SithLord" }}, "user": { "id": 620239971, "name": "Kali Turner", "screen_name": "kaley_turner", "lang": "en", "location": "North Augusta, South Carolina ", "create_at": date("2012-06-27"), "description": "She's living in a world and it's on fire ✝", "followers_count": 480, "friends_count": 341, "statues_count": 4685 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fernandina Beach, FL", "id": "018c7487414ba078", "name": "Fernandina Beach", "place_type": "city", "bounding_box": rectangle("-81.483315,30.598822 -81.426432,30.705597") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12089, "countyName": "Nassau", "cityID": 1222175, "cityName": "Fernandina Beach" } }
+{ "create_at": datetime("2015-11-27T10:00:11.000Z"), "id": 670301104673001472, "text": "I'm so stressed all I want is to be with 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": 1282227494, "name": "Brayden Davis", "screen_name": "Brayden_Davis13", "lang": "en", "location": "Texas, USA", "create_at": date("2013-03-19"), "description": "Alyssa Breshay Luna is mine❤️", "followers_count": 217, "friends_count": 282, "statues_count": 482 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bastrop, TX", "id": "7d4a036b2ce1798b", "name": "Bastrop", "place_type": "city", "bounding_box": rectangle("-97.369763,30.090081 -97.277272,30.136312") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48021, "countyName": "Bastrop", "cityID": 4805864, "cityName": "Bastrop" } }
+{ "create_at": datetime("2015-11-27T10:00:11.000Z"), "id": 670301104673054720, "text": "#BuenViernes Solo 12% de Diputados del PRI presentaron iniciativas en septiembre. Video https://t.co/QkKIasx7gq RT https://t.co/1fR2YZKm9y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "BuenViernes" }}, "user": { "id": 3280268953, "name": "Parlamento y Debate", "screen_name": "ParlayDebate", "lang": "es", "location": "Mexico, ME", "create_at": date("2015-07-14"), "description": "Información Parlamentaria para masters, analistas, expertos y novice researcher. Difundimos lo que medios tradicionales no publican. #Follow", "followers_count": 2856, "friends_count": 3283, "statues_count": 2737 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mexico, PA", "id": "005ad92252b34585", "name": "Mexico", "place_type": "city", "bounding_box": rectangle("-77.367717,40.533752 -77.352142,40.55465") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42067, "countyName": "Juniata", "cityID": 4248904, "cityName": "Mexico" } }
+{ "create_at": datetime("2015-11-27T10:00:11.000Z"), "id": 670301104702271488, "text": "Afternoon thunderstorms this afternoon, high 76 (24 C). Low 50 (10 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712488, "name": "SanAntonioWX", "screen_name": "SanAntonioWX", "lang": "en", "location": "San Antonio, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for San Antonio, Texas", "followers_count": 245, "friends_count": 93, "statues_count": 8079 }, "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-27T10:00:11.000Z"), "id": 670301104744374272, "text": "@Tim_Riggins_ you just had a whole margarita machine !", "in_reply_to_status": 670297032326680576, "in_reply_to_user": 171306856, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 171306856 }}, "user": { "id": 163190638, "name": "Tresiah", "screen_name": "TreyTheJedi", "lang": "en", "location": "The Hidden Sauce Village", "create_at": date("2010-07-05"), "description": "College dropout but Kanye West is my role model so my life is on track", "followers_count": 1753, "friends_count": 994, "statues_count": 203770 }, "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("2015-11-27T10:00:11.000Z"), "id": 670301104953950208, "text": "CVS Health: Retail Store Shift Supervisor (#CUMBERLAND, RI) https://t.co/7OGRHJeogO #Retail #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.4328363,41.9667656"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CUMBERLAND", "Retail", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 59988957, "name": "TMJ-RI Retail Jobs", "screen_name": "tmj_ri_retail", "lang": "en", "location": "Rhode Island", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in Rhode Island Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 349, "friends_count": 298, "statues_count": 93 }, "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": 44007, "countyName": "Providence" } }
+{ "create_at": datetime("2015-11-27T10:00:11.000Z"), "id": 670301105025392640, "text": "@CatTipper #PawFriday wow! sweet!", "in_reply_to_status": 670293919876390912, "in_reply_to_user": 91823460, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PawFriday" }}, "user_mentions": {{ 91823460 }}, "user": { "id": 35033769, "name": "CorreyHopeTravel", "screen_name": "CorreyHope", "lang": "en", "location": "here there & everywhere NY", "create_at": date("2009-04-24"), "description": "⚓⌚️time2travel⏱️Looking for a trip, no hassles, with a reliable agent?Look no further. Mom and Special Education Teacher.Blogger of Travel & ED", "followers_count": 3591, "friends_count": 3429, "statues_count": 124545 }, "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-27T10:00:11.000Z"), "id": 670301105067167744, "text": "Check out our #listing in #Crestview #FL https://t.co/Mmm0PNzlgs #realestate #realtor https://t.co/zCrWK0ANdy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.59603,30.77955"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "Crestview", "FL", "realestate", "realtor" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crestview, FL", "id": "32bb9ace0723336d", "name": "Crestview", "place_type": "city", "bounding_box": rectangle("-86.635021,30.695757 -86.534517,30.805126") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1215475, "cityName": "Crestview" } }
+{ "create_at": datetime("2015-11-27T10:00:11.000Z"), "id": 670301105457405953, "text": "Wasted so much time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4056897124, "name": "Kake❣", "screen_name": "kayjimenez_", "lang": "en", "location": "my fridge", "create_at": date("2015-10-27"), "description": "the names kayla..", "followers_count": 237, "friends_count": 366, "statues_count": 2578 }, "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("2015-11-27T10:00:11.000Z"), "id": 670301105725681664, "text": "I just realized this MF played 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": 56291716, "name": "LeahAngelie", "screen_name": "Leehah88", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-07-12"), "description": "RnBass @RnBassMusic IG:Leehah88 SocialMedia Management for @RoseWoodRadio LosAngeles @DunkXChange Ambassador", "followers_count": 1139, "friends_count": 733, "statues_count": 36304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-27T10:00:11.000Z"), "id": 670301105755045888, "text": "Want to work at Starbucks? We're #hiring in ! Click for details: https://t.co/aFGKORfbyM #Hospitality #Veterans #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-107.7765322,39.5224987"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Hospitality", "Veterans", "Job", "Jobs", "CareerArc" }}, "user": { "id": 88075628, "name": "CO Hospitality Jobs", "screen_name": "tmj_co_hrta", "lang": "en", "location": "Colorado", "create_at": date("2009-11-06"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Colorado Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 491, "friends_count": 321, "statues_count": 1216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rifle, CO", "id": "7ed97e4d10dc910c", "name": "Rifle", "place_type": "city", "bounding_box": rectangle("-107.81853,39.518873 -107.729209,39.574798") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8045, "countyName": "Garfield", "cityID": 864255, "cityName": "Rifle" } }
+{ "create_at": datetime("2015-11-27T10:00:11.000Z"), "id": 670301105885044736, "text": "Who's actually shopping today? #BlackFriday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BlackFriday" }}, "user": { "id": 485427219, "name": "PiRATE TV", "screen_name": "GrigorAdamian", "lang": "en", "location": "Los Angeles, CA.", "create_at": date("2012-02-06"), "description": "YOU ARE NOW TUNED IN... #BlackMafia", "followers_count": 53373, "friends_count": 17598, "statues_count": 5878 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-27T10:00:11.000Z"), "id": 670301106107342849, "text": "Woke up full , haha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 171088644, "name": "⚡addicted2ink", "screen_name": "JayFromThe90s_", "lang": "en", "location": "Portland, OR", "create_at": date("2010-07-26"), "description": "I always spread love but sometimes I sin / Unprofessional skateboarder / @nikesb / music makes the world go ✌️", "followers_count": 2195, "friends_count": 1947, "statues_count": 22721 }, "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("2015-11-27T10:00:11.000Z"), "id": 670301106279288832, "text": "The stache was supposed to just be a 1-day thing. But then I thought to myself...what would @JordanWesty1 do? #WWJWD https://t.co/jeT5TSFsP5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.95132734,40.68512043"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WWJWD" }}, "user_mentions": {{ 309314422 }}, "user": { "id": 26384910, "name": "Jake Strawhecker", "screen_name": "thefinalSTRAW", "lang": "en", "location": "Holdrege, NE", "create_at": date("2009-03-24"), "description": "Galations 2:20 Christ-follower, Navigator, Husker, tennis player, sports fanatic, drummer, music lover...striving to know Christ and make Him known", "followers_count": 879, "friends_count": 2021, "statues_count": 22371 }, "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.999719 -95.30829,43.001708") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31099, "countyName": "Kearney" } }
+{ "create_at": datetime("2015-11-27T10:00:12.000Z"), "id": 670301106484985857, "text": "@TheJHilI getcha ass up I'm bout to shower then we out", "in_reply_to_status": 670300894035091457, "in_reply_to_user": 532858468, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 532858468 }}, "user": { "id": 283718865, "name": "TK #GiLU", "screen_name": "T_Kelllz", "lang": "en", "location": "Detroit, MI", "create_at": date("2011-04-17"), "description": "My reputations on the line, so I'm working on a better me. #GiLU RMR COD and Music consume all of me. PSN: T_Kelly313", "followers_count": 1990, "friends_count": 812, "statues_count": 106280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Royal Oak, MI", "id": "47cfe29f57708110", "name": "Royal Oak", "place_type": "city", "bounding_box": rectangle("-83.204418,42.474131 -83.111076,42.540305") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2670040, "cityName": "Royal Oak" } }
+{ "create_at": datetime("2015-11-27T10:00:12.000Z"), "id": 670301106585473024, "text": "@SamoraSloan @RapFavorites nice*", "in_reply_to_status": 670301039552131072, "in_reply_to_user": 501443984, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2408895999, 2604017857 }}, "user": { "id": 501443984, "name": "Jamal Williams Jr.", "screen_name": "JaYeWiLL90", "lang": "en", "location": "San Antonio, TX - Seguin, TX ", "create_at": date("2012-02-23"), "description": "TLU Bulldog C/O19 | MacArthur Alumni", "followers_count": 631, "friends_count": 356, "statues_count": 21467 }, "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("2015-11-27T10:00:12.000Z"), "id": 670301106631766016, "text": "It’s 73°F. It’s the Friday after Thankgsiving, three days before December. What is happening?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 5979042, "name": "Ian Sheddan", "screen_name": "imsvsims", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2007-05-11"), "description": "It's just me vs. myself.", "followers_count": 98, "friends_count": 110, "statues_count": 63499 }, "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("2015-11-27T10:00:12.000Z"), "id": 670301107156066304, "text": "@RealZachCochran yeah my native homeland", "in_reply_to_status": 670272759952498688, "in_reply_to_user": 595058777, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 595058777 }}, "user": { "id": 389433601, "name": "Forrest", "screen_name": "TheStonemeister", "lang": "en", "location": "The Basement", "create_at": date("2011-10-12"), "description": "#som9", "followers_count": 479, "friends_count": 348, "statues_count": 16504 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Intercourse, PA", "id": "0182301b6c7757c0", "name": "Intercourse", "place_type": "city", "bounding_box": rectangle("-76.118058,40.027049 -76.088475,40.039636") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42071, "countyName": "Lancaster", "cityID": 4237016, "cityName": "Intercourse" } }
+{ "create_at": datetime("2015-11-27T10:00:12.000Z"), "id": 670301107172708352, "text": "#CareerArc #Labor #Job alert: Package Handler - Part-Time | UPS | #LIHUE, HI https://t.co/K6u7z4UiF2 #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-159.3711111,21.9811111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CareerArc", "Labor", "Job", "LIHUE", "Jobs", "Hiring" }}, "user": { "id": 118676555, "name": "TMJ-HI Labor Jobs", "screen_name": "tmj_HI_LABOR", "lang": "en", "location": "Hawaii", "create_at": date("2010-03-01"), "description": "Follow this account for geo-targeted General Labor job tweets in Hawaii Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 284, "friends_count": 279, "statues_count": 7 }, "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("2015-11-27T10:00:12.000Z"), "id": 670301107265081345, "text": "@x1RZB thanks baby��", "in_reply_to_status": 670298249815851008, "in_reply_to_user": 348143528, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 348143528 }}, "user": { "id": 77111443, "name": "Birthday Girl♐️", "screen_name": "jaixo__", "lang": "en", "location": "610/215", "create_at": date("2009-09-24"), "description": "got you stuck off the realness❄️ SC: Viva_Jai", "followers_count": 2219, "friends_count": 1158, "statues_count": 73982 }, "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-27T10:00:12.000Z"), "id": 670301107458043904, "text": "What a lovely day. @ Kirksvile https://t.co/NokOUx2avK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.5782749,40.23135912"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36781204, "name": "becca", "screen_name": "beezysneezy", "lang": "en", "location": "null", "create_at": date("2009-04-30"), "description": "null", "followers_count": 360, "friends_count": 205, "statues_count": 16190 }, "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("2015-11-27T10:00:12.000Z"), "id": 670301107495694337, "text": "Afternoon rain this afternoon, high 43 (6 C). Low 36 (2 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712452, "name": "FortWorthWX", "screen_name": "FortWorthWX", "lang": "en", "location": "Fort Worth, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Fort Worth, Texas", "followers_count": 208, "friends_count": 93, "statues_count": 8066 }, "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("2015-11-27T10:00:12.000Z"), "id": 670301107495792640, "text": "Thanksgiving is probably my favorite holiday because I love reminding people how grateful I am to have them 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": 339872769, "name": "✨Sophia Lamothe✨", "screen_name": "SophiaLamothe", "lang": "en", "location": "null", "create_at": date("2011-07-21"), "description": "I spin things for fun Junior at Southridge high school✌️", "followers_count": 260, "friends_count": 301, "statues_count": 6095 }, "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("2015-11-27T10:00:12.000Z"), "id": 670301108208701440, "text": "Callie ❤️ https://t.co/EqdHABTOMv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 122122517, "name": "Amanda Elaine", "screen_name": "AmandaE1aine", "lang": "en", "location": "San Antonio, TX", "create_at": date("2010-03-11"), "description": "sped Teacher | Master's Degree holder | NM State Alumni", "followers_count": 452, "friends_count": 257, "statues_count": 15519 }, "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("2015-11-27T10:00:12.000Z"), "id": 670301108208709633, "text": "@brittanygwynn_2 thanks girl!!! https://t.co/BBwc9poLrl", "in_reply_to_status": 670300530850185216, "in_reply_to_user": 2829030512, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2829030512 }}, "user": { "id": 594288080, "name": "s y d ➵", "screen_name": "sydneygiraffes", "lang": "en", "location": "Burleson, Texas", "create_at": date("2012-05-29"), "description": "❂ count your blessings, not your burdens ❂", "followers_count": 830, "friends_count": 554, "statues_count": 9147 }, "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-27T10:00:12.000Z"), "id": 670301108221317120, "text": "Awks that was supposed to be a laughing face ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1740656611, "name": "Giselle ☹", "screen_name": "Giselle_Reyes24", "lang": "en", "location": "Houston.", "create_at": date("2013-09-06"), "description": "3's up // Cole World", "followers_count": 853, "friends_count": 556, "statues_count": 35807 }, "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("2015-11-27T10:00:12.000Z"), "id": 670301108489719808, "text": "Join the BJ's Restaurants, Inc. team! See our latest #Hospitality #job opening here: https://t.co/y5s1ZUMyD4 #Austin, TX #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.7895993,30.2629359"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Austin", "Hiring" }}, "user": { "id": 2377573225, "name": "BJ's Restaurants", "screen_name": "WorkAtBJs", "lang": "en", "location": "null", "create_at": date("2014-03-07"), "description": "null", "followers_count": 251, "friends_count": 0, "statues_count": 15582 }, "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-27T10:00:12.000Z"), "id": 670301108594728961, "text": "Let's scoop 88 ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 440187738, "name": "Jullian Snyder", "screen_name": "RJBigDad", "lang": "en", "location": "Richmond", "create_at": date("2011-12-18"), "description": "Chalice in Wonderland", "followers_count": 638, "friends_count": 460, "statues_count": 34158 }, "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("2015-11-27T10:00:12.000Z"), "id": 670301108774961152, "text": "@masteerjaay take yo shit off private so I can retweet ��", "in_reply_to_status": 670299828220354560, "in_reply_to_user": 1434292784, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1434292784 }}, "user": { "id": 2994941264, "name": "Roger Wilson", "screen_name": "SmuvveGuy10", "lang": "en", "location": "in the lab..", "create_at": date("2015-01-24"), "description": "❌C/O 16 ATH❌BIG STORM COMING☔️ Lay Easy Loved Ones❤️ LHS Tiger", "followers_count": 824, "friends_count": 755, "statues_count": 6056 }, "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("2015-11-27T10:00:12.000Z"), "id": 670301108913467392, "text": "Oh how things have changed�� https://t.co/KSjCmBT2PI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 866855365, "name": "Lee Olenyik", "screen_name": "LOlenyik", "lang": "en", "location": "null", "create_at": date("2012-10-07"), "description": "the y is silent | veeb | @e_wattzz", "followers_count": 628, "friends_count": 444, "statues_count": 5773 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mechanicsville, VA", "id": "91923d1d1be8347e", "name": "Mechanicsville", "place_type": "city", "bounding_box": rectangle("-77.421421,37.576879 -77.293309,37.669732") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51085, "countyName": "Hanover", "cityID": 5150856, "cityName": "Mechanicsville" } }
+{ "create_at": datetime("2015-11-27T10:00:12.000Z"), "id": 670301108951060480, "text": "Want to work at UMC? We're #hiring in #Lubbock, TX! Click for details: https://t.co/QMw9niRKou #Nursing #nurse #criticalcare #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-101.8919288,33.5881366"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Lubbock", "Nursing", "nurse", "criticalcare", "Job", "Jobs" }}, "user": { "id": 2940007148, "name": "UMC Nursing Jobs", "screen_name": "UMCNursingJobs", "lang": "en", "location": "null", "create_at": date("2014-12-22"), "description": "UMC Health System is the leader in comprehensive healthcare delivery in West Texas and Eastern New Mexico. Follow us to learn about our Nursing Jobs!", "followers_count": 82, "friends_count": 279, "statues_count": 62 }, "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("2015-11-27T10:00:12.000Z"), "id": 670301108992999424, "text": "Me and my love at his mothers house for Thanksgiving! #cozyhome #selfie… https://t.co/9HxA35ag1K", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.38916667,37.92416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "cozyhome", "selfie" }}, "user": { "id": 2163768385, "name": "Idiarys Reveron", "screen_name": "ireveron7", "lang": "en", "location": "San Francisco, CA", "create_at": date("2013-10-29"), "description": "Live, laugh, love! So blessed to be alive and have wonderful people around me!", "followers_count": 155, "friends_count": 442, "statues_count": 2639 }, "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("2015-11-27T10:00:12.000Z"), "id": 670301109119025153, "text": "For reals!!!! Couch potato all day..... #lazyday #meandbaby @ Casillas Home Sweet Home https://t.co/OvWbSVIegY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.1885529,35.4862785"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "lazyday", "meandbaby" }}, "user": { "id": 1595902909, "name": "Alicia❤️Casillas", "screen_name": "AliciaCasillas_", "lang": "en", "location": "Sanford NC", "create_at": date("2013-07-15"), "description": "Married my highschool sweet❤️Eduardo Casillas,Parents of 3boys Albert,Romeo n Levi.We have 4 furbabies Lady,Juliet,Lily n Charlie.Loving my life to the fullest.", "followers_count": 76, "friends_count": 131, "statues_count": 596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sanford, NC", "id": "25cd0c32e9559c17", "name": "Sanford", "place_type": "city", "bounding_box": rectangle("-79.241516,35.389316 -79.085936,35.559252") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37105, "countyName": "Lee", "cityID": 3759280, "cityName": "Sanford" } }
+{ "create_at": datetime("2015-11-27T10:00:12.000Z"), "id": 670301109198696448, "text": "I'm never looking at your snaps again man @Au_bria ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2739197836 }}, "user": { "id": 2190434493, "name": "❤️", "screen_name": "tamiawashere", "lang": "en", "location": "null", "create_at": date("2013-11-21"), "description": "null", "followers_count": 547, "friends_count": 605, "statues_count": 3938 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roxboro, NC", "id": "3c9977c63aacb378", "name": "Roxboro", "place_type": "city", "bounding_box": rectangle("-79.01172,36.341087 -78.955547,36.438522") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37145, "countyName": "Person", "cityID": 3758160, "cityName": "Roxboro" } }
+{ "create_at": datetime("2015-11-27T10:00:12.000Z"), "id": 670301109270020096, "text": "3 years ago today I found out I was pregnant ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 832492238, "name": "melissa landon", "screen_name": "melissa_landon", "lang": "en", "location": "null", "create_at": date("2012-09-18"), "description": "null", "followers_count": 365, "friends_count": 571, "statues_count": 1469 }, "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("2015-11-27T10:00:12.000Z"), "id": 670301109290864640, "text": "@Neck_oh_luss On Doug?", "in_reply_to_status": 670300953006862336, "in_reply_to_user": 314723564, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 314723564 }}, "user": { "id": 64244841, "name": "Grandmaster B-Hud", "screen_name": "The_Yodacat", "lang": "en", "location": "Mansfield, TX", "create_at": date("2009-08-09"), "description": "Ask me about Tom Coughlin. \nI've never been wrong before.", "followers_count": 1914, "friends_count": 442, "statues_count": 85458 }, "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-27T10:00:12.000Z"), "id": 670301109366444032, "text": "Why is it considered basic to like Adele? Maybe I like her bc she's one of the only singers in the genre with actual talent", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 615868898, "name": "Sarah Arnaud", "screen_name": "sarah_arnaud", "lang": "en", "location": "null", "create_at": date("2012-06-22"), "description": "null", "followers_count": 452, "friends_count": 247, "statues_count": 5220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Peachtree City, GA", "id": "fab01c4d14587744", "name": "Peachtree City", "place_type": "city", "bounding_box": rectangle("-84.626167,33.335655 -84.522236,33.449361") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13113, "countyName": "Fayette", "cityID": 1359724, "cityName": "Peachtree City" } }
+{ "create_at": datetime("2015-11-27T10:00:12.000Z"), "id": 670301109613805568, "text": "Need to do this homework", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 535829273, "name": "AJ...", "screen_name": "Jameire_", "lang": "en", "location": "DTX ✈️ The Hill", "create_at": date("2012-03-24"), "description": "Civil Engineer Major✨DREAMTEAM✨ #PVAMU19", "followers_count": 1179, "friends_count": 676, "statues_count": 19534 }, "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("2015-11-27T10:00:12.000Z"), "id": 670301109685063681, "text": "See a virtual tour of our listing on 5849 CALUMET Ct #Crestview #FL https://t.co/XHGIlnf2Ae #realestate https://t.co/Jc0oDZkrUp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.574751,30.800426"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Crestview", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6583 }, "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": 12091, "countyName": "Okaloosa", "cityID": 1215475, "cityName": "Crestview" } }
+{ "create_at": datetime("2015-11-27T10:00:12.000Z"), "id": 670301109743783936, "text": "Catering Coordinator - Panera Bread: (#Willowbrook, IL) https://t.co/gik5nH5Tnx #Hospitality #parttime #Veterans #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.944715,41.7587687"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Willowbrook", "Hospitality", "parttime", "Veterans", "Job", "Jobs", "Hiring" }}, "user": { "id": 88114220, "name": "TMJ-IL HRTA Jobs", "screen_name": "tmj_il_hrta", "lang": "en", "location": "Illinois", "create_at": date("2009-11-06"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Illinois Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 406, "friends_count": 291, "statues_count": 1344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willowbrook, IL", "id": "13d0994d4df436f8", "name": "Willowbrook", "place_type": "city", "bounding_box": rectangle("-87.967173,41.736609 -87.921342,41.783504") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1781938, "cityName": "Willowbrook" } }
+{ "create_at": datetime("2015-11-27T10:00:12.000Z"), "id": 670301109928366082, "text": "I'm a sucker for those blue eyes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 898577822, "name": "Sydn❁y", "screen_name": "Sydney_Cheyanne", "lang": "en", "location": "Jaden Luke", "create_at": date("2012-10-22"), "description": "life is an art", "followers_count": 259, "friends_count": 192, "statues_count": 2359 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Harrah, OK", "id": "018ceebd440a8a24", "name": "Harrah", "place_type": "city", "bounding_box": rectangle("-97.199246,35.428764 -97.15572,35.498384") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4032750, "cityName": "Harrah" } }
+{ "create_at": datetime("2015-11-27T10:00:12.000Z"), "id": 670301110033321985, "text": "Get off my cloud", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 442225096, "name": "Steve Rod", "screen_name": "steven151r0d", "lang": "en", "location": "The One", "create_at": date("2011-12-20"), "description": "Exhilerate Mind,Body & Soul.' Xander Zone' The Exclusive.. Mentality & Technique are Key", "followers_count": 158, "friends_count": 374, "statues_count": 6912 }, "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("2015-11-27T10:00:12.000Z"), "id": 670301110167449600, "text": "@Matthew517_ jonah pose and the stoplight", "in_reply_to_status": 670300960636342272, "in_reply_to_user": 2434879428, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2434879428 }}, "user": { "id": 2581736280, "name": "Jiraffe", "screen_name": "jaysestabillo", "lang": "en", "location": "Murrieta, CA", "create_at": date("2014-06-21"), "description": "giraffa camelopardalis", "followers_count": 61, "friends_count": 67, "statues_count": 1235 }, "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("2015-11-27T10:00:12.000Z"), "id": 670301110230364160, "text": "Afternoon rain this afternoon, high 46 (8 C). Low 37 (3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712424, "name": "DallasWX", "screen_name": "DallasWX", "lang": "en", "location": "Dallas, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Dallas, Texas", "followers_count": 191, "friends_count": 93, "statues_count": 8043 }, "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-27T10:00:12.000Z"), "id": 670301110285017088, "text": "@CuteEmergency @_tarzanne", "in_reply_to_status": 668752176584462337, "in_reply_to_user": 568825492, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 568825492, 365851583 }}, "user": { "id": 146984656, "name": "Ryan Scott", "screen_name": "rscottcapitals", "lang": "en", "location": "llege", "create_at": date("2010-05-22"), "description": "Do what you gotta do, and go home", "followers_count": 840, "friends_count": 560, "statues_count": 6867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Hunt, VA", "id": "9b24202596420d07", "name": "Fort Hunt", "place_type": "city", "bounding_box": rectangle("-77.082572,38.709705 -77.041274,38.769131") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5129136, "cityName": "Fort Hunt" } }
+{ "create_at": datetime("2015-11-27T10:00:13.000Z"), "id": 670301110729580544, "text": "Dirt bike time PC ~ @akgreen1989 ~ @rockstarenergy @oneindustries dragon_motorsports @bel_ray @fmf73… https://t.co/HmyKXYGSF4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.19115,34.864511"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1380936422, 28207950, 32561144, 204364487, 145425764 }}, "user": { "id": 255808431, "name": "James Hanson", "screen_name": "Professor296", "lang": "en", "location": "Vancouver, WA", "create_at": date("2011-02-21"), "description": "Rockstar Energy Drink Motocross Manager", "followers_count": 1674, "friends_count": 576, "statues_count": 8168 }, "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.32187,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37133, "countyName": "Onslow" } }
+{ "create_at": datetime("2015-11-27T10:00:13.000Z"), "id": 670301110863704065, "text": "Wind 2.0 mph SE. Barometer 30.165 in, Steady. Temperature 44.2 °F. Rain today 0.00 in. Humidity 30%", "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": 191, "friends_count": 58, "statues_count": 229368 }, "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("2015-11-27T10:00:13.000Z"), "id": 670301110909865985, "text": "Can you recommend anyone for this #Healthcare #job? https://t.co/qVtJhQhUVL #MedField #Soldotna, AK #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-151.0583332,60.4877778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "MedField", "Soldotna", "Hiring" }}, "user": { "id": 3278999924, "name": "CPH Jobs", "screen_name": "CPHJobs", "lang": "en", "location": "Soldotna, AK", "create_at": date("2015-07-13"), "description": "At CPH every employee is considered a caregiver, an integral part of our health care team. Follow us to learn more!", "followers_count": 3, "friends_count": 0, "statues_count": 50 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Soldotna, AK", "id": "350518f7c1ac93ff", "name": "Soldotna", "place_type": "city", "bounding_box": rectangle("-151.193024,60.458405 -151.048563,60.511008") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2122, "countyName": "Kenai Peninsula", "cityID": 271640, "cityName": "Soldotna" } }
+{ "create_at": datetime("2015-11-27T10:00:13.000Z"), "id": 670301111014703104, "text": "@NolteNC And that's just a start. A+", "in_reply_to_status": 670293912649662464, "in_reply_to_user": 16163639, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16163639 }}, "user": { "id": 19136850, "name": "Texacalirose™", "screen_name": "texacalirose", "lang": "en", "location": "NorCal & SoCal-Misplaced Texan", "create_at": date("2009-01-17"), "description": "Pre-Raphaelite woman in search of Macduff. Gregarious Misanthrope. AQHA, Two Reagan babies,Granny,Studio Teacher,RE Broker,former HS English teacher & LEO.", "followers_count": 3463, "friends_count": 3421, "statues_count": 12485 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-27T10:00:13.000Z"), "id": 670301111199342592, "text": "Found a hottie whilst tree searching @ Dehart Christmas Tree Farm https://t.co/lNA45xaUfc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.07026258,39.42243335"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 130521202, "name": "truegritink", "screen_name": "truegritink", "lang": "en", "location": "Franklin, Indiana", "create_at": date("2010-04-07"), "description": "my name is Bradford. Instagram @truegritinkbradford I live reckless in love w/ Jesus Christ. I'm here to walk relentless, and die glorious.", "followers_count": 593, "friends_count": 17, "statues_count": 4334 }, "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": 18081, "countyName": "Johnson" } }
+{ "create_at": datetime("2015-11-27T10:00:13.000Z"), "id": 670301111341969408, "text": "@walkselizabeth thank u walks!!!! love u so much ����", "in_reply_to_status": 670300711939387392, "in_reply_to_user": 344251935, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 344251935 }}, "user": { "id": 339290458, "name": "shawee", "screen_name": "shannonstroud7", "lang": "en", "location": "null", "create_at": date("2011-07-20"), "description": "#utk19", "followers_count": 852, "friends_count": 572, "statues_count": 5252 }, "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("2015-11-27T10:00:13.000Z"), "id": 670301111564242944, "text": "@KStreetHipster Mayhaps!", "in_reply_to_status": 670300935902507008, "in_reply_to_user": 415162673, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 415162673 }}, "user": { "id": 361460084, "name": "Adam Goldstein", "screen_name": "A_H_Goldstein", "lang": "en", "location": "null", "create_at": date("2011-08-24"), "description": "¯\\_(ツ)_/¯ Views expressed here are are my own, RT do not imply endorsement.", "followers_count": 1198, "friends_count": 1850, "statues_count": 29470 }, "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("2015-11-27T10:00:13.000Z"), "id": 670301111677534209, "text": "���� https://t.co/00NneKO30s", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2859654057, "name": "Jazzy❤️.", "screen_name": "DxddyJazz", "lang": "en", "location": "Rosaryville, MD", "create_at": date("2014-11-03"), "description": "I really don't give a shit . @TheyLoveMilann❤️", "followers_count": 454, "friends_count": 548, "statues_count": 2017 }, "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("2015-11-27T10:00:13.000Z"), "id": 670301111799160832, "text": "Canes question mark https://t.co/w6Qhax2lmO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 597845523, "name": "CEO of Virgin Inc.", "screen_name": "UglyBoysENT", "lang": "en", "location": "Skyway Bridge, FL", "create_at": date("2012-06-02"), "description": "my password is password #FaithfulNiggasOfTampaUnited", "followers_count": 784, "friends_count": 589, "statues_count": 89224 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Citrus Park, FL", "id": "bd975a41c2c796e9", "name": "Citrus Park", "place_type": "city", "bounding_box": rectangle("-82.582925,28.037803 -82.537741,28.114272") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1212425, "cityName": "Citrus Park" } }
+{ "create_at": datetime("2015-11-27T10:00:13.000Z"), "id": 670301111845163008, "text": "Logistics Associate - OfficeMax: (#Kahului, HI) https://t.co/VjN5kQ6Nj0 #Veterans #Job #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-156.454708,20.881571"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Kahului", "Veterans", "Job", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 38082277, "name": "TMJ-MAU Jobs", "screen_name": "tmj_mau_jobs", "lang": "en", "location": "Maui, HI", "create_at": date("2009-05-05"), "description": "Follow this account for geo-targeted Other job tweets in Maui, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 546, "friends_count": 535, "statues_count": 17 }, "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("2015-11-27T10:00:13.000Z"), "id": 670301111845302272, "text": "@AllieRoth fuck, thanks for letting me know tho", "in_reply_to_status": 670300901534384128, "in_reply_to_user": 4164097823, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4164097823 }}, "user": { "id": 250422723, "name": "bryan", "screen_name": "bigbry69", "lang": "en", "location": "UofL 17'", "create_at": date("2011-02-10"), "description": "Alex loveland is my lil ugly", "followers_count": 1590, "friends_count": 1815, "statues_count": 62250 }, "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-27T10:00:13.000Z"), "id": 670301111924977665, "text": "in the car .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2364445753, "name": "ken ❄️", "screen_name": "obey_cardae", "lang": "en", "location": "Rest Easy Uncle Kenny ❤", "create_at": date("2014-02-27"), "description": "quiet one with real nigga tendencies . ❤ Kayla ❤|", "followers_count": 1919, "friends_count": 2111, "statues_count": 39671 }, "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-27T10:00:13.000Z"), "id": 670301112004571136, "text": "i don't want to leave 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": 1056361747, "name": "Jen", "screen_name": "jennbriones", "lang": "en", "location": "Eden Prairie, MN", "create_at": date("2013-01-02"), "description": "Concerts & starbucks", "followers_count": 3744, "friends_count": 576, "statues_count": 34078 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eden Prairie, MN", "id": "a1a25de9f5efa69b", "name": "Eden Prairie", "place_type": "city", "bounding_box": rectangle("-93.520913,44.799239 -93.397928,44.893162") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2718116, "cityName": "Eden Prairie" } }
+{ "create_at": datetime("2015-11-27T10:00:13.000Z"), "id": 670301112122019840, "text": "@CBBEyeTest every since we lost against Arkansas Little Rock they've been on the bench to start", "in_reply_to_status": 670295932769050624, "in_reply_to_user": 2867060914, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2867060914 }}, "user": { "id": 486200933, "name": "San Diego Sports", "screen_name": "SanDiego_Sports", "lang": "en", "location": "San Diego", "create_at": date("2012-02-07"), "description": "We cover | #Chargers | #Padres | #SDGulls | #SDSU | @SDFlashSoccer & @SDSurfOfficial Media Sponsor | #USD | #Olympics | Award Winning Sports Blog |", "followers_count": 2451, "friends_count": 2826, "statues_count": 44735 }, "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-27T10:00:13.000Z"), "id": 670301112545746944, "text": "@hayitsshay he texted me and said he was. I was really kidding tho and being a big cry baby.", "in_reply_to_status": 670126748403634176, "in_reply_to_user": 153482729, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 153482729 }}, "user": { "id": 153482729, "name": "1st place cry baby", "screen_name": "hayitsshay", "lang": "en", "location": "IG: cheyennayoung ☀️", "create_at": date("2010-06-08"), "description": "living life I guess. ✨ #PROHOE✨", "followers_count": 1343, "friends_count": 1105, "statues_count": 77689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vermilion, OH", "id": "7cbe24f186029edb", "name": "Vermilion", "place_type": "city", "bounding_box": rectangle("-82.441995,41.38691 -82.282407,41.430782") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39093, "countyName": "Lorain", "cityID": 3979716, "cityName": "Vermilion" } }
+{ "create_at": datetime("2015-11-27T10:00:13.000Z"), "id": 670301112587517952, "text": "Want to work at Aerotek? We're #hiring in #Honolulu, HI! Click for details: https://t.co/l6rMf1Gw2Z #Manufacturing #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Honolulu", "Manufacturing", "Job", "Jobs" }}, "user": { "id": 23527770, "name": "TMJ - HON Manuf Jobs", "screen_name": "tmj_hon_manuf", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Manufacturing job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 224, "friends_count": 202, "statues_count": 1 }, "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("2015-11-27T10:00:13.000Z"), "id": 670301112612864000, "text": "Please don't hmu with bs then get offended when I give you an attitude in return.\n\nLike what did you expect lol.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 612870087, "name": "M i c T r o u t ⛽️", "screen_name": "JournellPierre", "lang": "en", "location": "|||13een-O.|||NotTrapMusic⇩|||", "create_at": date("2012-06-19"), "description": "|||C ' e s t L a V i e|||journellpierre@gmail.com for features and shit.|||S P L A S H G A N G||| |||O D D S Q U A D|||K R Y P T O N I T E|||", "followers_count": 1867, "friends_count": 999, "statues_count": 23450 }, "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-27T10:00:13.000Z"), "id": 670301112746926080, "text": "Afternoon thundershowers this afternoon, high 71 (22 C). Low 42 (6 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712472, "name": "AustinWX", "screen_name": "AustinWX", "lang": "en", "location": "Austin, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Austin, Texas", "followers_count": 1011, "friends_count": 1367, "statues_count": 7904 }, "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-27T10:00:13.000Z"), "id": 670301112965070848, "text": "@jessicajbeck Thanks. Next year I'm gonna make sure I'm out of town.", "in_reply_to_status": 670300306668916736, "in_reply_to_user": 46780261, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 46780261 }}, "user": { "id": 1035191, "name": "Steven Peterson", "screen_name": "squeakytoy", "lang": "en", "location": "Everywhere", "create_at": date("2007-03-12"), "description": "Solemn and shabby like a requiem in denim and leopardskin.\n\niOS dev at @iCracked. @routesy is my baby. Previously @Detour, @Genius, turntable.fm, @IMVU.", "followers_count": 2568, "friends_count": 1032, "statues_count": 66868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-27T10:00:13.000Z"), "id": 670301113250246657, "text": "See a virtual tour of our listing on 4621 Top Flight Dr #Crestview #FL https://t.co/4bDU9VnIgk #realestate https://t.co/3BcfNsGCwL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.559006,30.710081"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Crestview", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6584 }, "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": 12091, "countyName": "Okaloosa", "cityID": 1215475, "cityName": "Crestview" } }
+{ "create_at": datetime("2015-11-27T10:00:13.000Z"), "id": 670301113397223424, "text": "I screw soo many things up 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": 604061314, "name": "Catherine", "screen_name": "_cpaigee", "lang": "en", "location": "Roanoke, VA", "create_at": date("2012-06-09"), "description": "find that happy place and stay // ig cathyfisherr // sc cpf_xo", "followers_count": 1386, "friends_count": 763, "statues_count": 10792 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hollins, VA", "id": "1ae84c90df990839", "name": "Hollins", "place_type": "city", "bounding_box": rectangle("-79.998978,37.320223 -79.920429,37.363095") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51161, "countyName": "Roanoke", "cityID": 5137880, "cityName": "Hollins" } }
+{ "create_at": datetime("2015-11-27T10:00:13.000Z"), "id": 670301113418170368, "text": "Barnabas Health is hiring! Mental Health A #jobs in ELBERON Apply today https://t.co/2iqlCUpwlJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.993546,40.294334"), "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": 167, "friends_count": 42, "statues_count": 68938 }, "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("2015-11-27T10:00:13.000Z"), "id": 670301113451716608, "text": "@CracksofHate do ittttt", "in_reply_to_status": 670290330537869312, "in_reply_to_user": 2670817362, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2670817362 }}, "user": { "id": 195603528, "name": "jam", "screen_name": "Saucierthanu", "lang": "en", "location": "michigan ¯\\_(ツ)_/¯", "create_at": date("2010-09-26"), "description": "the only broken-hearted loser you'll ever need.", "followers_count": 1442, "friends_count": 876, "statues_count": 60212 }, "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("2015-11-27T10:00:13.000Z"), "id": 670301113518788611, "text": "My og goofy asl ���� she barely have cash on her now always got fucking cards .", "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": "BOSSKJ™", "screen_name": "365Kj_", "lang": "en", "location": "CHICAGO✈️MIAMI", "create_at": date("2011-08-11"), "description": "Big E & LB World ️ ️", "followers_count": 1553, "friends_count": 906, "statues_count": 50783 }, "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("2015-11-27T10:00:13.000Z"), "id": 670301113569136640, "text": "@RapSheet @tiffblackmon is he worth playing in Fantasy over Frank Gore and Deangelo williams?", "in_reply_to_status": 670299164178309120, "in_reply_to_user": 16403943, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16403943, 261881303 }}, "user": { "id": 380426525, "name": "Darrin Foreman", "screen_name": "DJF1323", "lang": "en", "location": "Tampa, Florida", "create_at": date("2011-09-26"), "description": "A Gentleman and a Scholar #USF #GOBULLS #TheU #JogaBonita", "followers_count": 226, "friends_count": 757, "statues_count": 4003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Trinity, FL", "id": "48754d0fac025a6f", "name": "Trinity", "place_type": "city", "bounding_box": rectangle("-82.699086,28.173238 -82.613321,28.198092") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12101, "countyName": "Pasco", "cityID": 1272442, "cityName": "Trinity" } }
+{ "create_at": datetime("2015-11-27T10:00:13.000Z"), "id": 670301113581772805, "text": "@ShareALittleSun Thanks for the mention!", "in_reply_to_status": 670286058400124928, "in_reply_to_user": 37656289, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 37656289 }}, "user": { "id": 1447275140, "name": "NSB Happenings", "screen_name": "NSBHappenings", "lang": "en", "location": "New Smyrna Beach Florida", "create_at": date("2013-05-21"), "description": "Your community resource center for where to stay, shop, dine, drink, and play in New Smyrna Beach Florida by a local who loves this town!", "followers_count": 760, "friends_count": 671, "statues_count": 3714 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Smyrna Beach, FL", "id": "df82825ad778ab15", "name": "New Smyrna Beach", "place_type": "city", "bounding_box": rectangle("-80.986841,28.987795 -80.867436,29.083799") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1248625, "cityName": "New Smyrna Beach" } }
+{ "create_at": datetime("2015-11-27T10:00:13.000Z"), "id": 670301113669685249, "text": "Interested in a #Retail #job near #BridgeCity, TX? This could be a great fit: https://t.co/AInADV5aAg #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.8457255,30.0207678"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "BridgeCity", "Hiring", "CareerArc" }}, "user": { "id": 59694310, "name": "TMJ-TX Retail Jobs", "screen_name": "tmj_tx_retail", "lang": "en", "location": "Texas", "create_at": date("2009-07-23"), "description": "Follow this account for geo-targeted Retail job tweets in Texas Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 506, "friends_count": 308, "statues_count": 7265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bridge City, TX", "id": "896c55a7e57f4627", "name": "Bridge City", "place_type": "city", "bounding_box": rectangle("-93.913159,30.005738 -93.799355,30.071914") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48361, "countyName": "Orange", "cityID": 4810252, "cityName": "Bridge City" } }
+{ "create_at": datetime("2015-11-27T10:00:13.000Z"), "id": 670301113946632192, "text": "He's my fave https://t.co/BwShcIiQD8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 619357137, "name": "aliiiiiii gator", "screen_name": "alibaker18", "lang": "en", "location": "null", "create_at": date("2012-06-26"), "description": "• photography • journalism • Jesus • football •", "followers_count": 775, "friends_count": 862, "statues_count": 7736 }, "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("2015-11-27T10:00:13.000Z"), "id": 670301114215043077, "text": "Yea https://t.co/eNN67p8yTD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2245065292, "name": "Jai", "screen_name": "JyrahK", "lang": "en", "location": "null", "create_at": date("2013-12-26"), "description": "Anything I do, I wanna do it well. |", "followers_count": 933, "friends_count": 607, "statues_count": 21966 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, FL", "id": "4d22324747f271a2", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-82.586238,27.524702 -82.52756,27.576514") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12081, "countyName": "Manatee", "cityID": 1244175, "cityName": "Memphis" } }
+{ "create_at": datetime("2015-11-27T10:00:13.000Z"), "id": 670301114294779904, "text": "@WeavusChrist yop", "in_reply_to_status": 670300416622632960, "in_reply_to_user": 194006029, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 194006029 }}, "user": { "id": 48941988, "name": "Nope", "screen_name": "iAmMrCarson", "lang": "en", "location": "Detroit", "create_at": date("2009-06-19"), "description": "• @WeighnsWorld\n• #MSU\r\n• @CrnrStoreGoods ™\r\n• IG: iammrcarson\nAin't no follow back bih", "followers_count": 2165, "friends_count": 971, "statues_count": 256406 }, "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("2015-11-27T10:00:13.000Z"), "id": 670301114365947904, "text": "Join the Lowe's team! See our latest #Labor #job opening here: https://t.co/geAYAqGb4F #Honolulu, HI #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Labor", "job", "Honolulu", "Hiring" }}, "user": { "id": 22777294, "name": "TMJ-HON Labor Jobs", "screen_name": "tmj_hon_labor", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-04"), "description": "Follow this account for geo-targeted General Labor job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 300, "friends_count": 291, "statues_count": 4 }, "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("2015-11-27T10:00:13.000Z"), "id": 670301114470830080, "text": "#CareerArc #Retail #Job alert: Center Store Associate | Winn Dixie | #NFTMYERS, FL https://t.co/JoGcPMwVm4 #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.8839367,26.6838525"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CareerArc", "Retail", "Job", "NFTMYERS", "Jobs", "Hiring" }}, "user": { "id": 59695864, "name": "TMJ-FL Retail Jobs", "screen_name": "tmj_fl_retail", "lang": "en", "location": "Florida", "create_at": date("2009-07-23"), "description": "Follow this account for geo-targeted Retail job tweets in Florida Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 473, "friends_count": 306, "statues_count": 6118 }, "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("2015-11-27T10:00:13.000Z"), "id": 670301114546302976, "text": "@weeburd1 Aww, thank you! Much appreciated.", "in_reply_to_status": 670300034139824128, "in_reply_to_user": 259328274, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 259328274 }}, "user": { "id": 1586002562, "name": "Glen Moyer", "screen_name": "GlenMoyer1", "lang": "en", "location": "Keithville, LA", "create_at": date("2013-07-11"), "description": "Host, Under The Tartan Sky podcast. Editor, BALLOONING. Freelance writing, design. Love Cats & Scotland! Views are my own.", "followers_count": 497, "friends_count": 272, "statues_count": 7204 }, "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("2015-11-27T10:00:14.000Z"), "id": 670301114739224576, "text": "Want to work at Hawaii Pacific Health? We're #hiring in #Honolulu, HI! Click for details: https://t.co/QSvx3qHnnU #Marketing #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8480364,21.3136151"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Honolulu", "Marketing", "Job", "Jobs" }}, "user": { "id": 23493115, "name": "TMJ-HON Advert Jobs", "screen_name": "tmj_hon_adv", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Marketing/Ad/PR job tweets in Honolulu, HI from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 290, "friends_count": 193, "statues_count": 9 }, "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("2015-11-27T10:00:14.000Z"), "id": 670301114907148288, "text": "Here's a very cool app worth picking up and it's been FREE for awhile. It's also one of the top… https://t.co/90AnIgNKLr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.034,32.7335"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 40165385, "name": "Tina L Rice", "screen_name": "ASHCROFT54", "lang": "en", "location": "California ", "create_at": date("2009-05-14"), "description": "Photographer~Blogger~Artist I publish a blog called Combo Apps: Mobile Extreme Editing. It's about mobile extreme editing, app reviews and product reviews.", "followers_count": 1604, "friends_count": 1408, "statues_count": 18432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lemon Grove, CA", "id": "9c86b10d0bc471b5", "name": "Lemon Grove", "place_type": "city", "bounding_box": rectangle("-117.063891,32.715113 -117.014083,32.75014") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 641124, "cityName": "Lemon Grove" } }
+{ "create_at": datetime("2015-11-27T10:00:14.000Z"), "id": 670301114936377344, "text": "Genesis Healthc is hiring! Dietary Aide #jobs in PARSIPPANY Apply today https://t.co/rLsNT218Fb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.404497,40.854216"), "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": 167, "friends_count": 42, "statues_count": 68939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Parsippany, NJ", "id": "01476f41278c3cd7", "name": "Parsippany", "place_type": "city", "bounding_box": rectangle("-74.483462,40.827956 -74.399375,40.885137") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34027, "countyName": "Morris" } }
+{ "create_at": datetime("2015-11-27T10:00:14.000Z"), "id": 670301115036995584, "text": "See our latest #Pittsburgh, PA #job and click to apply: Personal Banker Universal - https://t.co/RqBgdvlMpV #cfgjobs #Banking #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.9958864,40.4406248"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Pittsburgh", "job", "cfgjobs", "Banking", "Hiring" }}, "user": { "id": 22787034, "name": "TMJ-PIT Bank Jobs", "screen_name": "tmj_pit_banking", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2009-03-04"), "description": "Follow this account for geo-targeted Banking job tweets in Pittsburgh, PA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 372, "friends_count": 316, "statues_count": 83 }, "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("2015-11-27T10:00:14.000Z"), "id": 670301115146104832, "text": "My dad got me out of bed this morning by blasting Christmas music", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2305525099, "name": "Jamie Freitas", "screen_name": "FreitasJamie", "lang": "en", "location": "Livermore, CA", "create_at": date("2014-01-22"), "description": "GHS Class 2018//Bay Area", "followers_count": 191, "friends_count": 184, "statues_count": 1900 }, "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("2015-11-27T10:00:14.000Z"), "id": 670301115150426113, "text": "Tickets on sale now for @thecure & @thetwilightsad in Vancouver. https://t.co/xBeIKrRTQ2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 474833666, 23094396 }}, "user": { "id": 20106036, "name": "Chain of Flowers", "screen_name": "CraigatCoF", "lang": "en", "location": "New Orleans", "create_at": date("2009-02-04"), "description": "19 years of news, rumors, minutiae about The Cure.", "followers_count": 4543, "friends_count": 447, "statues_count": 29793 }, "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("2015-11-27T10:00:14.000Z"), "id": 670301115238322176, "text": "Seriously lol https://t.co/61SjPyya2T", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 345521589, "name": "Nat☪", "screen_name": "_Nattaliiee_", "lang": "en", "location": "Riv , CA", "create_at": date("2011-07-30"), "description": "Right where I'm supposed to be , FrankieAlvarez❤️", "followers_count": 1089, "friends_count": 691, "statues_count": 48937 }, "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-27T10:00:14.000Z"), "id": 670301115238322177, "text": "Haven't worked on my research, No Regrets! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1543255142, "name": "Edgar Vega", "screen_name": "Edgar_Vega23", "lang": "en", "location": "Needville, Texas", "create_at": date("2013-06-24"), "description": "Let's put our minds to ease.", "followers_count": 246, "friends_count": 202, "statues_count": 2674 }, "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("2015-11-27T10:00:14.000Z"), "id": 670301115548733440, "text": "Cloudy this afternoon, high 78 (26 C). Low 69 (21 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18712388, "name": "HoustonWX", "screen_name": "HoustonWX", "lang": "en", "location": "Houston, Texas", "create_at": date("2009-01-06"), "description": "Forecasts and advisories for Houston, Texas", "followers_count": 1737, "friends_count": 77, "statues_count": 7913 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-27T10:00:14.000Z"), "id": 670301115796205569, "text": "I shoulda known you would cross me !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 597911482, "name": "ryan keszeg", "screen_name": "ryan_keszeg", "lang": "en", "location": "null", "create_at": date("2012-06-02"), "description": "Whataburger Lover • TexasForever • The Dream Team • It's that easy", "followers_count": 1148, "friends_count": 734, "statues_count": 23867 }, "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("2015-11-27T10:00:14.000Z"), "id": 670301115842334720, "text": "If you ain't fuckin with young thug I ain't fuckin wit you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 954587330, "name": "KRI", "screen_name": "Aguilar_2014", "lang": "en", "location": "Huffman,Texas", "create_at": date("2012-11-17"), "description": "Lifelong Texans Fan. #IAH", "followers_count": 727, "friends_count": 624, "statues_count": 19893 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-27T10:00:14.000Z"), "id": 670301115905273856, "text": "������ https://t.co/wRG64JPEP1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 147094732, "name": "#PLUTO", "screen_name": "_PrinceAlonso", "lang": "en", "location": "HTX✈Nasty Nac", "create_at": date("2010-05-22"), "description": "#SFA17 .. Hair Long Money Long | #PLUTO | #LongLiveTheKidd | Height is just a Number™ | Heart Over Height ™| #PatsNation", "followers_count": 1328, "friends_count": 1067, "statues_count": 40505 }, "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("2015-11-27T10:00:14.000Z"), "id": 670301115943112704, "text": "@AintUJondell woah , I don't do those kind of activities but I'm just guessing that on a water bed the chances of fatalities are crucial", "in_reply_to_status": 670300243716517888, "in_reply_to_user": 2869992429, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2869992429 }}, "user": { "id": 3023222372, "name": "1216", "screen_name": "tvauht", "lang": "en", "location": "$$$", "create_at": date("2015-02-16"), "description": "| new account| ktx|", "followers_count": 479, "friends_count": 277, "statues_count": 19388 }, "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("2015-11-27T10:00:14.000Z"), "id": 670301116106719233, "text": "@SevieKoy thanks bro.", "in_reply_to_status": 670300804750942210, "in_reply_to_user": 261880389, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 261880389 }}, "user": { "id": 248022711, "name": "lilguttabumbaclot", "screen_name": "sungodpxllz", "lang": "en", "location": "Nawf Atlanta", "create_at": date("2011-02-05"), "description": "bwookfleacoo@gmail.com", "followers_count": 3296, "friends_count": 984, "statues_count": 207170 }, "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-27T10:00:14.000Z"), "id": 670301116459065344, "text": "@sergiodemichiel @vannabiagini @TinaCervasio @JennifersWayJE", "in_reply_to_status": 670298431538257920, "in_reply_to_user": 240317733, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 240317733, 1189530464, 31161814, 146249884 }}, "user": { "id": 2343607394, "name": "j paul", "screen_name": "carisbrook5", "lang": "en", "location": "new york city", "create_at": date("2014-02-14"), "description": "null", "followers_count": 449, "friends_count": 1999, "statues_count": 44240 }, "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("2015-11-27T10:00:14.000Z"), "id": 670301116475813888, "text": "Lies . https://t.co/D1K5FwEunF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 626173608, "name": "12/1✈️✨", "screen_name": "TiaaGuwop_", "lang": "en", "location": "Kentrell❤️", "create_at": date("2012-07-03"), "description": "Sittin back coutin rolls , laughing at broke hoes✈️", "followers_count": 5372, "friends_count": 1767, "statues_count": 51230 }, "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-27T10:00:14.000Z"), "id": 670301116588949504, "text": "Amazing surprise for our guests! Go see @creedmovie and you never know when @TheSlyStallone will stop by! https://t.co/MZsfNHZQjh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3253953780, 183310940 }}, "user": { "id": 17219041, "name": "ArcLight Cinemas", "screen_name": "ArcLightCinemas", "lang": "en", "location": "Southern California", "create_at": date("2008-11-06"), "description": "Lose yourself in a movie at one of our SoCal locations, @ArcLightDCArea, @ArcLightChicago. Reserved seating, no ads, & hassle-free. T&C http://ow.ly/nqQIF", "followers_count": 13563, "friends_count": 1018, "statues_count": 17374 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-27T10:00:14.000Z"), "id": 670301116626669568, "text": "Last Nights Wave Wit @_jayallen ������™ @ Richmond Bluffs https://t.co/tCBWfOYF5h", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.49756936,41.56521022"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 84178693 }}, "user": { "id": 76173859, "name": "Cordial™ & Polite™", "screen_name": "SE_ANT100", "lang": "en", "location": "S.E.uclid OHIO", "create_at": date("2009-09-21"), "description": "FATHER ... IG & Snap: SE_ANT100 The S.E. King Himself", "followers_count": 580, "friends_count": 298, "statues_count": 22830 }, "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("2015-11-27T10:00:14.000Z"), "id": 670301116907679745, "text": "Let's go get food", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325223692, "name": "Cheyenne Morris", "screen_name": "cheeyennemorris", "lang": "en", "location": "bakersfield, california.", "create_at": date("2011-06-27"), "description": "There’s a girl out there with love in her eyes and flowers in her hair ❁", "followers_count": 471, "friends_count": 376, "statues_count": 8816 }, "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("2015-11-27T10:00:14.000Z"), "id": 670301117251653632, "text": "My dad has been the only thing on my mind lately, I miss him so fucking much", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 499293804, "name": "brittt", "screen_name": "BrittaniPerales", "lang": "en", "location": "SA,TX", "create_at": date("2012-02-21"), "description": "18 • snap : brittaniperales • rest in peace dad love you forever and always", "followers_count": 341, "friends_count": 321, "statues_count": 10471 }, "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-27T10:00:14.000Z"), "id": 670301117905932288, "text": "Macy's Retail #Retail : Retail Gift Registry Advisor, Full Time: Honolulu, HI, Macy's Ala Moana... (#Honolulu, HI) https://t.co/WaPYrs0UDh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8429647,21.2912881"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "Honolulu" }}, "user": { "id": 28625039, "name": "TMJ-HON Retail Jobs", "screen_name": "tmj_hon_retail", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 370, "friends_count": 311, "statues_count": 451 }, "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("2015-11-27T10:00:14.000Z"), "id": 670301117985775618, "text": "Genesis Healthc is hiring! Registered Nurs #jobs in BARBOURS Apply today https://t.co/Boxr6MXsel", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.029912,41.279638"), "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": 167, "friends_count": 42, "statues_count": 68941 }, "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": 42081, "countyName": "Lycoming" } }
+{ "create_at": datetime("2015-11-27T10:00:14.000Z"), "id": 670301118115610624, "text": "Afternoon showers this afternoon, high 38 (3 C). Low 31 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575081, "name": "San Angelo Weather", "screen_name": "SanAngeloWX", "lang": "en", "location": "San Angelo, TX", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for San Angelo, TX", "followers_count": 1920, "friends_count": 92, "statues_count": 7803 }, "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("2015-11-27T10:00:15.000Z"), "id": 670301118941949952, "text": "Hugs, kisses, cuddles, penetration, climax #WhyINeedYouIn5Words", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WhyINeedYouIn5Words" }}, "user": { "id": 2944262935, "name": "Rational Dreamer", "screen_name": "Compelling_Doll", "lang": "en", "location": "over the rainbow ", "create_at": date("2014-12-26"), "description": "null", "followers_count": 630, "friends_count": 633, "statues_count": 4551 }, "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-27T10:00:15.000Z"), "id": 670301119063592961, "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": 3032124714, "name": "Jasmine Jae Pena", "screen_name": "jasminejae_19", "lang": "en", "location": "null", "create_at": date("2015-02-19"), "description": "Jazz♈ \nYo Don't talk to my server like that.\nAries: Looks like a bad bitch.. is a bad bitch.\n\n18||SPC|| Psychology major", "followers_count": 80, "friends_count": 160, "statues_count": 5497 }, "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("2015-11-27T10:00:15.000Z"), "id": 670301119193743360, "text": "#SickOrInjuredPerson at #HomeSuiteHomeApts, 2801 E Colonial Dr. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3474857,28.5535836"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SickOrInjuredPerson", "HomeSuiteHomeApts", "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": 661, "friends_count": 1, "statues_count": 66672 }, "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("2015-11-27T10:00:15.000Z"), "id": 670301119415906305, "text": "The background music ���� https://t.co/NYoG7yeS30", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 288351086, "name": "Logan Zavodny", "screen_name": "Gorilla_Lo", "lang": "en", "location": "Shawnee, Kansas", "create_at": date("2011-04-26"), "description": "Pitt state '17.", "followers_count": 610, "friends_count": 574, "statues_count": 4769 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shawnee, KS", "id": "f02a00a2aa11bfe3", "name": "Shawnee", "place_type": "city", "bounding_box": rectangle("-94.876678,38.941315 -94.703717,39.060541") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20091, "countyName": "Johnson", "cityID": 2064500, "cityName": "Shawnee" } }
+{ "create_at": datetime("2015-11-27T10:00:15.000Z"), "id": 670301119562670080, "text": "Tonight's forecast is overcast and a low of 30°F. #BuyNothingDay #NationalNativeAmericanHeritageDay #BlackFriday #PinsAndNeedlesDay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.5333,41.5167"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BuyNothingDay", "NationalNativeAmericanHeritageDay", "BlackFriday", "PinsAndNeedlesDay" }}, "user": { "id": 42666285, "name": "Quad Cities Weather", "screen_name": "qcweather", "lang": "en", "location": "Quad Cities", "create_at": date("2009-05-26"), "description": "Is it going to be warm, wet or cold in the Quad Cities today? You can also find us on Facebook.", "followers_count": 734, "friends_count": 0, "statues_count": 10473 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rock Island Arsenal, IL", "id": "8f966c3da836a196", "name": "Rock Island Arsenal", "place_type": "city", "bounding_box": rectangle("-90.570223,41.509992 -90.511303,41.523635") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17161, "countyName": "Rock Island", "cityID": 1765104, "cityName": "Rock Island Arsenal" } }
+{ "create_at": datetime("2015-11-27T10:00:15.000Z"), "id": 670301119814475780, "text": "Finna make a movie chapo in the cut is the title", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3485969476, "name": "ICHAPO", "screen_name": "lawsoninthecut", "lang": "en", "location": "Virginia, USA", "create_at": date("2015-08-29"), "description": "USMC ✊✊. Sc:CHAPOSOSA", "followers_count": 79, "friends_count": 79, "statues_count": 1622 }, "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("2015-11-27T10:00:15.000Z"), "id": 670301119902429184, "text": "Worst kind of niggas �� https://t.co/Av0IO8dCl4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3191199032, "name": "JOI T$UNAMI", "screen_name": "JoiAboutHers", "lang": "en", "location": "null", "create_at": date("2015-05-10"), "description": "✨ I AM THE INFLUENCE ✨", "followers_count": 203, "friends_count": 173, "statues_count": 4409 }, "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("2015-11-27T10:00:15.000Z"), "id": 670301119914991616, "text": "We're #hiring! Read about our latest #job opening here: Driver Opportunity - Big... - https://t.co/R10VDQN29y https://t.co/g57ar5vhfv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.3339324,42.4795019"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job" }}, "user": { "id": 2414879204, "name": "JOBS at AIM", "screen_name": "JobsAtAim", "lang": "en", "location": "www.aimntls.com", "create_at": date("2014-03-27"), "description": "STABILITY. ADVANCEMENT. IMPACT. We're looking for professionals like you to join our team!\nApply to our open #jobs below, and follow us here @aimntls", "followers_count": 27, "friends_count": 4, "statues_count": 75 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dunkirk, NY", "id": "7561ff4fba7e382d", "name": "Dunkirk", "place_type": "city", "bounding_box": rectangle("-79.400179,42.45681 -79.295345,42.506593") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36013, "countyName": "Chautauqua", "cityID": 3621105, "cityName": "Dunkirk" } }
+{ "create_at": datetime("2015-11-27T10:00:15.000Z"), "id": 670301120011481088, "text": "#CustomerService #Job in #Thomaston, ME: CSA Front End at Lowe's https://t.co/STYhPFM98r #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-69.1816955,44.0789574"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CustomerService", "Job", "Thomaston", "Jobs", "Hiring" }}, "user": { "id": 71973526, "name": "ME Cust. Srv. Jobs", "screen_name": "tmj_ME_cstsrv", "lang": "en", "location": "Maine", "create_at": date("2009-09-05"), "description": "Follow this account for geo-targeted Customer Service job tweets in Maine Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 275, "friends_count": 274, "statues_count": 39 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thomaston, ME", "id": "ea97d4d74a3fe542", "name": "Thomaston", "place_type": "city", "bounding_box": rectangle("-69.209542,44.062181 -69.13246,44.098596") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23013, "countyName": "Knox", "cityID": 2376330, "cityName": "Thomaston" } }
+{ "create_at": datetime("2015-11-27T10:00:15.000Z"), "id": 670301120317792256, "text": "We are open today until 5:30, if need something. #thanksgivingweekend #shoplocal #athensga @ House… https://t.co/XypPgsIUS6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.3953781,33.9473267"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thanksgivingweekend", "shoplocal", "athensga" }}, "user": { "id": 2516163110, "name": "House Electric", "screen_name": "HouseElectricGA", "lang": "en", "location": "1045 Baxter St. Athens,GA", "create_at": date("2014-05-22"), "description": "We sell light fixtures and home decor at any budget. Come see us in Athens and remember to #shoplocal wherever you are!", "followers_count": 164, "friends_count": 462, "statues_count": 484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Athens, GA", "id": "01f9c9fd7bb1aa6b", "name": "Athens", "place_type": "city", "bounding_box": rectangle("-83.521866,33.877554 -83.277693,34.035918") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13059, "countyName": "Clarke", "cityID": 1303440, "cityName": "Athens-Clarke County unified government (balance)" } }
+{ "create_at": datetime("2015-11-27T10:00:15.000Z"), "id": 670301120510717952, "text": "#VandalismCriminalMischief at 700 Grand St. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.389622,28.52496"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VandalismCriminalMischief", "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": 239, "friends_count": 1, "statues_count": 106368 }, "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("2015-11-27T10:00:15.000Z"), "id": 670301120523313153, "text": "I honestly don't remember the last time I got a haircut (at @GreatClips in Scottsdale, AZ) https://t.co/p5TsZMJQjI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.91202106,33.50961546"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21577629 }}, "user": { "id": 590722256, "name": "Kjersten Watkins", "screen_name": "kageruli", "lang": "en", "location": "null", "create_at": date("2012-05-26"), "description": "null", "followers_count": 21, "friends_count": 168, "statues_count": 864 }, "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("2015-11-27T10:00:15.000Z"), "id": 670301120544161792, "text": "HS practice in 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": 621538475, "name": "benevidez", "screen_name": "ABenevidez", "lang": "en", "location": "insta: abenevidez_", "create_at": date("2012-06-28"), "description": "✺ TCC GK & Pitcher ⚾️⚽️ // 18 & blessed ✺ taft college.", "followers_count": 1134, "friends_count": 780, "statues_count": 57354 }, "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("2015-11-27T10:00:15.000Z"), "id": 670301120569339905, "text": "Cloudy this afternoon, high 77 (25 C). Low 62 (17 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575147, "name": "Baton Rouge Weather", "screen_name": "BatonRougeWX", "lang": "en", "location": "Baton Rouge, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Baton Rouge, LA", "followers_count": 2858, "friends_count": 92, "statues_count": 7756 }, "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("2015-11-27T10:00:15.000Z"), "id": 670301120691052544, "text": "Houston's defense came ready to play thus far!! #NAVYvsHOU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NAVYvsHOU" }}, "user": { "id": 570376457, "name": "Van Hiles", "screen_name": "vstyles17", "lang": "en", "location": "Global", "create_at": date("2012-05-03"), "description": "Louisiana native, UK/NFL player, Product development firm owner, Golf enthusiast, Grillmaster", "followers_count": 135, "friends_count": 205, "statues_count": 8605 }, "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("2015-11-27T10:00:15.000Z"), "id": 670301121039110144, "text": "already. https://t.co/DayXebEhLt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 40681203, "name": "BIG Bank Hank", "screen_name": "MichaelStrauss_", "lang": "en", "location": "Trill, Texas", "create_at": date("2009-05-17"), "description": "For All Mines I'll Lay Yours; MOB", "followers_count": 614, "friends_count": 132, "statues_count": 88793 }, "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("2015-11-27T10:00:15.000Z"), "id": 670301121236176896, "text": "Loaded potato �� from tubers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1695471295, "name": "teeeeee♍️", "screen_name": "SmithLatecia", "lang": "en", "location": "null", "create_at": date("2013-08-23"), "description": "tee, young independent go getta, spoiled , chase is my world, living life", "followers_count": 1659, "friends_count": 2007, "statues_count": 34238 }, "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("2015-11-27T10:00:15.000Z"), "id": 670301121613848576, "text": "#HouseBusinessCheck at Lee Vista Blvd & S Chickasaw Trail. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2678054,28.4686465"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HouseBusinessCheck", "orlpol", "opd" }}, "user": { "id": 39129862, "name": "Police Calls 32829", "screen_name": "orlpol32829", "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": 94, "friends_count": 1, "statues_count": 8947 }, "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("2015-11-27T10:00:15.000Z"), "id": 670301121789820928, "text": "Check out our #listing in #Crestview #FL https://t.co/H04mMVBx4T #realestate #realtor https://t.co/zLOpWRQSMx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.444105,30.762696"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "Crestview", "FL", "realestate", "realtor" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6585 }, "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": 12091, "countyName": "Okaloosa" } }
+{ "create_at": datetime("2015-11-27T10:00:15.000Z"), "id": 670301121806667777, "text": "@1942bs @maomichu and I'm not allowed to help anyone I want to regardless on a paycheck or not? Lol goodbye.", "in_reply_to_status": 670300827039506433, "in_reply_to_user": 425569289, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 425569289, 42278243 }}, "user": { "id": 473524022, "name": "Gabriel", "screen_name": "go0ley", "lang": "en", "location": "null", "create_at": date("2012-01-24"), "description": "I'm a trainer, I like carrot cake and I probably don't like you. Houston, Texas. ✉️: gabe.gooley@gmail.com", "followers_count": 4366, "friends_count": 498, "statues_count": 87911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-27T10:00:15.000Z"), "id": 670301122020667402, "text": "I Thought You Was Someone Special❗️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3434239335, "name": "Mi No Compadre", "screen_name": "ShaunShizzyShiz", "lang": "en", "location": "Maintaining", "create_at": date("2015-08-21"), "description": "#LongLiveJohnJohn #FreeMyGang", "followers_count": 65, "friends_count": 80, "statues_count": 106 }, "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-27T10:00:15.000Z"), "id": 670301122054250496, "text": "im listening to strip it down by luke bryan and my grandma comes into my room and goes \"this is my favorite song\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 288985526, "name": "white mom", "screen_name": "giselleVEVO", "lang": "en", "location": "mccalla, al", "create_at": date("2011-04-27"), "description": "you have to be odd to be number one :)", "followers_count": 5040, "friends_count": 1649, "statues_count": 128497 }, "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("2015-11-27T10:00:15.000Z"), "id": 670301122070863874, "text": "Ugh it jst fckin bugs wjen she post stupid shit 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": 2464946534, "name": "Chula Loc14 ❤", "screen_name": "Pinche_Paola", "lang": "en", "location": "null", "create_at": date("2014-04-26"), "description": "NorthSide :: XIV Puto! Taken ❤11.06.15❤", "followers_count": 438, "friends_count": 194, "statues_count": 11071 }, "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("2015-11-27T10:00:15.000Z"), "id": 670301122116976640, "text": "Santa was just spotted in #OnwardReserve poncecitymarket... Almost everyone was excited to see him!… https://t.co/ytNFMmmNi2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.3660736,33.7722206"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "OnwardReserve" }}, "user": { "id": 557819363, "name": "OnwardReserve", "screen_name": "OnwardReserve", "lang": "en", "location": "null", "create_at": date("2012-04-19"), "description": "The official Twitter of Onward Reserve.com, the location for men's apparel and lifestyle guide. Follow us for style news and opinion and lifestyle guides.", "followers_count": 3148, "friends_count": 136, "statues_count": 1313 }, "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("2015-11-27T10:00:15.000Z"), "id": 670301122461085696, "text": "@mickeyydeee I guess I can't argue with that too much", "in_reply_to_status": 670300827496652800, "in_reply_to_user": 187371977, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 187371977 }}, "user": { "id": 152329162, "name": "Madeleine Silvey", "screen_name": "MAKSilvey", "lang": "en", "location": "null", "create_at": date("2010-06-05"), "description": "Drake's ghost writer~", "followers_count": 793, "friends_count": 626, "statues_count": 13018 }, "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("2015-11-27T10:00:15.000Z"), "id": 670301122490429440, "text": "Hell naw ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 447242069, "name": "DaddyT", "screen_name": "Unknown__Killer", "lang": "en", "location": "null", "create_at": date("2011-12-26"), "description": "aint no bio biihhhhh", "followers_count": 1433, "friends_count": 1303, "statues_count": 27285 }, "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("2015-11-27T10:00:15.000Z"), "id": 670301122641268736, "text": "@CrispyDanny_J it's a day before mines lol", "in_reply_to_status": 670253675634278402, "in_reply_to_user": 379858479, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 379858479 }}, "user": { "id": 607961154, "name": "alyssa", "screen_name": "_marielyssa", "lang": "en", "location": "Dallas, TX", "create_at": date("2012-06-13"), "description": "love you Josh, rest in paradise \r\n\r\n#A1SinceDay1", "followers_count": 1086, "friends_count": 431, "statues_count": 35113 }, "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-27T10:00:15.000Z"), "id": 670301123064889344, "text": "Department of Veterans Affairs #Healthcare #Job: PHARMACY TECHNICIAN (#Honolulu, HI) https://t.co/qHGWaRQzyu #Veterans #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8583333,21.3069444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "Job", "Honolulu", "Veterans", "Jobs", "Hiring" }}, "user": { "id": 23526678, "name": "TMJ- HON Health Jobs", "screen_name": "tmj_hon_health", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-03-09"), "description": "Follow this account for geo-targeted Healthcare job tweets in Honolulu, HI. Need help? Tweet us at @CareerArc!", "followers_count": 406, "friends_count": 304, "statues_count": 185 }, "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("2015-11-27T10:00:15.000Z"), "id": 670301123077660672, "text": "Best thing about after thanksgiving? The leftovers. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 183672104, "name": "Dana Jay", "screen_name": "danaj94", "lang": "en", "location": "Indianapolis, Indiana", "create_at": date("2010-08-27"), "description": "Wisco|Indy Butler University.", "followers_count": 650, "friends_count": 827, "statues_count": 22619 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pleasant Prairie, WI", "id": "01ce73922fe260b2", "name": "Pleasant Prairie", "place_type": "city", "bounding_box": rectangle("-87.952181,42.492766 -87.801335,42.574318") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55059, "countyName": "Kenosha", "cityID": 5563300, "cityName": "Pleasant Prairie" } }
+{ "create_at": datetime("2015-11-27T10:00:16.000Z"), "id": 670301123186569216, "text": "Mostly cloudy this afternoon, high 77 (25 C). Low 63 (17 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575204, "name": "New Orleans Weather", "screen_name": "NewOrleansWX", "lang": "en", "location": "New Orleans, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for New Orleans, LA", "followers_count": 2897, "friends_count": 92, "statues_count": 7774 }, "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("2015-11-27T10:00:16.000Z"), "id": 670301123509620736, "text": "Sometimes when there's no space in the whip you gotta have the nigga who ain't put in towards the weed walk to the next destination ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2240390040, "name": "Frank Lucas", "screen_name": "FlossyDinero", "lang": "en", "location": "null", "create_at": date("2013-12-11"), "description": "19 years young; NYC", "followers_count": 88, "friends_count": 91, "statues_count": 2886 }, "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-27T10:00:16.000Z"), "id": 670301123626975232, "text": "Except for like three people �� https://t.co/XWUkNHglBF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 201002636, "name": "southern goth ♣️", "screen_name": "BaltoAngel", "lang": "en", "location": "Nechanitz, Texas", "create_at": date("2010-10-10"), "description": "amy lee. kelly clarkson. lgbt. beh. lana parrilla. chris young. adele. itm, ffdp, gaga, halestorm. spn, ouat, twd, soa, gac, lotr, hp. rtc. texas. 11-15-15 ✌️✨", "followers_count": 1176, "friends_count": 1757, "statues_count": 8121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, TX", "id": "4cb67de7b2f50d0a", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-96.901163,29.8832 -96.854642,29.932747") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48149, "countyName": "Fayette", "cityID": 4840276, "cityName": "La Grange" } }
+{ "create_at": datetime("2015-11-27T10:00:16.000Z"), "id": 670301123648086017, "text": "Hummus baby! @kababparadise @ Kabab Paradise https://t.co/FO6hzjjWe0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.3821618,40.88094702"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 73240927 }}, "user": { "id": 66321660, "name": "Veronique N. Deblois", "screen_name": "foodwinechickie", "lang": "en", "location": "NY / NJ Metro", "create_at": date("2009-08-17"), "description": "Food blogger / wine blogger #craftbeer #socialmedia #Eagles #Flyers Also at @travelv Wine Editor @jerseybites Instagram at foodwinechickie #FoodieChats", "followers_count": 9890, "friends_count": 9365, "statues_count": 23314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Hiawatha, NJ", "id": "00b2b97902b39bef", "name": "Lake Hiawatha", "place_type": "city", "bounding_box": rectangle("-74.420492,40.844743 -74.340821,40.902021") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34027, "countyName": "Morris" } }
+{ "create_at": datetime("2015-11-27T10:00:16.000Z"), "id": 670301123920588800, "text": "This unreleased music goes dumb. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 870941168, "name": "TORRE$", "screen_name": "TorrestheMC", "lang": "en", "location": "$$$, TX. ", "create_at": date("2012-10-09"), "description": "lost in my mind somewhere. Business - RODRIGUEZRAF97@GMAIL.COM MGMT: @LILRFROMTEXAS", "followers_count": 1736, "friends_count": 976, "statues_count": 44600 }, "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("2015-11-27T10:00:16.000Z"), "id": 670301124549853184, "text": "Literally bought half of Victoria's Secret and mom's making me wait till Christmas:,-)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 796088401, "name": "Landis Davidson", "screen_name": "landisdavidson", "lang": "en", "location": "nc", "create_at": date("2012-09-01"), "description": "a little lost, a little found", "followers_count": 569, "friends_count": 404, "statues_count": 4220 }, "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("2015-11-27T10:00:16.000Z"), "id": 670301124814049280, "text": "Too much stuff that was moved in ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160321438, "name": "Shan M.", "screen_name": "shanbann_", "lang": "en", "location": "Somewhere you not ..", "create_at": date("2010-06-27"), "description": "WSSU19 Miss Daddy #RIP", "followers_count": 1490, "friends_count": 2087, "statues_count": 42279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Griffin, GA", "id": "a846e2ce51e1233f", "name": "Griffin", "place_type": "city", "bounding_box": rectangle("-84.317652,33.202291 -84.228626,33.295471") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13255, "countyName": "Spalding", "cityID": 1335324, "cityName": "Griffin" } }
+{ "create_at": datetime("2015-11-27T10:00:16.000Z"), "id": 670301124830842880, "text": "I'm always in the mood for ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 587755694, "name": "❥ ❥ǰ∊ssأʗɑ", "screen_name": "jessicalynn5777", "lang": "en", "location": "☾☮ new york ", "create_at": date("2012-05-22"), "description": "❁ insta: jessicalynn57775.", "followers_count": 414, "friends_count": 553, "statues_count": 13532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Northport, NY", "id": "9530ea3d29844638", "name": "East Northport", "place_type": "city", "bounding_box": rectangle("-73.35523,40.862419 -73.286426,40.900011") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3622612, "cityName": "East Northport" } }
+{ "create_at": datetime("2015-11-27T10:00:16.000Z"), "id": 670301124885377025, "text": "I can't do half of this without heels https://t.co/ONVzVV5G14", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 501439771, "name": "Chelsea", "screen_name": "_Chelseeeeea", "lang": "en", "location": "null", "create_at": date("2012-02-23"), "description": "lil rae baby...✨", "followers_count": 1016, "friends_count": 491, "statues_count": 42356 }, "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("2015-11-27T10:00:16.000Z"), "id": 670301125237563392, "text": "@randominitea Good!! This makes me think I should've done this for Troy", "in_reply_to_status": 670300871222288384, "in_reply_to_user": 472648874, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 472648874 }}, "user": { "id": 1003373696, "name": "um that's Liam", "screen_name": "Thishawtjail", "lang": "en", "location": "los angeles-miami", "create_at": date("2012-12-10"), "description": "mostly for talkin bout music and entertainment,especially LP/1d. if you dont like liam payne dont bother. stream/buy MITAM", "followers_count": 607, "friends_count": 562, "statues_count": 58415 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-27T10:00:16.000Z"), "id": 670301125334159360, "text": "@LeannSanto ������ its so cruel but also funny", "in_reply_to_status": 670300943963963392, "in_reply_to_user": 2728259201, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2728259201 }}, "user": { "id": 575106641, "name": "MEEK⚡️", "screen_name": "King_Shit5", "lang": "en", "location": "Cloud9", "create_at": date("2012-05-08"), "description": "NO! Not Like Meek Mill #TheHIVE", "followers_count": 1084, "friends_count": 1295, "statues_count": 27234 }, "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("2015-11-27T10:00:16.000Z"), "id": 670301125388693504, "text": "Think I'm gonna watch OTH again", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2388267686, "name": "Cali", "screen_name": "calista_miles", "lang": "en", "location": "Junior @ Alliance High", "create_at": date("2014-03-13"), "description": "| People Always Leave | Follow my son @_davidearley_ | Varsity Cheerleader | Track | Dance | @justinbieber ❤| #PURPOSE | #SkilliesMadeIt |", "followers_count": 1252, "friends_count": 481, "statues_count": 29589 }, "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("2015-11-27T10:00:16.000Z"), "id": 670301125749272577, "text": "Afternoon showers this afternoon, high 73 (23 C). Low 64 (18 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575255, "name": "Shreveport Weather", "screen_name": "ShreveportWX", "lang": "en", "location": "Shreveport, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Shreveport, LA", "followers_count": 691, "friends_count": 92, "statues_count": 7832 }, "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("2015-11-27T10:00:16.000Z"), "id": 670301125904629760, "text": "Stringing up the lights! #boston #christmas @ Boston Christmas Trees https://t.co/sGbVeqSIuv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.1324692,42.3550987"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "boston", "christmas" }}, "user": { "id": 95107576, "name": "BostonChristmasTrees", "screen_name": "bostonxmastrees", "lang": "en", "location": "22 Harvard Ave., Allston, MA", "create_at": date("2009-12-06"), "description": "Located at 22 Harvard Ave. in Allston, serving Boston, Brighton, Brookline, Allston, Cambridge, Newton, Watertown, and beyond since 1969!", "followers_count": 111, "friends_count": 137, "statues_count": 230 }, "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("2015-11-27T10:00:16.000Z"), "id": 670301126026104832, "text": "Temp 43.8°F Falling, Pressure 30.304in Steady, Dew point 42.5°, Wind N 0mph, Rain today 0.46in https://t.co/gPtnr2dDtV #STLwx #MOwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.35555556,38.47694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "STLwx", "MOwx" }}, "user": { "id": 1191304230, "name": "Oakville Weather", "screen_name": "OakvilleWx", "lang": "en", "location": "Oakville, Missouri, USA", "create_at": date("2013-02-17"), "description": "Current conditions on the hour brought to you by @DaveStLou", "followers_count": 87, "friends_count": 80, "statues_count": 23336 }, "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("2015-11-27T10:00:16.000Z"), "id": 670301126110126080, "text": "when u really wanna be part of the grext :| https://t.co/pq58jJwJpi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 283040298, "name": "Payton Lima", "screen_name": "its_PAY_day", "lang": "en", "location": "null", "create_at": date("2011-04-16"), "description": "ECE alumni | MHSVC 2x d1 state champ | uTampa cheerleading coed black '19 ❤️⚫️ | everyday is pay day", "followers_count": 987, "friends_count": 885, "statues_count": 16111 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Methuen Town, MA", "id": "01597161672b6499", "name": "Methuen Town", "place_type": "city", "bounding_box": rectangle("-71.255938,42.6704 -71.115547,42.794273") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25009, "countyName": "Essex", "cityID": 2540710, "cityName": "Methuen Town" } }
+{ "create_at": datetime("2015-11-27T10:00:16.000Z"), "id": 670301126256783360, "text": "@brittanyrenee64 same! ��", "in_reply_to_status": 670297436422668288, "in_reply_to_user": 127295312, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 127295312 }}, "user": { "id": 141069060, "name": "hello~clouds", "screen_name": "anaa_xo", "lang": "en", "location": "Los Angeles, California", "create_at": date("2010-05-06"), "description": "Melt my happiness, some kind of fucked up mess", "followers_count": 758, "friends_count": 742, "statues_count": 37190 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-27T10:00:16.000Z"), "id": 670301126386806784, "text": "@monicahbabee Why?", "in_reply_to_status": 670300089139642368, "in_reply_to_user": 168342766, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 168342766 }}, "user": { "id": 2457218882, "name": "avril", "screen_name": "AvrilAguileraa", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2014-04-21"), "description": "null", "followers_count": 526, "friends_count": 420, "statues_count": 6449 }, "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("2015-11-27T10:00:16.000Z"), "id": 670301126600847360, "text": "My hair dresser is currently online shopping instead of doing my hair.. K.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 265141780, "name": "Ayanna Morgan", "screen_name": "a_marshayxo", "lang": "en", "location": "Middletown | Athens", "create_at": date("2011-03-12"), "description": "Ohio University '18", "followers_count": 1042, "friends_count": 722, "statues_count": 42332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monroe, OH", "id": "000aa1f231c00b1d", "name": "Monroe", "place_type": "city", "bounding_box": rectangle("-84.462794,39.413012 -84.278288,39.476285") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39017, "countyName": "Butler", "cityID": 3951310, "cityName": "Monroe" } }
+{ "create_at": datetime("2015-11-27T10:00:16.000Z"), "id": 670301126609100801, "text": "Feels good to be apreciated https://t.co/dLuw69Ccts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2558054857, "name": "Мayan M.", "screen_name": "obeyladymars", "lang": "en", "location": "PSHS", "create_at": date("2014-06-09"), "description": "》》I eat what you feel, And I'm insatiable.《《\nprofessional Dendro and Astrophilist《\nVeni.\nVeci.\nVedi.", "followers_count": 340, "friends_count": 653, "statues_count": 9288 }, "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("2015-11-27T10:00:16.000Z"), "id": 670301126705696768, "text": "I gonna get out of bed, go get food, and bring it back 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": 1321717130, "name": "Alex Sudduth", "screen_name": "alexsudduth22", "lang": "en", "location": "Pace, FL", "create_at": date("2013-04-01"), "description": "snapchat• suddutha22 insta• alexsudduth22", "followers_count": 481, "friends_count": 215, "statues_count": 2247 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pace, FL", "id": "bbf79409c9a1df68", "name": "Pace", "place_type": "city", "bounding_box": rectangle("-87.213989,30.577338 -87.102923,30.648512") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12113, "countyName": "Santa Rosa", "cityID": 1253725, "cityName": "Pace" } }
+{ "create_at": datetime("2015-11-27T10:00:16.000Z"), "id": 670301126793805825, "text": "@CbM_Project lol fr", "in_reply_to_status": 670300461140938752, "in_reply_to_user": 291319356, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 291319356 }}, "user": { "id": 566425718, "name": "Murk.", "screen_name": "iamMurk_", "lang": "en", "location": "Kinghill... Murkgomery,AL", "create_at": date("2012-04-29"), "description": "follow me on the gram @maddymax_", "followers_count": 3977, "friends_count": 2063, "statues_count": 77546 }, "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("2015-11-27T10:00:16.000Z"), "id": 670301126881734656, "text": "@settern Congrats! I proposed to my wife on one knee, too. #likeagentleman", "in_reply_to_status": 670298282082570241, "in_reply_to_user": 14957252, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "likeagentleman" }}, "user_mentions": {{ 14957252 }}, "user": { "id": 14593458, "name": "Jeff Kirvin", "screen_name": "jeffkirvin", "lang": "en", "location": "Denver", "create_at": date("2008-04-29"), "description": "I Fight For The Users", "followers_count": 770, "friends_count": 151, "statues_count": 26433 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Northglenn, CO", "id": "1fff770504a7d5a5", "name": "Northglenn", "place_type": "city", "bounding_box": rectangle("-105.015875,39.868118 -104.944623,39.921489") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8001, "countyName": "Adams", "cityID": 854330, "cityName": "Northglenn" } }
+{ "create_at": datetime("2015-11-27T10:00:16.000Z"), "id": 670301126936281088, "text": "Wow, I should’ve started my ride earlier: https://t.co/mGM46TeMZu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14848078, "name": "Alec Perkins", "screen_name": "alecperkins", "lang": "en", "location": "Hoboken, NJ", "create_at": date("2008-05-20"), "description": "Designer with a coding problem. Building content tools at @marquee.\n\n[human Human]", "followers_count": 884, "friends_count": 616, "statues_count": 32971 }, "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-27T10:00:16.000Z"), "id": 670301127116632065, "text": "Can you recommend anyone for this #Accounting #job? https://t.co/kECUGVepWv #GLENALLEN, VA #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.5063739,37.665978"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Accounting", "job", "GLENALLEN", "Hiring", "CareerArc" }}, "user": { "id": 26811878, "name": "TMJ-RCH Acct. Jobs", "screen_name": "tmj_rch_acct", "lang": "en", "location": "Richmond, VA", "create_at": date("2009-03-26"), "description": "Follow this account for geo-targeted Accounting job tweets in Richmond, VA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 371, "friends_count": 296, "statues_count": 54 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glen Allen, VA", "id": "2c98dd736bb2e02f", "name": "Glen Allen", "place_type": "city", "bounding_box": rectangle("-77.527806,37.579878 -77.407641,37.694137") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51087, "countyName": "Henrico", "cityID": 5131200, "cityName": "Glen Allen" } }
+{ "create_at": datetime("2015-11-27T10:00:16.000Z"), "id": 670301127125127168, "text": "anyone wanna hang", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 928671691, "name": "coach j", "screen_name": "jewwmorgan", "lang": "en", "location": "Boston Red Sox ", "create_at": date("2012-11-05"), "description": "lions don't lose sleep over the opinion of sheep married to hoodie Allen since 2011", "followers_count": 351, "friends_count": 298, "statues_count": 23260 }, "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("2015-11-27T10:00:16.000Z"), "id": 670301127150301184, "text": "@jplunkz I'm about to try buying some today", "in_reply_to_status": 670270821538127872, "in_reply_to_user": 2288732027, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2288732027 }}, "user": { "id": 584923416, "name": "jess maxwell", "screen_name": "jaybabymax", "lang": "en", "location": "Stockton U '18", "create_at": date("2012-05-19"), "description": "i like burritos and traveling", "followers_count": 384, "friends_count": 305, "statues_count": 7487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tuckerton, NJ", "id": "c47ea510a5f285e6", "name": "Tuckerton", "place_type": "city", "bounding_box": rectangle("-74.390565,39.573212 -74.310431,39.638258") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean", "cityID": 3474210, "cityName": "Tuckerton" } }
+{ "create_at": datetime("2015-11-27T10:00:16.000Z"), "id": 670301127259373568, "text": "I always forget and ingredient after I put it in 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": 401579848, "name": "Lauren Bauer", "screen_name": "Lbau3r17", "lang": "en", "location": "null", "create_at": date("2011-10-30"), "description": "❥M.R❥", "followers_count": 594, "friends_count": 633, "statues_count": 28335 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Selden, NY", "id": "fe93976a4f056ccb", "name": "Selden", "place_type": "city", "bounding_box": rectangle("-73.065046,40.839865 -73.026696,40.903175") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3666212, "cityName": "Selden" } }
+{ "create_at": datetime("2015-11-27T10:00:16.000Z"), "id": 670301127305469952, "text": "I just got sperry's for $23����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 250924750, "name": "reen rose", "screen_name": "sarynaroberts", "lang": "en", "location": "null", "create_at": date("2011-02-11"), "description": "bad day not a bad life", "followers_count": 898, "friends_count": 388, "statues_count": 12806 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willow Grove, PA", "id": "b473d3b0c8ee5cf5", "name": "Willow Grove", "place_type": "city", "bounding_box": rectangle("-75.143114,40.129922 -75.091487,40.171482") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42091, "countyName": "Montgomery", "cityID": 4285408, "cityName": "Willow Grove" } }
+{ "create_at": datetime("2015-11-27T10:00:17.000Z"), "id": 670301127443808256, "text": "@woedoctor @poglankford @VocalMinoritySD and the old brown is clashy with the new brown.", "in_reply_to_status": 670300271583428609, "in_reply_to_user": 131884083, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 131884083, 2361775909, 137751937 }}, "user": { "id": 238418348, "name": "Kevin Thompson", "screen_name": "ParkAtThePark", "lang": "en", "location": "Minneapolis MN", "create_at": date("2011-01-14"), "description": "Just a Padres fan since 1984", "followers_count": 1195, "friends_count": 877, "statues_count": 28089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shorewood, MN", "id": "681f2a239b2b0db0", "name": "Shorewood", "place_type": "city", "bounding_box": rectangle("-93.646539,44.89089 -93.523058,44.920585") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2760016, "cityName": "Shorewood" } }
+{ "create_at": datetime("2015-11-27T10:00:17.000Z"), "id": 670301127519301632, "text": "What's 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": 893775216, "name": "A.", "screen_name": "__luvA", "lang": "en", "location": "with T5 ", "create_at": date("2012-10-20"), "description": "null", "followers_count": 821, "friends_count": 684, "statues_count": 76536 }, "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("2015-11-27T10:00:17.000Z"), "id": 670301127531823104, "text": "Thank you baby �� https://t.co/KkQFpAwke9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 170835352, "name": "peaches", "screen_name": "_erikamichellem", "lang": "en", "location": "null", "create_at": date("2010-07-25"), "description": "I love Beyoncé", "followers_count": 1828, "friends_count": 862, "statues_count": 86976 }, "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-27T10:00:17.000Z"), "id": 670301127619928064, "text": "We're #hiring! Click to apply: ELECTRICIAN - https://t.co/fdkKNOkhZ7 #FacilitiesMgmt #WestHaven, CT #Veterans #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.9470471,41.2706527"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "FacilitiesMgmt", "WestHaven", "Veterans", "Job", "Jobs", "CareerArc" }}, "user": { "id": 173541362, "name": "CT Facility Mgmt.", "screen_name": "tmj_CT_facmgmt", "lang": "en", "location": "Connecticut", "create_at": date("2010-08-01"), "description": "Follow this account for geo-targeted Facilities Management job tweets in Connecticut Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 135, "friends_count": 125, "statues_count": 17 }, "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("2015-11-27T10:00:17.000Z"), "id": 670301127733211137, "text": "bring on the Christmas movies��☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 730039218, "name": "Laura Ramirez", "screen_name": "lramirez2017", "lang": "en", "location": "Blackwell, Oklahoma ", "create_at": date("2012-07-31"), "description": "null", "followers_count": 948, "friends_count": 347, "statues_count": 3536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Blackwell, OK", "id": "65917418c2ba180b", "name": "Blackwell", "place_type": "city", "bounding_box": rectangle("-97.319024,36.776634 -97.266768,36.818722") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40071, "countyName": "Kay", "cityID": 4006600, "cityName": "Blackwell" } }
+{ "create_at": datetime("2015-11-27T10:00:17.000Z"), "id": 670301128106471424, "text": "I swear man this dude is so annoying", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 363911823, "name": "Erick", "screen_name": "ThatFooErick", "lang": "en", "location": "Ontario", "create_at": date("2011-08-28"), "description": "Sadie❤️", "followers_count": 263, "friends_count": 196, "statues_count": 38037 }, "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("2015-11-27T10:00:17.000Z"), "id": 670301128148537344, "text": "work, work, work", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2593348452, "name": "Lisa McMurry", "screen_name": "7043006763", "lang": "en", "location": "null", "create_at": date("2014-06-28"), "description": "null", "followers_count": 130, "friends_count": 33, "statues_count": 17680 }, "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("2015-11-27T10:00:17.000Z"), "id": 670301128186269696, "text": "I'll sleep like a baby if I had this https://t.co/YQdgiXcOyX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 929188747, "name": "raised by wolves", "screen_name": "deathtoodeaunte", "lang": "en", "location": "null", "create_at": date("2012-11-05"), "description": "FREEJOESPH", "followers_count": 192, "friends_count": 157, "statues_count": 5907 }, "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("2015-11-27T10:00:17.000Z"), "id": 670301128207245312, "text": "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": 970980331, "name": "tab", "screen_name": "tszymczak", "lang": "en", "location": "oh/-nc", "create_at": date("2012-11-25"), "description": "nw senior", "followers_count": 626, "friends_count": 385, "statues_count": 49679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Ridge, NC", "id": "00e7e25b3b6dbd8f", "name": "Oak Ridge", "place_type": "city", "bounding_box": rectangle("-80.03646,36.129405 -79.950553,36.220418") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3748480, "cityName": "Oak Ridge" } }
+{ "create_at": datetime("2015-11-27T10:00:17.000Z"), "id": 670301128538480641, "text": "Cloudy/wind this afternoon, high 77 (25 C). Low 64 (18 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575312, "name": "Lake Charles Weather", "screen_name": "LakeCharlesWX", "lang": "en", "location": "Lake Charles, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Lake Charles, LA", "followers_count": 536, "friends_count": 92, "statues_count": 7816 }, "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-27T10:00:17.000Z"), "id": 670301128559464448, "text": "[Fri Nov 27th, 12:00pm] Temperature: 66.38 F, 19.1 C; Humidity: 41.7%; Light: 24.32%. https://t.co/UI5qWnQ21d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 269430133, "name": "Shoemaker Sandbox", "screen_name": "shoe_sandbox", "lang": "en", "location": "Earth (for now). ", "create_at": date("2011-03-20"), "description": "I am a bot. Developer account for @shoemaker.", "followers_count": 7, "friends_count": 1, "statues_count": 23570 }, "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("2015-11-27T10:00:17.000Z"), "id": 670301128794345472, "text": "Don't complain about trump eventually he'll 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": 2907721738, "name": "samlozada", "screen_name": "samlozadajr", "lang": "en", "location": "Culver City, CA", "create_at": date("2014-12-06"), "description": "null", "followers_count": 46, "friends_count": 22, "statues_count": 26423 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-27T10:00:17.000Z"), "id": 670301128970493952, "text": "Every time I see a cashier I call them \"Boss\" there's a reason behind it tho.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 902955973, "name": "LayLow", "screen_name": "lalobustillos", "lang": "en", "location": "DogTown, Tx ", "create_at": date("2012-10-24"), "description": "Disturbing The Public", "followers_count": 728, "friends_count": 586, "statues_count": 21526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manor, TX", "id": "014ad9c3076d0676", "name": "Manor", "place_type": "city", "bounding_box": rectangle("-97.574266,30.332117 -97.507788,30.377591") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4846440, "cityName": "Manor" } }
+{ "create_at": datetime("2015-11-27T10:00:17.000Z"), "id": 670301129104687104, "text": "Majestic buds. @ South Mountain Park https://t.co/TEpFAcuNjw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.06944444,33.33611111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 838524716, "name": "Valerie", "screen_name": "violentresval", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2012-09-21"), "description": "Grief counselor. Funny guy. Photographer. Co-founder/Editor for @violentres. I also write for @echoesanddust and @aboutheavymetal", "followers_count": 544, "friends_count": 438, "statues_count": 4528 }, "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": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2015-11-27T10:00:17.000Z"), "id": 670301129654337538, "text": "Fashooooo https://t.co/dn3IwQqC3f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1320344910, "name": "lenn", "screen_name": "IaliyahL", "lang": "en", "location": "Indiana", "create_at": date("2013-04-01"), "description": "prolly @ work", "followers_count": 1695, "friends_count": 1707, "statues_count": 10821 }, "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("2015-11-27T10:00:17.000Z"), "id": 670301129792688128, "text": "My mama don't like you, and she likes everyone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 518963619, "name": "Shelbs", "screen_name": "Shelbysaelens", "lang": "en", "location": "null", "create_at": date("2012-03-08"), "description": "this world needed someone like me. mi - fl - la", "followers_count": 627, "friends_count": 508, "statues_count": 11620 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burton, MI", "id": "d4f8c782da0a3b7b", "name": "Burton", "place_type": "city", "bounding_box": rectangle("-83.692863,42.959625 -83.573753,43.048152") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2612060, "cityName": "Burton" } }
+{ "create_at": datetime("2015-11-27T10:00:17.000Z"), "id": 670301129868189696, "text": "I kicked Trev's butt at cornhole @ Great Smokey Mountains https://t.co/cmIWLzxcEl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.39690852,35.13199009"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 231892301, "name": "Kelly Proctor", "screen_name": "kproc57", "lang": "en", "location": "Kproc's Krib", "create_at": date("2010-12-29"), "description": "My hair smells like tacos", "followers_count": 146, "friends_count": 232, "statues_count": 1336 }, "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.32187,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37113, "countyName": "Macon" } }
+{ "create_at": datetime("2015-11-27T10:00:17.000Z"), "id": 670301130027499521, "text": "I consensus regarding New Ceilings 2", "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": "blessed.humble.educated.focused", "followers_count": 1846, "friends_count": 971, "statues_count": 111121 }, "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("2015-11-27T10:00:17.000Z"), "id": 670301130178600961, "text": "Holeyshit I just realized the tickets I ordered are on our anniversary ! Two birds one stone �� I'm good", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1005052543, "name": "KylieRae", "screen_name": "KylieRae12", "lang": "en", "location": "Pure Michigan ✋", "create_at": date("2012-12-11"), "description": "| future Mrs. Berger | child fitness instructor| personal volleyball trainer | infant teacher | workout junkie| health freak| dog mom | Wonder Woman |", "followers_count": 401, "friends_count": 297, "statues_count": 5619 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, MI", "id": "b01feb39060d4a4c", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-84.874418,42.538565 -84.778182,42.596917") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26045, "countyName": "Eaton", "cityID": 2614820, "cityName": "Charlotte" } }
+{ "create_at": datetime("2015-11-27T10:00:17.000Z"), "id": 670301130543357952, "text": "Seriously, 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": 53518319, "name": "ColinH", "screen_name": "njd3b1ink", "lang": "en", "location": "null", "create_at": date("2009-07-03"), "description": "I am faster than 80% of all snakes - Dwight Schrute", "followers_count": 56, "friends_count": 78, "statues_count": 1646 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florida, NY", "id": "72ed4dfa0d001246", "name": "Florida", "place_type": "city", "bounding_box": rectangle("-74.372774,41.31459 -74.331961,41.371199") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36071, "countyName": "Orange", "cityID": 3626319, "cityName": "Florida" } }
+{ "create_at": datetime("2015-11-27T10:00:17.000Z"), "id": 670301130560262144, "text": "Gt ten toes down mood", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "nl", "is_retweet": false, "user": { "id": 86849363, "name": "KD", "screen_name": "kd___313", "lang": "en", "location": "null", "create_at": date("2009-11-01"), "description": "LO❤\n6 Mile to PA", "followers_count": 371, "friends_count": 352, "statues_count": 12651 }, "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-27T10:00:17.000Z"), "id": 670301130585481218, "text": "Unless you want a starring role in somebody's ratchet #BlackFriday fight video, keep it classy, folks!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BlackFriday" }}, "user": { "id": 379821773, "name": "Brian Allen", "screen_name": "brianallentech", "lang": "en", "location": "Atlanta", "create_at": date("2011-09-25"), "description": "Tech/media analyst/journalist. Editor and publisher of @DNewsNation. Atlantan tweeting about tech, media, and everything else. http://muckrack.com/brian-c-allen", "followers_count": 186, "friends_count": 229, "statues_count": 17447 }, "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("2015-11-27T10:00:17.000Z"), "id": 670301131147448320, "text": "how long do you have to lead a political party's presidential polls before you ARE the establishment?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 259395895, "name": "John Harwood", "screen_name": "JohnJHarwood", "lang": "en", "location": "Washington, DC", "create_at": date("2011-03-01"), "description": "a Dad who covers Washington, the economy and national politics for CNBC and the New York Times. Find my Speakeasy interviews at http://cnbc.com/Speakeasy", "followers_count": 58142, "friends_count": 935, "statues_count": 33417 }, "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("2015-11-27T10:00:17.000Z"), "id": 670301131185106945, "text": "Mostly cloudy this afternoon, high 79 (26 C). Low 63 (17 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575370, "name": "Lafayette Weather", "screen_name": "LafayetteWX", "lang": "en", "location": "Lafayette, LA", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Lafayette, LA", "followers_count": 464, "friends_count": 92, "statues_count": 7855 }, "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("2015-11-27T10:00:17.000Z"), "id": 670301131348664320, "text": "If you're a #Purchasing professional in #BocaRaton, FL, check out this #job: https://t.co/Iq1XVycF2X #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.0830984,26.3586885"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Purchasing", "BocaRaton", "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": 903, "friends_count": 139, "statues_count": 9439 }, "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("2015-11-27T10:00:17.000Z"), "id": 670301131432677376, "text": "Emojis emojis emojis", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "lt", "is_retweet": false, "user": { "id": 2607458011, "name": "†DoubleM ⛽an⛽", "screen_name": "_Therealjbanga", "lang": "en", "location": "@jamaree ❤ ", "create_at": date("2014-07-06"), "description": "∞ TajGangOrNoGang", "followers_count": 558, "friends_count": 485, "statues_count": 8663 }, "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("2015-11-27T10:00:17.000Z"), "id": 670301131453636608, "text": "My favorites. rogersjd16 @ Lucy Vincent Beach https://t.co/zBEXR6jzCT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.72684044,41.3390364"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24228381, "name": "Rachel Rogers", "screen_name": "rhsrogers", "lang": "en", "location": "Webstah", "create_at": date("2009-03-13"), "description": "Working mom, loving wife, Skimm'Bassador", "followers_count": 103, "friends_count": 194, "statues_count": 581 }, "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.886811") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25007, "countyName": "Dukes" } }
+{ "create_at": datetime("2015-11-27T10:00:17.000Z"), "id": 670301131491405824, "text": "@elaina_foster https://t.co/LcrPBr166k", "in_reply_to_status": -1, "in_reply_to_user": 574083439, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 574083439 }}, "user": { "id": 60674088, "name": "|", "screen_name": "PattHorton", "lang": "en", "location": "IG: PattHorton", "create_at": date("2009-07-27"), "description": "enjoying the ride", "followers_count": 575, "friends_count": 155, "statues_count": 38046 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Schenectady, NY", "id": "389fab2fead253c3", "name": "Schenectady", "place_type": "city", "bounding_box": rectangle("-73.983266,42.763657 -73.893018,42.842612") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady", "cityID": 3665508, "cityName": "Schenectady" } }
+{ "create_at": datetime("2015-11-27T10:00:18.000Z"), "id": 670301131541639168, "text": "@paisleywood7 is that actually him???", "in_reply_to_status": 670299644094615554, "in_reply_to_user": 3007433299, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3007433299 }}, "user": { "id": 3431436912, "name": "Anna Russell", "screen_name": "CowgirlToughFFA", "lang": "en", "location": "Cambridge, IA", "create_at": date("2015-09-02"), "description": "Ballard FFA freshman that likes country music", "followers_count": 209, "friends_count": 638, "statues_count": 4041 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cambridge, IA", "id": "003cfb007d3c4a68", "name": "Cambridge", "place_type": "city", "bounding_box": rectangle("-93.581066,41.892956 -93.52442,41.906839") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19169, "countyName": "Story", "cityID": 1910225, "cityName": "Cambridge" } }
+{ "create_at": datetime("2015-11-27T10:00:18.000Z"), "id": 670301131696812032, "text": "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": 514544484, "name": "✨Cassie Fischer✨", "screen_name": "Cassie__Fischer", "lang": "en", "location": "null", "create_at": date("2012-03-04"), "description": "there are good people all around, if you cant find one then be one #playfor11", "followers_count": 754, "friends_count": 565, "statues_count": 26818 }, "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-27T10:00:18.000Z"), "id": 670301131747135489, "text": "My @Dreamville Dream Beanie is on the way !!! ������☺️������ https://t.co/WyUv809SKk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 805020986 }}, "user": { "id": 230342336, "name": "MissOctober12th", "screen_name": "blackgirrlmagic", "lang": "en", "location": "CHICAGO", "create_at": date("2010-12-24"), "description": "23 y.o. |Proverbs 31 woman| I don't play about Jermaine. @JColeNC, that is |Your Favorite Flight Attendant ✈️✈️✈️ IG: @catchmeintheairica #coleworld #dreamville", "followers_count": 1140, "friends_count": 1070, "statues_count": 21392 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301131969601536, "text": "@macyb_10 https://t.co/1TQgzEGukV", "in_reply_to_status": -1, "in_reply_to_user": 1663134374, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1663134374 }}, "user": { "id": 1358145954, "name": "Brandon S❌ith", "screen_name": "SMITHsanity2", "lang": "en", "location": " Northgate", "create_at": date("2013-04-16"), "description": "River '16 | Golf & Basketball | God | @macyb_10", "followers_count": 1182, "friends_count": 795, "statues_count": 23750 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Martinsville, WV", "id": "21dfd369b26b14c7", "name": "New Martinsville", "place_type": "city", "bounding_box": rectangle("-80.869291,39.628405 -80.843077,39.694927") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54103, "countyName": "Wetzel", "cityID": 5458684, "cityName": "New Martinsville" } }
+{ "create_at": datetime("2015-11-27T10:00:18.000Z"), "id": 670301132028121088, "text": "At 1:18 you can see @harrygornto �� https://t.co/ioThklBLWg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3231389300 }}, "user": { "id": 3309949338, "name": "TPATT", "screen_name": "Tannerpattion25", "lang": "en", "location": "Fort Walton Beach, FL", "create_at": date("2015-08-08"), "description": "null", "followers_count": 229, "friends_count": 416, "statues_count": 911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ocean City, FL", "id": "6787a2c532e12416", "name": "Ocean City", "place_type": "city", "bounding_box": rectangle("-86.627057,30.43002 -86.590633,30.452727") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1250925, "cityName": "Ocean City" } }
+{ "create_at": datetime("2015-11-27T10:00:18.000Z"), "id": 670301132057673728, "text": "ohbymgod last night at thanksgiving dinner i heard \"trump\" and \"bernie sanders\" and my head snapped up so fast my mom gave me the look", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1835337223, "name": "mistletoe!mol", "screen_name": "holdmedovn", "lang": "en", "location": "(she/they)", "create_at": date("2013-09-08"), "description": "i can eat my cereal and not wonder if he likes bananas over his cheerios too", "followers_count": 2741, "friends_count": 186, "statues_count": 70877 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Apopka, FL", "id": "18ff21ef9765522d", "name": "Apopka", "place_type": "city", "bounding_box": rectangle("-81.579909,28.664067 -81.479167,28.757424") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1201700, "cityName": "Apopka" } }
+{ "create_at": datetime("2015-11-27T10:00:18.000Z"), "id": 670301132120588288, "text": "With j j Abrams lenses flare @ Camp Horn View Point https://t.co/wgvsi56vi5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.18552012,45.58804398"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 11787622, "name": "Jamie Woods", "screen_name": "woodsjam", "lang": "en", "location": "Portland, OR", "create_at": date("2008-01-02"), "description": "PSU Economics Professor, buttinsky, interferer, interloper", "followers_count": 996, "friends_count": 1229, "statues_count": 7292 }, "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.002493") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah" } }
+{ "create_at": datetime("2015-11-27T10:00:18.000Z"), "id": 670301132229480448, "text": "i need 'the last american virgin' soundtrack", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2522816564, "name": "aris", "screen_name": "AMBERRICALDAY", "lang": "en", "location": "Corpus Christi, TX", "create_at": date("2014-05-25"), "description": "rambunctiously soft spoken♻", "followers_count": 172, "friends_count": 85, "statues_count": 398 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301132279910402, "text": "#CCM ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "CCM" }}, "user": { "id": 55479191, "name": "Josh ✈️", "screen_name": "Joshfrank1994", "lang": "en", "location": "Snellville, Ga ➡ Savannah, Ga", "create_at": date("2009-07-09"), "description": "21 | Single Employed x ✌️ #SCState ➡️ #SSU Marketing major #Army IG/SC: @joshfrank1994", "followers_count": 3927, "friends_count": 3708, "statues_count": 116438 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301132284035072, "text": "�� https://t.co/3yjIdihpYt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 404222409, "name": "Joshua", "screen_name": "thee_bown", "lang": "en", "location": "Tumwater, WA", "create_at": date("2011-11-03"), "description": "ask me random questions that get us thinking random thoughts |karma is real| watch what happens when you take a chance on me.", "followers_count": 255, "friends_count": 781, "statues_count": 11184 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301132304945153, "text": "See a virtual tour of our listing on 106 Jamie Ct #Crestview #FL https://t.co/QGbTrEt7Qu #realestate https://t.co/EDAne1wbbW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.559426,30.769866"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Crestview", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crestview, FL", "id": "32bb9ace0723336d", "name": "Crestview", "place_type": "city", "bounding_box": rectangle("-86.635021,30.695757 -86.534517,30.805126") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1215475, "cityName": "Crestview" } }
+{ "create_at": datetime("2015-11-27T10:00:18.000Z"), "id": 670301132657459200, "text": "Happy birthday Love����I love you so much❤️ live it up @kaloodeedee_ https://t.co/67DzDcf2Lx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2712735472 }}, "user": { "id": 769929974, "name": "hanan ahmed", "screen_name": "Hailhanan", "lang": "en", "location": "null", "create_at": date("2012-08-20"), "description": "The smarter you get, the less you speak", "followers_count": 579, "friends_count": 367, "statues_count": 8780 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301132716154880, "text": "Web Interface with Flask and Beaglebone Black https://t.co/7waSvUjufZ #BeagleboneBlack #Python #Flask", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.4867028,35.3395079"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BeagleboneBlack", "Python", "Flask" }}, "user": { "id": 17205941, "name": "Doctor Geek-Nerd", "screen_name": "martymartin12", "lang": "en", "location": "Oklahoma", "create_at": date("2008-11-05"), "description": "Not a real doctor", "followers_count": 129, "friends_count": 177, "statues_count": 2670 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301132804067328, "text": "making plans w victor is impossible", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 268134755, "name": "Jasmine∞Ealy", "screen_name": "jasmineealy3", "lang": "en", "location": "Dallas, TX - Natchitoches, LA", "create_at": date("2011-03-17"), "description": "Northwestern State Cheer. #NSULA19", "followers_count": 1936, "friends_count": 724, "statues_count": 75215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Elm, TX", "id": "bd331d141f66eead", "name": "Little Elm", "place_type": "city", "bounding_box": rectangle("-96.983818,33.140885 -96.881861,33.199847") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4843012, "cityName": "Little Elm" } }
+{ "create_at": datetime("2015-11-27T10:00:18.000Z"), "id": 670301132946698240, "text": "Can you recommend anyone for this #Labor #job? https://t.co/g7gIehznBL #Portsmouth, VA #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.3463624,36.8420949"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Labor", "job", "Portsmouth", "Hiring", "CareerArc" }}, "user": { "id": 22779528, "name": "TMJ-ORF Labor Jobs", "screen_name": "tmj_orf_labor", "lang": "en", "location": "Norfolk area, VA", "create_at": date("2009-03-04"), "description": "Follow this account for geo-targeted General Labor job tweets in Norfolk area, VA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 300, "friends_count": 290, "statues_count": 12 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301132946845697, "text": "Björn's Daily Inspirations - Romantic Red https://t.co/9UJRN0vqx4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0059413,40.7127837"), "retweet_count": 0, "lang": "nl", "is_retweet": false, "user": { "id": 913871414, "name": "BjörnandCo", "screen_name": "BjornandCo", "lang": "en", "location": "New York City", "create_at": date("2012-10-29"), "description": "A Boutique Event & Wedding Planning Firm located in New York City.", "followers_count": 601, "friends_count": 1824, "statues_count": 1032 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301133202710530, "text": "TGf{P} EP091- Black Friday for Eternal Life with Comfort Food https://t.co/Uf8tsCVXYD", "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": 271167530, "name": "The Guys From", "screen_name": "guysfromblank", "lang": "en", "location": "Calgary, Alberta", "create_at": date("2011-03-23"), "description": "Home of sarcastic insight into film, tv, sports, news and gaming. Plus news on The Guys From {BLANK} comedy", "followers_count": 246, "friends_count": 18, "statues_count": 484 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301133395599360, "text": "Encouragement https://t.co/vDYNCRYbsS #enlightenment #encouragement #marfa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.020623,30.3094622"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "enlightenment", "encouragement", "marfa" }}, "user": { "id": 312397191, "name": "Amanda K Holstien Sr", "screen_name": "THERAPISTJOKES", "lang": "en", "location": "Marfa, TX", "create_at": date("2011-06-06"), "description": "Texan, comedienne, artiste", "followers_count": 192, "friends_count": 107, "statues_count": 3615 }, "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": 48377, "countyName": "Presidio", "cityID": 4846620, "cityName": "Marfa" } }
+{ "create_at": datetime("2015-11-27T10:00:18.000Z"), "id": 670301133689200640, "text": "Oooooin coisa maisi linda https://t.co/3qntYYqDKc", "in_reply_to_status": 670292531272396800, "in_reply_to_user": 3047069923, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 3047069923, "name": "júlia", "screen_name": "juxzzi", "lang": "pt", "location": "nova iorque", "create_at": date("2015-02-27"), "description": "aliens, pitbulls e couve", "followers_count": 97, "friends_count": 206, "statues_count": 1062 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301133718593537, "text": "The Russell On Air Show: Survival Of The Fittest https://t.co/nfsDJifD8q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.1193763,41.7353591"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 302272155, "name": "Russell Tanzillo", "screen_name": "RussellOnAir", "lang": "en", "location": "Chicagoland, IL", "create_at": date("2011-05-20"), "description": "Hey I’m Russell I grew up in the suburbs of Chicago listening to any music I could get my hands/ears on. Now I am on the radio @101WKQX!", "followers_count": 284, "friends_count": 573, "statues_count": 1213 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301133894586368, "text": "Rain this afternoon, high 48 (9 C). Low 41 (5 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575409, "name": "Fort Smith Weather", "screen_name": "FortSmithWX", "lang": "en", "location": "Fort Smith, AR", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Fort Smith, AR", "followers_count": 440, "friends_count": 92, "statues_count": 7904 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301133991100416, "text": "This #Nursing #job might be a great fit for you: Resident Associate - https://t.co/IycnKgGXJ8 #STNA #training #Beachwood, OH #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4899224,41.5011199"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "STNA", "training", "Beachwood", "Hiring" }}, "user": { "id": 214122801, "name": "MenorahParkJobs", "screen_name": "MenorahParkJobs", "lang": "en", "location": "Beachwood, Ohio", "create_at": date("2010-11-10"), "description": "Menorah Park Center for Senior Living is one of the largest senior living communities in the state of Ohio.", "followers_count": 128, "friends_count": 82, "statues_count": 1378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beachwood, OH", "id": "fa5bd676191716ee", "name": "Beachwood", "place_type": "city", "bounding_box": rectangle("-81.532635,41.449669 -81.48037,41.501448") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3904500, "cityName": "Beachwood" } }
+{ "create_at": datetime("2015-11-27T10:00:18.000Z"), "id": 670301134045614080, "text": "To say we're inlove is daaaangerous but girl im sooo glad we're aquainted", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 498359229, "name": "The REAL Daddy", "screen_name": "Hiller_Jake", "lang": "en", "location": "Grenloch NJ", "create_at": date("2012-02-20"), "description": "Snapchat- jakehiller http://ask.fm/jakehiller instagram- jakehiller_aka_daddy", "followers_count": 572, "friends_count": 499, "statues_count": 20496 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301134230171648, "text": "I knew there was a reason I liked her https://t.co/zuaYncGo0j", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1260709878, "name": "Hana.", "screen_name": "hanaattaran_", "lang": "en", "location": "null", "create_at": date("2013-03-11"), "description": "null", "followers_count": 892, "friends_count": 301, "statues_count": 50891 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301134230192128, "text": "Going on expensive dates is what you do on your birthday/ anniversary.", "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": "Lala", "screen_name": "Nigerian__InMe", "lang": "en", "location": "Akwa Ibom, TX", "create_at": date("2015-06-10"), "description": "RN to be| Dallas based hairstylist✂️ #AfroLuxeHair| Insta/SC: ayoshenigerian Dallas/PV/Nac", "followers_count": 1047, "friends_count": 541, "statues_count": 23412 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301134368677888, "text": "@LiamNigro thanks Liam ������", "in_reply_to_status": 670283303459676160, "in_reply_to_user": 1011945756, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1011945756 }}, "user": { "id": 1452517638, "name": "Nick Chillbae", "screen_name": "N_Kilbane30", "lang": "en", "location": "CLE | 420 11 |", "create_at": date("2013-05-23"), "description": "Purveyor of Island Lifestyles... call me goat Ignatius Football '17 Mack Pack #32", "followers_count": 446, "friends_count": 325, "statues_count": 2017 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301134544748545, "text": "�������� https://t.co/NzY9FoNLEM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3321057210, "name": "♛B", "screen_name": "BLUXO_", "lang": "en", "location": "null", "create_at": date("2015-08-20"), "description": "Self-absorbed✨", "followers_count": 1766, "friends_count": 38, "statues_count": 1881 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-27T10:00:18.000Z"), "id": 670301134637019136, "text": "now time to eat left overs for breakfast ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2959282003, "name": "Naomi", "screen_name": "zerreitugn", "lang": "en", "location": "Lakewood, CO", "create_at": date("2015-01-04"), "description": "@_V_IX❣", "followers_count": 821, "friends_count": 686, "statues_count": 22627 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301134729273344, "text": "I'm about to pig tf out on all my family's food since ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 466964021, "name": "Collin Brimberry", "screen_name": "BallinCrimberry", "lang": "en", "location": "College Station, TX", "create_at": date("2012-01-17"), "description": "null", "followers_count": 624, "friends_count": 394, "statues_count": 21832 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301134884462592, "text": "I know I don't need weave but weave is life >>>", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 312292052, "name": "♑️", "screen_name": "_jaassmine", "lang": "en", "location": "null", "create_at": date("2011-06-06"), "description": "9.7.14❤️ Hakuna Matata", "followers_count": 997, "friends_count": 753, "statues_count": 8453 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301135085793280, "text": "Can it be Christmas?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1720698690, "name": "Ariel Monique ❤️", "screen_name": "arielreyes___", "lang": "en", "location": "Rancho Cordova, CA", "create_at": date("2013-09-01"), "description": "Livin life like I should be.\n\nSnapChat: Ariielr", "followers_count": 600, "friends_count": 621, "statues_count": 12414 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301135152873472, "text": "She likes what she likes @ Cherche Midi Bowery https://t.co/BeQxBmQ9rf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9928436,40.7239189"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14816125, "name": "Grace", "screen_name": "graceclarke", "lang": "en", "location": "New York", "create_at": date("2008-05-17"), "description": "team @madewell1937", "followers_count": 1064, "friends_count": 457, "statues_count": 12823 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301135308201984, "text": "@ajaaaaaaayy I had the weirdest dream about you last night ����", "in_reply_to_status": -1, "in_reply_to_user": 3191252205, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3191252205 }}, "user": { "id": 351910162, "name": "melanie tuttle", "screen_name": "_melaaaanie", "lang": "en", "location": "null", "create_at": date("2011-08-09"), "description": "null", "followers_count": 1886, "friends_count": 1092, "statues_count": 17345 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301135559917569, "text": "Honeymoon is the best album that has come out all year", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 306548105, "name": "Nazz", "screen_name": "kaylaaa5", "lang": "en", "location": "Westchester, NY", "create_at": date("2011-05-27"), "description": "Favorite things: Lana Del Rey and cream of chicken & wild rice soup from Panera. NYG, NYK, NYR, NYM", "followers_count": 496, "friends_count": 887, "statues_count": 31841 }, "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("2015-11-27T10:00:18.000Z"), "id": 670301135664750592, "text": "Kaleb's turn (@ My Barber's Lounge in Greenbelt, MD w/ @thad2009 @rob_cook87 @daurbannerd) https://t.co/w7diyaPfPa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.85289856,38.98994125"), "retweet_count": 0, "lang": "de", "is_retweet": false, "user_mentions": {{ 40724413, 1331924906, 17931742 }}, "user": { "id": 43119995, "name": "Reggie T!! Bitches!", "screen_name": "EnigmaRDT", "lang": "en", "location": "Maryland, USA #DMV", "create_at": date("2009-05-28"), "description": "Interior Decor! Freaky! All around cool guy! Hated on all the time! Known to be a friend forever! #TeamKaleb #TeamBraxton #TeamSexy #TeamUncle", "followers_count": 5482, "friends_count": 5579, "statues_count": 122004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Goddard, MD", "id": "a17cffc32f88a01f", "name": "Goddard", "place_type": "city", "bounding_box": rectangle("-76.866905,38.977774 -76.833858,39.019795") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2470775, "cityName": "Seabrook" } }
+{ "create_at": datetime("2015-11-27T10:00:19.000Z"), "id": 670301135945641984, "text": "We are looking for a buyer on 247 Holland St #Crestview #FL https://t.co/l20PcEZKJo #realestate https://t.co/9UgLEcUczU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.582068,30.768301"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Crestview", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crestview, FL", "id": "32bb9ace0723336d", "name": "Crestview", "place_type": "city", "bounding_box": rectangle("-86.635021,30.695757 -86.534517,30.805126") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1215475, "cityName": "Crestview" } }
+{ "create_at": datetime("2015-11-27T10:00:19.000Z"), "id": 670301136411316226, "text": "Happy b-day bro! Hope your having a good one! @JT_Shalvey6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2322905994 }}, "user": { "id": 2888386953, "name": "Noah Hefke", "screen_name": "noahhefke86", "lang": "en", "location": "Ohio, USA", "create_at": date("2014-11-22"), "description": "null", "followers_count": 491, "friends_count": 314, "statues_count": 2954 }, "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("2015-11-27T10:00:19.000Z"), "id": 670301136449118208, "text": "@Kolo_Martin @tulip_jenkins @chalkybear13 @ltluchristian @ToddyFur@sherfordbear @pip_mouse @3DogsWhite @Dutchie_Duffy Very well behaved boys", "in_reply_to_status": 670300365913460736, "in_reply_to_user": 39573397, "favorite_count": 0, "coordinate": point("-74.88570122,40.12365413"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 39573397, 3119505485, 831097237, 629461055, 3002479836, 537409474, 2648532859 }}, "user": { "id": 2569856559, "name": "Obe Wan Kenobe", "screen_name": "ajstenaka", "lang": "en", "location": "U.S. Pennsylvania", "create_at": date("2014-05-28"), "description": "My name is Obe I am a Captain in #TheAviators MARRIED to @wellytopping my kids are Little Nikki, Tom, John & Teddy ! My Bff's are Binky & Didi! #keepfits, #BBOT", "followers_count": 3674, "friends_count": 3622, "statues_count": 194075 }, "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("2015-11-27T10:00:19.000Z"), "id": 670301136910336000, "text": "@its_miliaaa bih I'm just talking ��", "in_reply_to_status": 670300930584121344, "in_reply_to_user": 825659377, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 825659377 }}, "user": { "id": 525859131, "name": "tionnia ❤", "screen_name": "Tionniaa", "lang": "en", "location": "null", "create_at": date("2012-03-15"), "description": "null", "followers_count": 1633, "friends_count": 683, "statues_count": 59042 }, "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("2015-11-27T10:00:19.000Z"), "id": 670301137082458113, "text": "Shoulda been a clean sweep but taking 2 out 3 from @JrJay23 and the family is okay 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": {{ 456016538 }}, "user": { "id": 320215107, "name": "Chet Ubetcha ♛", "screen_name": "CJ13Spiller", "lang": "en", "location": "Pallet Town", "create_at": date("2011-06-19"), "description": "TC15. los dos. | 307 Carrer d'Aragon | XIII |Everyday I'm Hustlin | 01776 | 06106 | HMS |", "followers_count": 333, "friends_count": 263, "statues_count": 7098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milford, MA", "id": "002564726a89af1d", "name": "Milford", "place_type": "city", "bounding_box": rectangle("-71.57581,42.116846 -71.47777,42.191514") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25027, "countyName": "Worcester", "cityID": 2541200, "cityName": "Milford" } }
+{ "create_at": datetime("2015-11-27T10:00:19.000Z"), "id": 670301137115869184, "text": "Rain this afternoon, high 69 (21 C). Low 57 (14 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20575444, "name": "Little Rock Weather", "screen_name": "LittleRockWX", "lang": "en", "location": "Little Rock, AR", "create_at": date("2009-02-10"), "description": "Forecasts and advisories for Little Rock, AR", "followers_count": 1834, "friends_count": 92, "statues_count": 7778 }, "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("2015-11-27T10:00:19.000Z"), "id": 670301137312940034, "text": "@Jimboslice7197 shut up you ass ��", "in_reply_to_status": 670299790446493696, "in_reply_to_user": 617237992, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 617237992 }}, "user": { "id": 43332022, "name": "shannon", "screen_name": "happpy_shan", "lang": "en", "location": "la la land ", "create_at": date("2009-05-29"), "description": "ig: nevershoutshannon", "followers_count": 2427, "friends_count": 628, "statues_count": 33991 }, "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("2015-11-27T10:00:19.000Z"), "id": 670301137975685120, "text": "I'm still looking yaaaaaay woofwoof", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3237736601, "name": "nova wolf", "screen_name": "arielrae618", "lang": "en", "location": "Yakima, WA", "create_at": date("2015-05-05"), "description": "Hi my name is nova: ) I loves making new friends shy but friendly and curious i loves to play and i likes cookies num num Lol and I'm single", "followers_count": 683, "friends_count": 1171, "statues_count": 12223 }, "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("2015-11-27T10:00:19.000Z"), "id": 670301137996783616, "text": "Just getting up ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3044218774, "name": "January 1️⃣7️⃣th", "screen_name": "xKaySimone_", "lang": "en", "location": "Tarboro, NC", "create_at": date("2015-02-17"), "description": "#LongLiveJamiya", "followers_count": 623, "friends_count": 834, "statues_count": 5045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Princeville, NC", "id": "01db12161b0a6332", "name": "Princeville", "place_type": "city", "bounding_box": rectangle("-77.534452,35.865631 -77.498625,35.893649") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37065, "countyName": "Edgecombe", "cityID": 3753840, "cityName": "Princeville" } }
+{ "create_at": datetime("2015-11-27T10:00:19.000Z"), "id": 670301138172977152, "text": "@HLadnier so funny", "in_reply_to_status": 670290813319053313, "in_reply_to_user": 777098138, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 777098138 }}, "user": { "id": 371341727, "name": "Sal", "screen_name": "young_salvatore", "lang": "en", "location": "The Kitchen", "create_at": date("2011-09-10"), "description": "JCU Wrestling '17 | SPE | EBG", "followers_count": 479, "friends_count": 461, "statues_count": 23752 }, "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("2015-11-27T10:00:19.000Z"), "id": 670301138516770816, "text": "@BeltwayPanda 2 proven War Criminals are GW Bush&Dick Cheney.[Irak War]", "in_reply_to_status": 670300081229307909, "in_reply_to_user": 355028519, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 355028519 }}, "user": { "id": 85881268, "name": "alejandro santos", "screen_name": "wonderfullone", "lang": "en", "location": "florida usa", "create_at": date("2009-10-28"), "description": "Ret. Sgt. USMC. Viet Nam Vet. Awarded Purple Heart. Ret. Correctional Sergeant, Dept. Of Corrections California.", "followers_count": 4807, "friends_count": 5281, "statues_count": 124279 }, "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-27T10:00:19.000Z"), "id": 670301138575495169, "text": "Denver Area Jobs #Healthcare #Job: Personal Care Aides (#Denver, CO) https://t.co/dHwmnjixdi #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.0002242,39.7541032"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "Job", "Denver", "Jobs", "Hiring" }}, "user": { "id": 21681745, "name": "Denver Health Jobs", "screen_name": "tmj_den_health", "lang": "en", "location": "Denver, CO", "create_at": date("2009-02-23"), "description": "Follow this account for geo-targeted Healthcare job tweets in Denver, CO. Need help? Tweet us at @CareerArc!", "followers_count": 616, "friends_count": 305, "statues_count": 892 }, "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-27T10:00:19.000Z"), "id": 670301138655121408, "text": "We're #hiring! Click to apply: PCT - https://t.co/APJhGj6Br3 #Healthcare #dialysis #StoneMountain, GA #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.170196,33.8081608"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Healthcare", "dialysis", "StoneMountain", "Job", "Jobs", "CareerArc" }}, "user": { "id": 21299091, "name": "TMJ- ATL Health Jobs", "screen_name": "tmj_atl_health", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-02-19"), "description": "Follow this account for geo-targeted Healthcare job tweets in Atlanta, GA. Need help? Tweet us at @CareerArc!", "followers_count": 517, "friends_count": 339, "statues_count": 663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stone Mountain, GA", "id": "45c857f393d8251e", "name": "Stone Mountain", "place_type": "city", "bounding_box": rectangle("-84.187013,33.78857 -84.150893,33.820632") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1373816, "cityName": "Stone Mountain" } }
+{ "create_at": datetime("2015-11-27T10:00:19.000Z"), "id": 670301138697195520, "text": "Boom. @ La Jolla Cove Seal Beach https://t.co/bJz4HuV7fF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.27469329,32.84901578"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21853, "name": "Chris J. Davis", "screen_name": "chrisjdavis", "lang": "en", "location": "San Diego, CA", "create_at": date("2006-11-25"), "description": "Wookie, Unicorn, Lover. Director of User Experience @Voxox & Founder at @habari. A proud Unicorn before it was cool, and had a name. Makes music occasionally.", "followers_count": 1011, "friends_count": 710, "statues_count": 15762 }, "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-27T10:00:19.000Z"), "id": 670301138701414400, "text": "Just posted a photo @ Yeira's Natural Style https://t.co/vBkr11Okuk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-66.1265356,18.4383629"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1566655837, "name": "YEIRAS NATURAL STYLE", "screen_name": "yeirarivera1", "lang": "es", "location": "null", "create_at": date("2013-07-03"), "description": "null", "followers_count": 51, "friends_count": 68, "statues_count": 553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cataño, Puerto Rico", "id": "4b9638c2a43588f1", "name": "Cataño", "place_type": "city", "bounding_box": rectangle("-66.170302,18.420804 -66.110133,18.462554") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72061, "countyName": "Guaynabo", "cityID": 7232522, "cityName": "Guaynabo" } }
+{ "create_at": datetime("2015-11-27T10:00:19.000Z"), "id": 670301138864898048, "text": "Happy Black Friday shopping! #blackfriday #EngagementRings #cachevalley https://t.co/sa5kqdQPDm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "blackfriday", "EngagementRings", "cachevalley" }}, "user": { "id": 4195624878, "name": "LoganRingGallery", "screen_name": "LoganRings", "lang": "en", "location": "Logan, UT", "create_at": date("2015-11-10"), "description": "null", "followers_count": 0, "friends_count": 0, "statues_count": 4 }, "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("2015-11-27T10:00:19.000Z"), "id": 670301139519205376, "text": "Pull up banger 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": 1450284379, "name": "Clo⛎t L♉️rd MAX", "screen_name": "JuiceCrew_E", "lang": "en", "location": "Newark, NJ", "create_at": date("2013-05-22"), "description": "#OTV †Preparation is the best motivation..✟ ITL... #Juicecrew #Clippersnation", "followers_count": 1327, "friends_count": 1230, "statues_count": 19377 }, "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("2015-11-27T10:00:19.000Z"), "id": 670301139653390337, "text": "Ok this is so cute I actually want it https://t.co/wBjJIFOYjT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3897247094, "name": "nayeli", "screen_name": "rowftsabi", "lang": "en", "location": "bay area", "create_at": date("2015-10-14"), "description": "don't kermit suicide", "followers_count": 147, "friends_count": 149, "statues_count": 2066 }, "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("2015-11-27T10:00:20.000Z"), "id": 670301139913433089, "text": "@TracFoneCalls y'all about to have a new customer cause.. https://t.co/yH3grRO7TN", "in_reply_to_status": -1, "in_reply_to_user": 64719796, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 64719796 }}, "user": { "id": 44759961, "name": "artavion ✨", "screen_name": "socialenemy_", "lang": "en", "location": "#BlackLivesMatter", "create_at": date("2009-06-04"), "description": "Class of 2016 ⚡️| Louisiana♌️| Melanated | 17", "followers_count": 3474, "friends_count": 938, "statues_count": 109718 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coushatta, LA", "id": "ec8a3475eef248b3", "name": "Coushatta", "place_type": "city", "bounding_box": rectangle("-93.355292,32.003847 -93.322713,32.049889") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22081, "countyName": "Red River", "cityID": 2218055, "cityName": "Coushatta" } }
+{ "create_at": datetime("2015-11-27T10:00:20.000Z"), "id": 670301139942838272, "text": "Wintry mix this afternoon, high 29 (-2 C). Low 25 (-4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26195145, "name": "Wichita Weather", "screen_name": "WichitaWeather", "lang": "en", "location": "Wichita, KS", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Wichita, KS", "followers_count": 1852, "friends_count": 88, "statues_count": 7633 }, "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("2015-11-27T10:00:20.000Z"), "id": 670301140194607104, "text": "I'm at @DollarTree in Raleigh, NC https://t.co/Qg9KEN2AuX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.623506,35.87317"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 94558801 }}, "user": { "id": 20487375, "name": "Karl", "screen_name": "wahoo103", "lang": "en", "location": "Raleigh, NC", "create_at": date("2009-02-09"), "description": "Hospitality Consultant Marlin Events Catering Consulting ritzcracker103@gmail.com", "followers_count": 1150, "friends_count": 2004, "statues_count": 16601 }, "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("2015-11-27T10:00:20.000Z"), "id": 670301140265730048, "text": "@MBoffin you got a new Surface Book? Impressions?", "in_reply_to_status": 669577211653939200, "in_reply_to_user": 3651621, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3651621 }}, "user": { "id": 36904502, "name": "Tad Reeves", "screen_name": "TurboDad", "lang": "en", "location": "Lake Oswego, OR", "create_at": date("2009-05-01"), "description": "Adobe CQ / Cloud Platform Engineer DevOps Individual, Husband/Daddy-O", "followers_count": 361, "friends_count": 327, "statues_count": 3156 }, "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("2015-11-27T10:00:20.000Z"), "id": 670301140312006660, "text": "Wind 3.0 mph S. Barometer 30.336 in, Falling. Temperature 56.0 °F. Rain today 0.01 in. Humidity 86%", "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": 23370 }, "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("2015-11-27T10:00:20.000Z"), "id": 670301140416864256, "text": "temperature up 32°F -> 39°F\nhumidity down 79% -> 62%\nwind 7mph -> 6mph", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.29075,37.95367"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 120332988, "name": "Stockton Weather", "screen_name": "_StocktonCA", "lang": "en", "location": "Stockton, CA", "create_at": date("2010-03-05"), "description": "Weather updates, forecast, warnings and information for Stockton, CA. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 157, "friends_count": 3, "statues_count": 25504 }, "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("2015-11-27T10:00:20.000Z"), "id": 670301140450304001, "text": "We are looking for a buyer on 740 E Chestnut Ave #Crestview #FL https://t.co/AWxEucOhU8 #realestate https://t.co/cy7TGkgRuR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.561095,30.75771"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Crestview", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crestview, FL", "id": "32bb9ace0723336d", "name": "Crestview", "place_type": "city", "bounding_box": rectangle("-86.635021,30.695757 -86.534517,30.805126") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1215475, "cityName": "Crestview" } }
+{ "create_at": datetime("2015-11-27T10:00:20.000Z"), "id": 670301140462911489, "text": "I'm the coolest serious/childish nigga you will ever meet,I just don't take shit from nobody��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3304268792, "name": "mula_boyひ", "screen_name": "zirslum", "lang": "en", "location": "Dickinson, TX.", "create_at": date("2015-08-02"), "description": "we all get two assets,24 hours in a day and the power to make choices(no excuse just produce) born to lose,BUILT TO WIN", "followers_count": 283, "friends_count": 203, "statues_count": 3092 }, "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("2015-11-27T10:00:20.000Z"), "id": 670301141108813824, "text": "@shawacademy #shawnutritionchallenge CHO: 420-720g Lunch=118g\n�� over 300g to go! https://t.co/WfYK2xlT1m", "in_reply_to_status": -1, "in_reply_to_user": 2228394355, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "shawnutritionchallenge" }}, "user_mentions": {{ 2228394355 }}, "user": { "id": 184965468, "name": "Kilo Kitty", "screen_name": "Kilo_Kitty", "lang": "en", "location": "Greenville SC", "create_at": date("2010-08-30"), "description": "I love pole!\r\nI am a mother of four legged children and a wife", "followers_count": 287, "friends_count": 214, "statues_count": 894 }, "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("2015-11-27T10:00:20.000Z"), "id": 670301141222092800, "text": "We're #hiring! Read about our latest #job opening here: Sales Representative - https://t.co/j69urooXE3 #Worcester, MA #Sales #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.8022934,42.2625932"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Worcester", "Sales", "CareerArc" }}, "user": { "id": 22038280, "name": "TMJ-MA-US Sales Jobs", "screen_name": "tmj_MA_sales", "lang": "en", "location": "Massachusetts", "create_at": date("2009-02-26"), "description": "Follow this account for geo-targeted Sales job tweets in Massachusetts Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 398, "friends_count": 308, "statues_count": 438 }, "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("2015-11-27T10:00:20.000Z"), "id": 670301141553381376, "text": "I want to go back to CSTAT and get the hell out of this town..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 330104736, "name": "J E S S✨", "screen_name": "Jesserrkaa", "lang": "en", "location": "ETX | CSTAT", "create_at": date("2011-07-05"), "description": "i will praise you in this storm | ΚΦ | #tamu19", "followers_count": 481, "friends_count": 340, "statues_count": 11830 }, "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.658059,33.091542 -95.556096,33.158547") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48223, "countyName": "Hopkins", "cityID": 4870904, "cityName": "Sulphur Springs" } }
+{ "create_at": datetime("2015-11-27T10:00:20.000Z"), "id": 670301141608083457, "text": "@primetime_tony yes they did they were a bunch of Plebs.", "in_reply_to_status": 670297807538946048, "in_reply_to_user": 574251558, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 574251558 }}, "user": { "id": 2738030983, "name": "Fíllipo Pío Apollo", "screen_name": "IlPrincipeXIV", "lang": "en", "location": "Chicago, Illinois", "create_at": date("2014-08-16"), "description": "Official Twitter of Boxer Phillip Pío Apollo. Welterweight (147lbs). Next Fight: January TBA. LewisU Flyers #IlPrincipe", "followers_count": 163, "friends_count": 154, "statues_count": 8988 }, "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("2015-11-27T10:00:20.000Z"), "id": 670301141733785600, "text": "#Healthcare #Job alert: RN / Registered Nurse / OR / Virginia | Supplemental Health Care | #FallsChurch, Virginia https://t.co/GYI7IjXtET", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.1710914,38.882334"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "Job", "FallsChurch" }}, "user": { "id": 20828529, "name": "TMJ- WAS Health Jobs", "screen_name": "tmj_dc_health", "lang": "en", "location": "Washington, DC", "create_at": date("2009-02-13"), "description": "Follow this account for geo-targeted Healthcare job tweets in DC Metro. Need help? Tweet us at @CareerArc!", "followers_count": 418, "friends_count": 307, "statues_count": 843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Falls Church, VA", "id": "077e6b03695c28ee", "name": "Falls Church", "place_type": "city", "bounding_box": rectangle("-77.19419,38.872266 -77.149803,38.896296") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51610, "countyName": "Falls Church", "cityID": 5127200, "cityName": "Falls Church" } }
+{ "create_at": datetime("2015-11-27T10:00:20.000Z"), "id": 670301141897465856, "text": "Safety Lapses and Deaths Amid a Building Boom in New York https://t.co/AwQYHAKYeF #immigration #undocumented #OSHA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "immigration", "undocumented", "OSHA" }}, "user": { "id": 62326766, "name": "Guillermo Yrizar B.", "screen_name": "gyrizar", "lang": "en", "location": "New York City", "create_at": date("2009-08-02"), "description": "PhD candidate #Sociology @GC_CUNY | research on international migration | futbolero y taquero", "followers_count": 428, "friends_count": 417, "statues_count": 2326 }, "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("2015-11-27T10:00:20.000Z"), "id": 670301142098837504, "text": "@WorthyAffiliate thank you☺️", "in_reply_to_status": 670298672886861825, "in_reply_to_user": 2363476790, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2363476790 }}, "user": { "id": 77111443, "name": "Birthday Girl♐️", "screen_name": "jaixo__", "lang": "en", "location": "610/215", "create_at": date("2009-09-24"), "description": "got you stuck off the realness❄️ SC: Viva_Jai", "followers_count": 2219, "friends_count": 1158, "statues_count": 73983 }, "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-27T10:00:20.000Z"), "id": 670301142220324864, "text": "My problems are dumb but I care about them alot that's why they're problems lolol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1905982525, "name": "Cassie Espinosa", "screen_name": "cassieissassyy", "lang": "en", "location": "Napa Valley ", "create_at": date("2013-09-25"), "description": "I flipped over my chip chop", "followers_count": 672, "friends_count": 515, "statues_count": 8084 }, "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("2015-11-27T10:00:20.000Z"), "id": 670301142560051201, "text": "anyways, going to Vegas with my parents and I'm not 100% sure why", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 502351507, "name": "️kayla chamberlin", "screen_name": "kaylaorwhatever", "lang": "en", "location": "null", "create_at": date("2012-02-24"), "description": "thinking of you", "followers_count": 698, "friends_count": 275, "statues_count": 38787 }, "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("2015-11-27T10:00:20.000Z"), "id": 670301143130488832, "text": "@andrewcrisp @albear that doesn't include the \"honorable mentions\" like the murder-suicide, or the couple that murdered a store employee", "in_reply_to_status": 670300827240787972, "in_reply_to_user": 12085742, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 12085742, 16819772 }}, "user": { "id": 18882459, "name": "Jillian", "screen_name": "JillianWould", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-01-11"), "description": "Inappropriate with a high chance of offending.", "followers_count": 950, "friends_count": 360, "statues_count": 11183 }, "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-27T10:00:20.000Z"), "id": 670301143134695424, "text": "Pretty sure I gained all the weight I lost back yesterday lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229246224, "name": "AG", "screen_name": "_diimesz", "lang": "en", "location": "van nuys,ca", "create_at": date("2010-12-21"), "description": "LA Valley College ❥ 19.\nSC: dimesz.818", "followers_count": 12192, "friends_count": 868, "statues_count": 1009 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-27T10:00:20.000Z"), "id": 670301143218724864, "text": "I got to restore my faith", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2203928748, "name": "Crystal Martinez", "screen_name": "oheycrys", "lang": "en", "location": "Cartersville, GA ", "create_at": date("2013-11-19"), "description": "positive vibes", "followers_count": 552, "friends_count": 311, "statues_count": 7398 }, "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("2015-11-27T10:00:20.000Z"), "id": 670301143331954689, "text": "@k8rukujzo aw i love you kate❤️❤️❤️", "in_reply_to_status": 670129727361843201, "in_reply_to_user": 3230756252, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3230756252 }}, "user": { "id": 924249014, "name": "big daddy", "screen_name": "ssaaaamm", "lang": "en", "location": "the cloud", "create_at": date("2012-11-03"), "description": "greek af // @_apricity @nawdss #PMW | OVOXO", "followers_count": 788, "friends_count": 445, "statues_count": 20157 }, "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("2015-11-27T10:00:20.000Z"), "id": 670301143356981248, "text": "Cloudy this afternoon, high 29 (-2 C). Low 23 (-5 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26199170, "name": "Omaha Weather", "screen_name": "OmahaWX", "lang": "en", "location": "Omaha, Nebraska", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Omaha, Nebraska", "followers_count": 2575, "friends_count": 88, "statues_count": 7626 }, "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("2015-11-27T10:00:20.000Z"), "id": 670301143369539585, "text": "Oracle #IT #Job: Applications Developer 3 (#ColoradoSprings, CO) https://t.co/s746eIyKJL #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.8213634,38.8338816"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "IT", "Job", "ColoradoSprings", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 112955427, "name": "TMJ-CO IT Jobs", "screen_name": "tmj_CO_it", "lang": "en", "location": "Colorado", "create_at": date("2010-02-09"), "description": "Follow this account for geo-targeted Software Dev. - General/IT job tweets in Colorado Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 387, "friends_count": 321, "statues_count": 260 }, "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("2015-11-27T10:00:20.000Z"), "id": 670301143394832385, "text": "@markiplier did you have a girl before? and put this comment in a video or i will kill you", "in_reply_to_status": 667477495315169280, "in_reply_to_user": 517077573, "favorite_count": 0, "coordinate": point("-86.7085445,36.1294118"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 517077573 }}, "user": { "id": 3908188758, "name": "Pigzilla Gaming", "screen_name": "PigzillaG", "lang": "en", "location": "null", "create_at": date("2015-10-15"), "description": "pigzillaG", "followers_count": 7, "friends_count": 11, "statues_count": 16 }, "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.99451 -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("2015-11-27T10:00:20.000Z"), "id": 670301143529037824, "text": "Hell no. ���� https://t.co/DnudL50xcN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 904007395, "name": "Raven", "screen_name": "RavenNotSymone", "lang": "en", "location": "colυмвιa, ѕc", "create_at": date("2012-10-25"), "description": "protect your peace, get rid of toxicity, cleanse your space, cultivate love ✨@hoesandhennessy❤️", "followers_count": 701, "friends_count": 454, "statues_count": 18593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aliquippa, PA", "id": "671e49bd6a612709", "name": "Aliquippa", "place_type": "city", "bounding_box": rectangle("-80.319793,40.562008 -80.235012,40.653209") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42007, "countyName": "Beaver", "cityID": 4200820, "cityName": "Aliquippa" } }
+{ "create_at": datetime("2015-11-27T10:00:20.000Z"), "id": 670301143797522433, "text": "@Mr_IamDebonair absolutely", "in_reply_to_status": 670292836038897664, "in_reply_to_user": 37013606, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 37013606 }}, "user": { "id": 2448906037, "name": "Melanin Monroe♍️❤️", "screen_name": "xxHeadFullaHair", "lang": "en", "location": "From the Fifth Floor ", "create_at": date("2014-04-16"), "description": "Taraji's Younger Sister #FVSU Alumni Grad student at #CAU You'll be calling me Doctor soon..", "followers_count": 620, "friends_count": 361, "statues_count": 47129 }, "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("2015-11-27T10:00:20.000Z"), "id": 670301143830958080, "text": "#StudioLife #Florida\n@thewinlabinc\n@iammgeezy\n#MonstaKodi \n@monstakodi \n#DIAMONDINTHERUFF… https://t.co/RAUjaO6pVx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.1857376,25.79006"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "StudioLife", "Florida", "MonstaKodi", "DIAMONDINTHERUFF" }}, "user_mentions": {{ 396350823, 27354245, 2846130717 }}, "user": { "id": 17738954, "name": "MANNY MADDOG", "screen_name": "MANNYMADDOG", "lang": "en", "location": "MIAMI / NYC / CHICAGO / ATL", "create_at": date("2008-11-29"), "description": "MARKETING & PROMO GURU / BRAND AWARENESS.. 28 yrs in the Entertainment Industry & Still going STRONG", "followers_count": 5613, "friends_count": 5650, "statues_count": 42085 }, "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("2015-11-27T10:00:20.000Z"), "id": 670301144065818624, "text": "Lol me https://t.co/GQWatUbdNv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3224658048, "name": "⚠️ brit ⚠️", "screen_name": "BritUghKnee", "lang": "en", "location": "Wayne, MI", "create_at": date("2015-05-23"), "description": "vodka the only bitch to not let me down • single • computer science major • Madonna Univeristy '19 • @xoxosierra524", "followers_count": 512, "friends_count": 571, "statues_count": 2611 }, "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("2015-11-27T10:00:21.000Z"), "id": 670301144548175872, "text": "These guys are getting $500 in free Best Buy money just for upgrading to new Samsung devices, Go Sprint!!! https://t.co/kgBLE0AT9L", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1403726502, "name": "Paige Martin", "screen_name": "csucess_paige", "lang": "en", "location": "null", "create_at": date("2013-05-04"), "description": "null", "followers_count": 87, "friends_count": 292, "statues_count": 193 }, "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("2015-11-27T10:00:21.000Z"), "id": 670301144745275392, "text": "Happy birthday @Doyoudream__ ! See you tonight at dreamstate!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3322790191 }}, "user": { "id": 33248446, "name": "Robby", "screen_name": "AnjunaRobbert", "lang": "en", "location": "Long Beach, CA", "create_at": date("2009-04-19"), "description": "Autonation Infiniti", "followers_count": 752, "friends_count": 430, "statues_count": 85290 }, "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("2015-11-27T10:00:21.000Z"), "id": 670301144766377984, "text": "@Flock24 for some reason everyone thinks so though", "in_reply_to_status": 670298233688715264, "in_reply_to_user": 314948630, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 314948630 }}, "user": { "id": 1148154080, "name": "Iff", "screen_name": "Darkcoco14", "lang": "en", "location": "null", "create_at": date("2013-02-04"), "description": "Only man above me is God himself.", "followers_count": 812, "friends_count": 413, "statues_count": 27016 }, "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("2015-11-27T10:00:21.000Z"), "id": 670301144996954113, "text": "@AThompson45 yes please��", "in_reply_to_status": 670300846022725632, "in_reply_to_user": 450914210, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 450914210 }}, "user": { "id": 2922140642, "name": "☀Sunshine☀", "screen_name": "rissa392", "lang": "en", "location": "null", "create_at": date("2014-12-07"), "description": "When your legs can't run anymore, run with your heart❤️ •XC•", "followers_count": 231, "friends_count": 295, "statues_count": 1537 }, "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("2015-11-27T10:00:21.000Z"), "id": 670301145185837056, "text": "\"Too racist\" That really is all I can hope for. I stopped hoping about the weight thing years ago. https://t.co/qoXTVJCYp7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29913811, "name": "Rebecca Moore", "screen_name": "beccaamoore", "lang": "en", "location": "Washington, DC", "create_at": date("2009-04-08"), "description": "Writer, NASA Enthusiast, Queer Feminist Mormon, @TheMarsGen", "followers_count": 408, "friends_count": 480, "statues_count": 992 }, "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("2015-11-27T10:00:21.000Z"), "id": 670301145277972480, "text": "#Dunkin Donuts https://t.co/m3ykfEUMNa ok https://t.co/u1VrSklZRR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.57611968,44.25564762"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Dunkin" }}, "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": 4112, "friends_count": 876, "statues_count": 332302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montpelier, VT", "id": "495a55057ac886b9", "name": "Montpelier", "place_type": "city", "bounding_box": rectangle("-72.625573,44.234872 -72.544556,44.312702") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50023, "countyName": "Washington", "cityID": 5046000, "cityName": "Montpelier" } }
+{ "create_at": datetime("2015-11-27T10:00:21.000Z"), "id": 670301145303150592, "text": "@dunkindonuts #Dunkin Donuts / https://t.co/m3ykfEUMNa ok https://t.co/5qnPdWB4Eq", "in_reply_to_status": -1, "in_reply_to_user": 8771022, "favorite_count": 0, "coordinate": point("-72.57611968,44.25564762"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Dunkin" }}, "user_mentions": {{ 8771022 }}, "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": 4112, "friends_count": 876, "statues_count": 332302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montpelier, VT", "id": "495a55057ac886b9", "name": "Montpelier", "place_type": "city", "bounding_box": rectangle("-72.625573,44.234872 -72.544556,44.312702") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50023, "countyName": "Washington", "cityID": 5046000, "cityName": "Montpelier" } }
+{ "create_at": datetime("2015-11-27T10:00:21.000Z"), "id": 670301145575763969, "text": "Forward Air Transportation Services: CDL Class A Truckers / Truck Driver Owner... (#Carson, CA) https://t.co/94PYS4XbPQ #Transportation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.2820165,33.8314058"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Carson", "Transportation" }}, "user": { "id": 148606848, "name": "TMJ-CA Transport.", "screen_name": "tmj_CA_transp", "lang": "en", "location": "California", "create_at": date("2010-05-26"), "description": "Follow this account for geo-targeted Transportation job tweets in California Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 423, "friends_count": 296, "statues_count": 721 }, "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("2015-11-27T10:00:21.000Z"), "id": 670301145609445376, "text": "@feelthechar lmao", "in_reply_to_status": 670300865677230080, "in_reply_to_user": 3857827757, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3857827757 }}, "user": { "id": 1117060092, "name": "fl edm man", "screen_name": "_boogersugar", "lang": "en", "location": "CHiCARGO", "create_at": date("2013-01-24"), "description": "eyes wider than ohio", "followers_count": 2601, "friends_count": 293, "statues_count": 116666 }, "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("2015-11-27T10:00:21.000Z"), "id": 670301146011930624, "text": "Starbucks #Art : store design administrator, Store Design and Development - Miami, FL (#Miami, Florida) https://t.co/xkyT3RdJEb #Veterans", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.2264393,25.7889689"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Art", "Miami", "Veterans" }}, "user": { "id": 23082136, "name": "TMJ-MIA Art Jobs", "screen_name": "tmj_mia_art", "lang": "en", "location": "Miami, FL", "create_at": date("2009-03-06"), "description": "Follow this account for geo-targeted Art/Design job tweets in Miami, FL from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 234, "friends_count": 183, "statues_count": 12 }, "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("2015-11-27T10:00:21.000Z"), "id": 670301146129436672, "text": "I need coffee... Or a rockstar. Kill me please.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1210250580, "name": "k y l i e ☯", "screen_name": "Kylieee123", "lang": "en", "location": "Vancouver, WA", "create_at": date("2013-02-22"), "description": "•• kylie pekar •• 20 •• pnw •• II I MMIX ••", "followers_count": 176, "friends_count": 401, "statues_count": 10058 }, "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("2015-11-27T10:00:21.000Z"), "id": 670301146209095681, "text": "@x_Weirdo_ ������", "in_reply_to_status": 670299336354504704, "in_reply_to_user": 287881173, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 287881173 }}, "user": { "id": 534963325, "name": "braydon la flare", "screen_name": "Keep_UpWithBray", "lang": "en", "location": "cali - texas ✈️", "create_at": date("2012-03-23"), "description": "• yeezy taught me • unt'18", "followers_count": 1719, "friends_count": 1074, "statues_count": 57969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fate, TX", "id": "015f441c1bd0a858", "name": "Fate", "place_type": "city", "bounding_box": rectangle("-96.418302,32.918454 -96.372343,32.955358") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48397, "countyName": "Rockwall", "cityID": 4825572, "cityName": "Fate" } }
+{ "create_at": datetime("2015-11-27T10:00:21.000Z"), "id": 670301146288750592, "text": "Rain this afternoon, high 47 (8 C). Low 37 (3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433220, "name": "St. Louis Weather", "screen_name": "StLouisWeather", "lang": "en", "location": "St. Louis Metro Area", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for St. Louis Metro Area", "followers_count": 3677, "friends_count": 88, "statues_count": 7599 }, "place": { "country": "United States", "country_code": "United States", "full_name": "St. Louis, MO", "id": "60e6df5778ff9dac", "name": "St. Louis", "place_type": "city", "bounding_box": rectangle("-90.320515,38.531852 -90.166409,38.774346") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29510, "countyName": "St. Louis", "cityID": 2965000, "cityName": "St. Louis" } }
+{ "create_at": datetime("2015-11-27T10:00:21.000Z"), "id": 670301146431361024, "text": "January 4 �� https://t.co/1tFujkKU3z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3003963204, "name": "Paolo Picasso ♨", "screen_name": "hollywoodm0bbin", "lang": "en", "location": "htx", "create_at": date("2015-01-30"), "description": "null", "followers_count": 383, "friends_count": 328, "statues_count": 10284 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-27T10:00:21.000Z"), "id": 670301146473324544, "text": "@DezBryant huge fan, and a lot of respect, cause if a guy was like Norman was trashing me I'd lose my cool. Glad you don't.", "in_reply_to_status": -1, "in_reply_to_user": 174533632, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 174533632 }}, "user": { "id": 414664955, "name": "Travis Owens", "screen_name": "towenscomedy", "lang": "en", "location": "null", "create_at": date("2011-11-17"), "description": "Texan, Arkansan, Male, grown up version of me 20 something years ago. Making things happen.", "followers_count": 5972, "friends_count": 309, "statues_count": 2587 }, "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("2015-11-27T10:00:21.000Z"), "id": 670301146594996224, "text": "@danielchalfa I'm here rn", "in_reply_to_status": 670301060947247104, "in_reply_to_user": 247087497, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 247087497 }}, "user": { "id": 1724247324, "name": "Çürlyfrïëš", "screen_name": "rickyardo__", "lang": "en", "location": "null", "create_at": date("2013-09-02"), "description": "BoNita // Im a bit sarcastic", "followers_count": 407, "friends_count": 269, "statues_count": 9309 }, "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("2015-11-27T10:00:21.000Z"), "id": 670301146834055168, "text": "how r u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2962465598, "name": "Burr Apologist™", "screen_name": "TJEFFFUCKER666", "lang": "en", "location": "xe/xem | maggie", "create_at": date("2015-01-06"), "description": "Mr. Burr Please Let Me Leave ♡ @latinaavenger", "followers_count": 1532, "friends_count": 549, "statues_count": 31743 }, "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-27T10:00:21.000Z"), "id": 670301146951487489, "text": "Join the CVS Health team! See our latest #Retail #job opening here: https://t.co/ANyZVjPuho #Chalfont, PA #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.2090623,40.2884395"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "Chalfont", "Hiring", "CareerArc" }}, "user": { "id": 28536854, "name": "TMJ-PHL Retail Jobs", "screen_name": "tmj_phl_retail", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Philadelphia, PA. Need help? Tweet us at @CareerArc!", "followers_count": 431, "friends_count": 314, "statues_count": 1005 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chalfont, PA", "id": "7a1e514fe55d7b87", "name": "Chalfont", "place_type": "city", "bounding_box": rectangle("-75.225463,40.277487 -75.190206,40.30341") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42017, "countyName": "Bucks", "cityID": 4212504, "cityName": "Chalfont" } }
+{ "create_at": datetime("2015-11-27T10:00:21.000Z"), "id": 670301147358367744, "text": "Part Time - Order Picker/Packer/Sorters - Hiring Event, Irving, TX - Kelly Services: (#Irving, TX) https://t.co/T9rD730iBi #Manufacturing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.9477532,32.8916766"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Irving", "Manufacturing" }}, "user": { "id": 20831805, "name": "TMJ - DFW Manuf Jobs", "screen_name": "tmj_dfw_manuf", "lang": "en", "location": "DFW, Texas", "create_at": date("2009-02-13"), "description": "Follow this account for geo-targeted Manufacturing job tweets in DFW, Texas from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 339, "friends_count": 204, "statues_count": 135 }, "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("2015-11-27T10:00:21.000Z"), "id": 670301147559829504, "text": "I'm at orange theory fitness carrollwood in Carrollwood, FL https://t.co/NhHx6BVx3l", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.50642459,28.06578641"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14237766, "name": "Tony Torres", "screen_name": "tonytorres", "lang": "en", "location": "Tampa, FL", "create_at": date("2008-03-27"), "description": "null", "followers_count": 258, "friends_count": 466, "statues_count": 4908 }, "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("2015-11-27T10:00:21.000Z"), "id": 670301147849166848, "text": "Tour and shop the Battleship today. Save 10% on purchases. Open to 3pm. https://t.co/q7RvdX2Ji6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.00434699,39.85798068"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 38019109, "name": "USS New Jersey", "screen_name": "BattleshipNJ", "lang": "en", "location": "Camden Waterfront, NJ", "create_at": date("2009-05-05"), "description": "Take a tour of our nation's largest & most decorated ship - the Battleship New Jersey, named 2012 Best of Philly Historic Attraction, on Camden Waterfront, NJ", "followers_count": 5775, "friends_count": 3050, "statues_count": 1845 }, "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("2015-11-27T10:00:21.000Z"), "id": 670301147958145024, "text": "CVS Health #Retail #Job: Retail Store Shift Supervisor (#Wilmington, NC) https://t.co/LG38aqmxKc #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.9447102,34.2257255"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "Job", "Wilmington", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 28701039, "name": "TMJ-WIL Retail Jobs", "screen_name": "tmj_wil_retail", "lang": "en", "location": "Wilmington, NC", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Wilmington, NC from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 354, "friends_count": 310, "statues_count": 85 }, "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.32187,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37129, "countyName": "New Hanover", "cityID": 3774440, "cityName": "Wilmington" } }
+{ "create_at": datetime("2015-11-27T10:00:22.000Z"), "id": 670301148310474752, "text": "@YetriTV I just put on the face and BOOOM it's trending", "in_reply_to_status": 670300865098551296, "in_reply_to_user": 2584412496, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2584412496 }}, "user": { "id": 3098421852, "name": "Beeism", "screen_name": "iiQueenBeee", "lang": "en", "location": "in Scriptons Deving Dungeon", "create_at": date("2015-03-19"), "description": "Kindly deposit your 2 cents in my stfu jar (☞゚∀゚)☞ 18 y/o (rarely politically correct, slightly nsfw & infamous for lack of driving skills)", "followers_count": 1308, "friends_count": 482, "statues_count": 7299 }, "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-27T10:00:22.000Z"), "id": 670301148532748288, "text": "See a virtual tour of our listing on 288 E Chestnut Ave #Crestview #FL https://t.co/zhBpOmsNjK #realestate https://t.co/krnYz2UPBe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.566887,30.757548"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Crestview", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6591 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crestview, FL", "id": "32bb9ace0723336d", "name": "Crestview", "place_type": "city", "bounding_box": rectangle("-86.635021,30.695757 -86.534517,30.805126") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1215475, "cityName": "Crestview" } }
+{ "create_at": datetime("2015-11-27T10:00:22.000Z"), "id": 670301149501784064, "text": "@madisonnhxoxo thanks so much ��", "in_reply_to_status": 670300365946884096, "in_reply_to_user": 473570074, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 473570074 }}, "user": { "id": 350684339, "name": "madi", "screen_name": "madiirenaee", "lang": "en", "location": "glenpool | tahlequah", "create_at": date("2011-08-07"), "description": "null", "followers_count": 814, "friends_count": 362, "statues_count": 7763 }, "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("2015-11-27T10:00:22.000Z"), "id": 670301149816205312, "text": "������ https://t.co/3sj5WYkE9T", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 748478851, "name": "Liaaaa Mae ❗️❤️", "screen_name": "liaadurham", "lang": "en", "location": "LongLiveMon .. I Love You ❗️", "create_at": date("2012-08-09"), "description": "#MONSWORLD ❤️ | @LilHunter_ ❤️ |", "followers_count": 1000, "friends_count": 673, "statues_count": 54826 }, "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-27T10:00:22.000Z"), "id": 670301149845680132, "text": "@Jimmy_Burch Do me a favor and either DM or email jon.chuckery@cbsradio.com your contact number...My producers name is Orin..Thanks Jimmy!", "in_reply_to_status": 670300682704949249, "in_reply_to_user": 266083428, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 266083428 }}, "user": { "id": 860185507, "name": "Jon M Chuckery", "screen_name": "jmch316", "lang": "en", "location": "Atlanta, GA", "create_at": date("2012-10-03"), "description": "Co-host @cfbgametime929 from 9am-12pm every Saturday on @929TheGame...Also your favorite EOG - Until host...Sarcasm Daily! Canton OH to Atlanta GA", "followers_count": 648, "friends_count": 594, "statues_count": 6565 }, "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("2015-11-27T10:00:22.000Z"), "id": 670301149853908992, "text": "Numbed 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": 364039373, "name": "Tylersmind?", "screen_name": "TylersMind", "lang": "en", "location": "null", "create_at": date("2011-08-28"), "description": "It just is. 51615", "followers_count": 355, "friends_count": 335, "statues_count": 7605 }, "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("2015-11-27T10:00:22.000Z"), "id": 670301149854113793, "text": "@georgeerideoutt https://t.co/TNpQcPFCIt", "in_reply_to_status": -1, "in_reply_to_user": 2259303165, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2259303165 }}, "user": { "id": 2300458731, "name": "taylor", "screen_name": "taylor_ormann", "lang": "en", "location": "null", "create_at": date("2014-01-23"), "description": "null", "followers_count": 218, "friends_count": 206, "statues_count": 107 }, "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("2015-11-27T10:00:22.000Z"), "id": 670301149862494208, "text": "LMAO https://t.co/1VQkLgDx5C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 38951953, "name": "㊙️", "screen_name": "NoJuiceOrWoes", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2009-05-09"), "description": "RIP Lah & Ju", "followers_count": 1755, "friends_count": 1298, "statues_count": 34638 }, "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("2015-11-27T10:00:22.000Z"), "id": 670301150093012992, "text": "If I didn't have to work, I'd be starting fights at Walmart.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 278782122, "name": "Erick", "screen_name": "Erickonfire", "lang": "en", "location": "San Marcos, Tx", "create_at": date("2011-04-07"), "description": "Find what you love and let it kill you.", "followers_count": 185, "friends_count": 226, "statues_count": 18680 }, "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-27T10:00:22.000Z"), "id": 670301150286098432, "text": "@Freelos10 shut up always saying some", "in_reply_to_status": 670301009915346944, "in_reply_to_user": 2802897205, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2802897205 }}, "user": { "id": 3373616014, "name": "9/1❣", "screen_name": "demigotfans", "lang": "en", "location": "with/montell❤️", "create_at": date("2015-07-13"), "description": "trill Sammy ❤️", "followers_count": 250, "friends_count": 227, "statues_count": 4750 }, "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("2015-11-27T10:00:22.000Z"), "id": 670301150449537024, "text": "if my calculations are right, i have to read 401 hours a day to finish my book in 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": 947145661, "name": "cc", "screen_name": "cierraaacastro", "lang": "en", "location": "ehs var soccer / arsenal ecnl", "create_at": date("2012-11-13"), "description": "null", "followers_count": 1045, "friends_count": 894, "statues_count": 16979 }, "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-27T10:00:22.000Z"), "id": 670301150449676288, "text": "less than 2 hours 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": 3438310289, "name": "James Brown", "screen_name": "jbrown2_", "lang": "en", "location": "null", "create_at": date("2015-08-24"), "description": "This lost boy got fly without Peter Pan -Drake", "followers_count": 64, "friends_count": 73, "statues_count": 855 }, "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("2015-11-27T10:00:22.000Z"), "id": 670301150495813632, "text": "I'm at Nailtime in Houston, TX https://t.co/uxNgk3Xp3z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.40610147,29.75350486"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 151742290, "name": "Cin Break", "screen_name": "sinbreak", "lang": "en", "location": "Texas, USA", "create_at": date("2010-06-03"), "description": "If I'm always underestimated, then I'll always overperform.", "followers_count": 62, "friends_count": 168, "statues_count": 1828 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-27T10:00:22.000Z"), "id": 670301150722289664, "text": "Adele pendant nickel @ArteriorsHome modern lighting https://t.co/0Pz5YHQYCr polished light fixture https://t.co/U9NpZP0Tgj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 56803879 }}, "user": { "id": 32629529, "name": "Clayton Gray Home", "screen_name": "ClaytonGrayHome", "lang": "en", "location": "Florida", "create_at": date("2009-04-17"), "description": "Modern home decor website\r\nFOLLOW US : \r\nhttps://t.co/bk7EK5kF7D\r\nhttp://t.co/UK8yCDXTDF\r\nhttp://t.co/iuREE31Xoi", "followers_count": 903, "friends_count": 949, "statues_count": 2121 }, "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("2015-11-27T10:00:22.000Z"), "id": 670301150898315265, "text": "We're #hiring! Click to apply: Assistant General Manager - Anaheim - https://t.co/ahIzul8b0W #BusinessMgmt #management #Anaheim, CA #Job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.9145036,33.8352932"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "BusinessMgmt", "management", "Anaheim", "Job" }}, "user": { "id": 2228943546, "name": "Ghirardelli Careers", "screen_name": "ghirardellijobs", "lang": "en", "location": "null", "create_at": date("2013-12-03"), "description": "Ghirardelli invites you to explore our open #jobs and discover a place where your ideas come to life, as part of our delicious legacy.", "followers_count": 52, "friends_count": 1, "statues_count": 98 }, "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-27T10:00:22.000Z"), "id": 670301150919458816, "text": "@YoAylex_23 @Matt_Rozic we were something happened thoo��", "in_reply_to_status": 670300028909584385, "in_reply_to_user": 291351663, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 291351663, 318124103 }}, "user": { "id": 2823952619, "name": "Anthony Linert", "screen_name": "anthonylinert", "lang": "en", "location": "null", "create_at": date("2014-10-11"), "description": "Life goes on", "followers_count": 290, "friends_count": 253, "statues_count": 1569 }, "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("2015-11-27T10:00:22.000Z"), "id": 670301151204646912, "text": "I'm in Waukesha where you at @nicolemoser13", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 31284616 }}, "user": { "id": 700597998, "name": "Erin", "screen_name": "Erin_Falko7", "lang": "en", "location": "null", "create_at": date("2012-07-17"), "description": "you're ugly.", "followers_count": 828, "friends_count": 771, "statues_count": 40369 }, "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("2015-11-27T10:00:22.000Z"), "id": 670301151829577732, "text": "can summer hurry up?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1587229867, "name": "✨hanjaababyy✨", "screen_name": "hanjamorales", "lang": "en", "location": "IG: adorethehispanic ", "create_at": date("2013-07-11"), "description": "The Love Be So Fake But The Hate Be So Real ..", "followers_count": 408, "friends_count": 355, "statues_count": 12243 }, "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("2015-11-27T10:00:22.000Z"), "id": 670301152181817344, "text": "We are looking for a buyer on 704 E Chestnut Ave #Crestview #FL https://t.co/bKXLD1iNxu #realestate https://t.co/jtYamEiahg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.561744,30.757919"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Crestview", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6592 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crestview, FL", "id": "32bb9ace0723336d", "name": "Crestview", "place_type": "city", "bounding_box": rectangle("-86.635021,30.695757 -86.534517,30.805126") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1215475, "cityName": "Crestview" } }
+{ "create_at": datetime("2015-11-27T10:00:22.000Z"), "id": 670301152211304449, "text": "I know just pissing them off revealing there true inner racist https://t.co/TXulPAmBZQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27309052, "name": "†B", "screen_name": "MissHonieB", "lang": "en", "location": "east coast", "create_at": date("2009-03-28"), "description": "free spirit with a wild heart. . . . . . . . . snapchat & IG @MissHonieB", "followers_count": 1176, "friends_count": 802, "statues_count": 65182 }, "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("2015-11-27T10:00:22.000Z"), "id": 670301152253112320, "text": "@solaaast I figured �� thanks boo", "in_reply_to_status": 670300916176674816, "in_reply_to_user": 46509448, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 46509448 }}, "user": { "id": 369573845, "name": "dalia✨", "screen_name": "loveeexdaliaaa", "lang": "en", "location": "dallas, tx", "create_at": date("2011-09-07"), "description": "wise girl chasin' her dreams.", "followers_count": 276, "friends_count": 161, "statues_count": 11335 }, "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("2015-11-27T10:00:23.000Z"), "id": 670301152492261376, "text": "I really wanted to eat my plate from yesterday high but looks like that's not 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": 386756998, "name": "IFITAINTABOUTAMANI", "screen_name": "Robo_kushu", "lang": "en", "location": "broward county ", "create_at": date("2011-10-07"), "description": "(I) (want) happiness", "followers_count": 880, "friends_count": 728, "statues_count": 77345 }, "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("2015-11-27T10:00:23.000Z"), "id": 670301152660037632, "text": "Mexico ... Mexico all after next weekend ...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1069098223, "name": "B47", "screen_name": "BrahadonR10", "lang": "en", "location": "null", "create_at": date("2013-01-07"), "description": "Call me B IG: Brahadon10r & Bcreativ3 22 Chicago Artist", "followers_count": 636, "friends_count": 414, "statues_count": 41274 }, "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("2015-11-27T10:00:23.000Z"), "id": 670301152760721409, "text": "@Aximerrr shame on him man", "in_reply_to_status": 670296900134961157, "in_reply_to_user": 281234028, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 281234028 }}, "user": { "id": 283718865, "name": "TK #GiLU", "screen_name": "T_Kelllz", "lang": "en", "location": "Detroit, MI", "create_at": date("2011-04-17"), "description": "My reputations on the line, so I'm working on a better me. #GiLU RMR COD and Music consume all of me. PSN: T_Kelly313", "followers_count": 1990, "friends_count": 812, "statues_count": 106281 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Royal Oak, MI", "id": "47cfe29f57708110", "name": "Royal Oak", "place_type": "city", "bounding_box": rectangle("-83.204418,42.474131 -83.111076,42.540305") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2670040, "cityName": "Royal Oak" } }
+{ "create_at": datetime("2015-11-27T10:00:23.000Z"), "id": 670301153372966912, "text": "Had a dream I took @porterrobinson to shirokiya to eat takoyaki in Hawaii. Still bummed I cant see you tomorrow -_________-", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.618061,39.99910117"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 159708091 }}, "user": { "id": 568942306, "name": "Jessa 道子【=◈︿◈=】", "screen_name": "Pinkaybuu", "lang": "en", "location": "HNL ✈ PDX ✈ LAX", "create_at": date("2012-05-02"), "description": "Hawai'i grown ⛅ Electronic music is my escape ✨✌ @porterrobinson 's bb ♥", "followers_count": 237, "friends_count": 443, "statues_count": 4308 }, "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": 8001, "countyName": "Adams" } }
+{ "create_at": datetime("2015-11-27T10:00:23.000Z"), "id": 670301153456881664, "text": "Se volvió loco Frodo ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 365254425, "name": "Vj", "screen_name": "VictorJoves", "lang": "es", "location": "null", "create_at": date("2011-08-30"), "description": "Moonshine", "followers_count": 293, "friends_count": 206, "statues_count": 6032 }, "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("2015-11-27T10:00:23.000Z"), "id": 670301153536557056, "text": "Yesterday was awesome !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 171088644, "name": "⚡addicted2ink", "screen_name": "JayFromThe90s_", "lang": "en", "location": "Portland, OR", "create_at": date("2010-07-26"), "description": "I always spread love but sometimes I sin / Unprofessional skateboarder / @nikesb / music makes the world go ✌️", "followers_count": 2195, "friends_count": 1947, "statues_count": 22722 }, "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("2015-11-27T10:00:23.000Z"), "id": 670301153544900614, "text": "Antay-antay na sa mga ganap mamaya.. but meantime, have to go to work.. lol. but magbabalik para sa power tweet! #ALDUBStaySTRONG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "ALDUBStaySTRONG" }}, "user": { "id": 437153064, "name": "ALDUB/Missouri", "screen_name": "Teri_40", "lang": "en", "location": "Manchester", "create_at": date("2011-12-14"), "description": "null", "followers_count": 49, "friends_count": 43, "statues_count": 6020 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballwin, MO", "id": "cc1dc38105ac44d4", "name": "Ballwin", "place_type": "city", "bounding_box": rectangle("-90.59217,38.54757 -90.509959,38.624853") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2903160, "cityName": "Ballwin" } }
+{ "create_at": datetime("2015-11-27T10:00:23.000Z"), "id": 670301153557483520, "text": "Finally got caught up on 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": 512786027, "name": "Loxley™", "screen_name": "A_Marconi55", "lang": "en", "location": "null", "create_at": date("2012-03-02"), "description": "I am the MANIC, I am the GHOUL • 19 • Houston, TX • IG @A_Marconi55 • Snapchat @A_marconi55 #RespectFewFearNone #TrainToWin", "followers_count": 222, "friends_count": 283, "statues_count": 11143 }, "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-27T10:00:23.000Z"), "id": 670301154144821248, "text": "@LimitedHangOut1 es una vergüenza", "in_reply_to_status": 670298409526538240, "in_reply_to_user": 2304222866, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 2304222866 }}, "user": { "id": 74768666, "name": "Richo", "screen_name": "Kemprich", "lang": "en", "location": "Manhattan, New York", "create_at": date("2009-09-16"), "description": "null", "followers_count": 319, "friends_count": 639, "statues_count": 9580 }, "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("2015-11-27T10:00:23.000Z"), "id": 670301154149036032, "text": "@fabyvil #CURLS", "in_reply_to_status": 670299766253842432, "in_reply_to_user": 156697748, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "CURLS" }}, "user_mentions": {{ 156697748 }}, "user": { "id": 1476247884, "name": "Annette", "screen_name": "Annette00159182", "lang": "en", "location": "Ohio", "create_at": date("2013-06-01"), "description": "TMJT July 10, 2013 Moonshine Jungle Tour June 28, 2014 June 30, 2014", "followers_count": 608, "friends_count": 490, "statues_count": 34759 }, "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("2015-11-27T10:00:23.000Z"), "id": 670301154237132800, "text": "when you were excited to make bank at work and then your manager tells you not to come in https://t.co/EKJXnKf29v", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 283835945, "name": "Daws", "screen_name": "_millbauer", "lang": "en", "location": "probably Cadey's house", "create_at": date("2011-04-17"), "description": "@cadeymccormack is the loml", "followers_count": 868, "friends_count": 650, "statues_count": 36168 }, "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("2015-11-27T10:00:23.000Z"), "id": 670301155054895104, "text": "See our latest #LoneTree, CO #job and click to apply: Managing Director, Business Learning - https://t.co/sgNdb04tkI #HR #Veterans #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.8963682,39.5360997"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LoneTree", "job", "HR", "Veterans", "Hiring" }}, "user": { "id": 313571057, "name": "Colorado HR Jobs", "screen_name": "tmj_CO_HR", "lang": "en", "location": "Colorado Non-Metro", "create_at": date("2011-06-08"), "description": "Follow this account for geo-targeted Human Resources job tweets in Colorado Non-Metro from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 364, "friends_count": 285, "statues_count": 27 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carriage Club, CO", "id": "2d9dc2a8b478da71", "name": "Carriage Club", "place_type": "city", "bounding_box": rectangle("-104.904327,39.521709 -104.885217,39.53651") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8035, "countyName": "Douglas", "cityID": 845955, "cityName": "Lone Tree" } }
+{ "create_at": datetime("2015-11-27T10:00:23.000Z"), "id": 670301155130388480, "text": "Thought that was a chili cheese dog https://t.co/5LpF0nRiba", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 445048857, "name": "Pero Like", "screen_name": "mellamopapi_", "lang": "en", "location": "Earth", "create_at": date("2011-12-23"), "description": "Watch me get this money", "followers_count": 3987, "friends_count": 1324, "statues_count": 165514 }, "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("2015-11-27T10:00:23.000Z"), "id": 670301155138732032, "text": "que lloren lo que yo quiero es que lloren", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2387257286, "name": "mamiii", "screen_name": "suggaarrr", "lang": "en", "location": "null", "create_at": date("2014-03-13"), "description": "soy de Durango, el estado más chingon ⠀⠀⠀⠀⠀⠀ ⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀", "followers_count": 1987, "friends_count": 1535, "statues_count": 8038 }, "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("2015-11-27T10:00:23.000Z"), "id": 670301156103450625, "text": "Check out our #listing in #Crestview #FL https://t.co/jOxYxunkOi #realestate #realtor https://t.co/bvfTd36nPA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.561833,30.758687"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "Crestview", "FL", "realestate", "realtor" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crestview, FL", "id": "32bb9ace0723336d", "name": "Crestview", "place_type": "city", "bounding_box": rectangle("-86.635021,30.695757 -86.534517,30.805126") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1215475, "cityName": "Crestview" } }
+{ "create_at": datetime("2015-11-27T10:00:23.000Z"), "id": 670301156132810752, "text": "Hey could someone please turn this thing on? It's a little nippy out here today! https://t.co/OkWXJtOtz8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30493597, "name": "Teresa Williams", "screen_name": "motherofts", "lang": "en", "location": "null", "create_at": date("2009-04-11"), "description": "null", "followers_count": 143, "friends_count": 212, "statues_count": 790 }, "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-27T10:00:23.000Z"), "id": 670301156179079168, "text": "queen���� https://t.co/tnUUERsnOv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2914375857, "name": "Del", "screen_name": "dkendrick_3", "lang": "en", "location": "Cohoes, NY", "create_at": date("2014-12-10"), "description": "null", "followers_count": 235, "friends_count": 218, "statues_count": 2689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cohoes, NY", "id": "284c8600e6f91140", "name": "Cohoes", "place_type": "city", "bounding_box": rectangle("-73.731704,42.751281 -73.678546,42.794208") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36001, "countyName": "Albany", "cityID": 3616749, "cityName": "Cohoes" } }
+{ "create_at": datetime("2015-11-27T10:00:23.000Z"), "id": 670301156485132288, "text": "Then, let's say redheads, or people with curly hair. All I'm saying is capitalism needs to exploit @PoliticsPeach @Ramazone2 @NewNatTurner", "in_reply_to_status": 670299512255197184, "in_reply_to_user": 124227622, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 124227622, 102736422, 2881918551 }}, "user": { "id": 330488319, "name": "Connie Collins", "screen_name": "khanknee", "lang": "en", "location": "null", "create_at": date("2011-07-06"), "description": "Love Peace & Light ❤️ Initiator of #Blackpeopleconfessions & #Whitepeopleconfessions Let's Get Real about Racism It Kills Dreams & People", "followers_count": 3239, "friends_count": 1041, "statues_count": 68867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-27T10:00:24.000Z"), "id": 670301156858572801, "text": "Stealing a moment with our lovely couple before the festivities commenced: How beautiful are… https://t.co/fYocWyZ3UE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.45723299,29.74123413"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19133417, "name": "A Day To Remember", "screen_name": "DayToRemember", "lang": "en", "location": "Houston, TX, serving worldwide", "create_at": date("2009-01-17"), "description": "Award-winning Wedding Planners serving Houston and worldwide.\r\nLove life. Live happy. Dance like nobody's business!", "followers_count": 3052, "friends_count": 1112, "statues_count": 15718 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-27T10:00:24.000Z"), "id": 670301157026344964, "text": "@tacobell �� + ��", "in_reply_to_status": 663563502238330880, "in_reply_to_user": 7831092, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 7831092 }}, "user": { "id": 242587223, "name": "Shelly Duren", "screen_name": "shellyduren", "lang": "en", "location": "ATL", "create_at": date("2011-01-24"), "description": "gotta let your soul shine", "followers_count": 380, "friends_count": 332, "statues_count": 16786 }, "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("2015-11-27T10:00:24.000Z"), "id": 670301157030391808, "text": "Sports Medicine is right to believe that common stretching applying indiscriminate force to parts of the body are dangerous.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.7767558,32.7676428"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3871451054, "name": "Toni Romeo-Robinzine", "screen_name": "ToniCoreFitness", "lang": "en", "location": "null", "create_at": date("2015-10-12"), "description": "null", "followers_count": 356, "friends_count": 2088, "statues_count": 528 }, "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-27T10:00:24.000Z"), "id": 670301157844082688, "text": "I just want some caldo de pollo and for Buddha's dog to stfu and 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": 3037256407, "name": "Manchester United", "screen_name": "sirAlex22ManU", "lang": "en", "location": "null", "create_at": date("2015-02-22"), "description": "life ain't nothing but a struggle so man the fuck up and don't act like a bitch", "followers_count": 97, "friends_count": 147, "statues_count": 2481 }, "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("2015-11-27T10:00:24.000Z"), "id": 670301158003507201, "text": "Such elegance. ❤️ https://t.co/pWSm3dZHCW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2680412366, "name": "yung hot ebony", "screen_name": "gl0hen", "lang": "en", "location": "DTX · PVTX ", "create_at": date("2014-07-25"), "description": "21. INDIGIRL. I'm Ebony. Your beautiful, dark, twisted fantasy. So I guess you could say Yeezy taught me. Fly High, Newt❤️| ❂☯☮ IG: yungebby_", "followers_count": 1499, "friends_count": 660, "statues_count": 86074 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-27T10:00:24.000Z"), "id": 670301158230093828, "text": "@amellywood I voted #Arrow \nVote here https://t.co/uy3rE1W1Rt\n#PeoplesChoiceAwards https://t.co/kyULkbTUGv", "in_reply_to_status": -1, "in_reply_to_user": 349300271, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Arrow", "PeoplesChoiceAwards" }}, "user_mentions": {{ 349300271 }}, "user": { "id": 474310832, "name": "Chris Ragan", "screen_name": "ChrisRagan99", "lang": "en", "location": "United States", "create_at": date("2012-01-25"), "description": "Houston Texans, Rockets, Astros, Cougars Texas Rangers, Longhorns & Dale Earnhardt Jr. Fan", "followers_count": 504, "friends_count": 1211, "statues_count": 40072 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hinesville, GA", "id": "6cc49882c2468f39", "name": "Hinesville", "place_type": "city", "bounding_box": rectangle("-81.678833,31.782055 -81.561514,31.87296") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13179, "countyName": "Liberty", "cityID": 1338964, "cityName": "Hinesville" } }
+{ "create_at": datetime("2015-11-27T10:00:24.000Z"), "id": 670301158427262976, "text": "So I decided to get the mane cleaned up at the hair salon today, my stylist has played have u meet Keegan three times in the last 20 minutes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.0358938,42.7226743"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1115431080, "name": "Poseidon", "screen_name": "Keegan_Roth", "lang": "en", "location": "CMU Jamestown", "create_at": date("2013-01-23"), "description": "Almost Famous, Always Dangerous", "followers_count": 589, "friends_count": 582, "statues_count": 6291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washington, MI", "id": "00778621419923a6", "name": "Washington", "place_type": "city", "bounding_box": rectangle("-83.097073,42.712762 -82.977233,42.786766") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26099, "countyName": "Macomb" } }
+{ "create_at": datetime("2015-11-27T10:00:24.000Z"), "id": 670301158460649472, "text": "@continuants I own many nerdy shirts from @Fangamer. I cannot recommend them enough (and they’re including Gift Cards with orders right now)", "in_reply_to_status": 670300953602584576, "in_reply_to_user": 17886323, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17886323, 23064821 }}, "user": { "id": 8402932, "name": "Alex Bradley", "screen_name": "abrad45", "lang": "en", "location": "New Jersey", "create_at": date("2007-08-24"), "description": "Programmer. INTJ. Tea addict. @Eagles fan.", "followers_count": 325, "friends_count": 73, "statues_count": 13722 }, "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("2015-11-27T10:00:24.000Z"), "id": 670301158515326977, "text": "@nubyaa_ thank u so much ����", "in_reply_to_status": 670300426483318784, "in_reply_to_user": 3271222998, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3271222998 }}, "user": { "id": 1637413914, "name": "fran ✨", "screen_name": "franrenee_", "lang": "en", "location": "atx", "create_at": date("2013-08-01"), "description": "null", "followers_count": 700, "friends_count": 564, "statues_count": 10488 }, "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-27T10:00:24.000Z"), "id": 670301158611791873, "text": "The movie Miracle is honest to god so inspiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 438648743, "name": "emily.", "screen_name": "em_branstetter", "lang": "en", "location": "soon to be btown ", "create_at": date("2011-12-16"), "description": "big butt, bigger heart. -Michael Scott", "followers_count": 476, "friends_count": 526, "statues_count": 12012 }, "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("2015-11-27T10:00:24.000Z"), "id": 670301158766809089, "text": "Probably going to make a run to Michael's and the dollar store today because I'm so excited to start Christmas things", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2637637495, "name": "Autumn-Nichole", "screen_name": "FuzzySoxRox444", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2014-07-13"), "description": "Las Vegas. UNLV. Cosmetologist. Ulta. I like weird hair colors, soft blankets, animals, and perfume. Libra. ❥∘❁∘★∘☾∘☮", "followers_count": 465, "friends_count": 765, "statues_count": 10532 }, "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("2015-11-27T10:00:24.000Z"), "id": 670301159001837568, "text": "Im sitting so close that I can reach out and touch sly, may not be so bad! #Creed #slyvesterstallone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.8669,41.3771"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Creed", "slyvesterstallone" }}, "user": { "id": 23444339, "name": "Stacy", "screen_name": "NY_Sassy", "lang": "en", "location": "New York-Ct", "create_at": date("2009-03-09"), "description": "The only person you are destined to become is the person you decide to be. — Ralph Waldo Emerson", "followers_count": 245, "friends_count": 665, "statues_count": 5006 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cinemark", "id": "08ab3b5864143001", "name": "Cinemark", "place_type": "poi", "bounding_box": rectangle("-72.8669001,41.3770999 -72.8669,41.3771") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 954940, "cityName": "North Haven" } }
+{ "create_at": datetime("2015-11-27T10:00:24.000Z"), "id": 670301159333036032, "text": "My nigga not gay chill https://t.co/jMDIgQDSah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 479140817, "name": "nawfside shawty", "screen_name": "KingJourdain", "lang": "en", "location": "Monroe, LA", "create_at": date("2012-01-30"), "description": "19. God first | #WhoDatNation | college student | snapchat: jourdain96", "followers_count": 3802, "friends_count": 1997, "statues_count": 58510 }, "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("2015-11-27T10:00:24.000Z"), "id": 670301159345618945, "text": "@ynelpau2616 Don't worry d lang ikaw ang may sakit na ganyan! Ako rin, hehehehe.... ��Aldub supporter!!! #ALDUBStaySTRONG", "in_reply_to_status": 670296289075138561, "in_reply_to_user": 4175917277, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "ALDUBStaySTRONG" }}, "user_mentions": {{ 4175917277 }}, "user": { "id": 155363992, "name": "Jen Dizon", "screen_name": "luvsjen", "lang": "en", "location": "sunny daygo", "create_at": date("2010-06-13"), "description": "null", "followers_count": 33, "friends_count": 77, "statues_count": 2216 }, "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("2015-11-27T10:00:24.000Z"), "id": 670301159379378176, "text": "Exit. #queens #subway #station #nyc #mta @ Kew Gardens – Union Turnpike https://t.co/DVEfs3Ci5Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.830786,40.714151"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "queens", "subway", "station", "nyc", "mta" }}, "user": { "id": 82967972, "name": "Jannis Werner", "screen_name": "janniswerner", "lang": "en", "location": "Berlin, Germany", "create_at": date("2009-10-16"), "description": "Lawyer/Photographer", "followers_count": 140, "friends_count": 222, "statues_count": 573 }, "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("2015-11-27T10:00:24.000Z"), "id": 670301159408730112, "text": "Mooch Is @ Viewmont Mall Until 6! Hate To Wait? Call Ahead! 570.862.8383! #whatthefork", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "whatthefork" }}, "user": { "id": 514027267, "name": "What The FORK", "screen_name": "eatwtfork", "lang": "en", "location": "Pennsylvania, USA", "create_at": date("2012-03-03"), "description": "570.215.8991", "followers_count": 4941, "friends_count": 2408, "statues_count": 4726 }, "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("2015-11-27T10:00:24.000Z"), "id": 670301159752470528, "text": "Happy bday ily!!!���� @abbylorio_", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3045265795 }}, "user": { "id": 2692015971, "name": "Ashleigh Ward", "screen_name": "ashward0304", "lang": "en", "location": "null", "create_at": date("2014-07-09"), "description": "Yes, i met fifth harmony", "followers_count": 398, "friends_count": 338, "statues_count": 1658 }, "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("2015-11-27T10:00:24.000Z"), "id": 670301159874138115, "text": "Somos como líneas paralelas: siempre cerca, pero nunca juntos.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2366323280, "name": "Peter", "screen_name": "cruxlaD1", "lang": "en", "location": "null", "create_at": date("2014-02-28"), "description": "Quizas pasaran mil primaveras, pero para ser feliz basta un suspiro sin importar razas ni fronteras; en este poema en ti me inspiro. Santa Ana, Ca.", "followers_count": 20678, "friends_count": 20546, "statues_count": 21445 }, "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("2015-11-27T10:00:24.000Z"), "id": 670301159916113920, "text": "Black Friday is not a holiday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1246470260, "name": "Äspën", "screen_name": "AspenMallery", "lang": "en", "location": "Vancouver, WA", "create_at": date("2013-03-06"), "description": "Damaged people are dangerous. They know they can survive. Instagram:punkybrewster769", "followers_count": 146, "friends_count": 138, "statues_count": 1770 }, "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("2015-11-27T10:00:24.000Z"), "id": 670301160159383554, "text": "We're #hiring! Read about our latest #job opening here: Substitute Teachers Needed at Young Scholars of... - https://t.co/IWZKzSRrot", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.0171609,40.3675797"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job" }}, "user": { "id": 22788803, "name": "TMJ- PIT Educ. Jobs", "screen_name": "tmj_pit_edu", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2009-03-04"), "description": "Follow this account for geo-targeted Education/Teaching job tweets in Pittsburgh, PA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 306, "friends_count": 234, "statues_count": 21 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Castle Shannon, PA", "id": "52b8e53ae833638f", "name": "Castle Shannon", "place_type": "city", "bounding_box": rectangle("-80.038088,40.354859 -80.001567,40.379642") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4211680, "cityName": "Castle Shannon" } }
+{ "create_at": datetime("2015-11-27T10:00:24.000Z"), "id": 670301160218071042, "text": "No Sir, I'm not your wife! \nWTF!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 75557014, "name": "Lummie", "screen_name": "Lummiie", "lang": "es", "location": "ÜT: 18.971327,-99.238609", "create_at": date("2009-09-19"), "description": "Ciencia, laboratorio, genes, albercas, una bici y viajar. Mami de @identidADN y @DICLIM #DistractinglySexy. Siempre necesito vacaciones.", "followers_count": 1949, "friends_count": 726, "statues_count": 49429 }, "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-27T10:00:24.000Z"), "id": 670301160306266115, "text": "So is San Diego State out of the NY6 conversation?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 138187024, "name": "nick price", "screen_name": "nickprice91", "lang": "en", "location": "Washington DC", "create_at": date("2010-04-28"), "description": "null", "followers_count": 260, "friends_count": 1970, "statues_count": 32078 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301160943837185, "text": "I looked 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": 2358212844, "name": "12/08 ♐️", "screen_name": "nikeishabee", "lang": "en", "location": "somewhere boolin", "create_at": date("2014-02-23"), "description": "don't worry about me!", "followers_count": 846, "friends_count": 617, "statues_count": 21251 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301160985640961, "text": "@Abbz_sparkles11 can you really ever sleep too much?", "in_reply_to_status": 670216601728249856, "in_reply_to_user": 453722641, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 453722641 }}, "user": { "id": 394497994, "name": "Big Daddy Sig", "screen_name": "pjarzola", "lang": "en", "location": "Orange,Ca #frosty ", "create_at": date("2011-10-19"), "description": "NMSU ΣΧ", "followers_count": 560, "friends_count": 502, "statues_count": 9090 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301161082122240, "text": "@mc_wr06 あっそうなんけ!?つけるの楽しみやわ��笑", "in_reply_to_status": 670221954222153729, "in_reply_to_user": 2386295970, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user_mentions": {{ 2386295970 }}, "user": { "id": 3861589099, "name": "Anri", "screen_name": "anrihashimoto9", "lang": "ja", "location": "California, USA", "create_at": date("2015-10-11"), "description": "null", "followers_count": 21, "friends_count": 17, "statues_count": 35 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301161237438464, "text": "too busy chasing money ������ baby, I ain't got no time 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": 614851692, "name": "$OLO DOLO.", "screen_name": "SanaeDestiniee", "lang": "en", "location": "killadelphia ", "create_at": date("2012-06-21"), "description": "on my own..... RIP GEE ❤", "followers_count": 1694, "friends_count": 707, "statues_count": 46144 }, "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-27T10:00:25.000Z"), "id": 670301161363255296, "text": "Game day ready!! #HBD to my beautiful mom!! #gotigers #MIZ ����❤️ @… https://t.co/afbXWAJceM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.1609,36.0764"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HBD", "gotigers", "MIZ" }}, "user": { "id": 1628291084, "name": "Kathryn Clubb", "screen_name": "kathryn_rau", "lang": "en", "location": "null", "create_at": date("2013-07-28"), "description": "Wifey. Teacher. Barely use this thang!", "followers_count": 88, "friends_count": 232, "statues_count": 409 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301161438625793, "text": "Freezing rain this afternoon, high 33 (1 C). Low 30 (-1 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433608, "name": "Kansas City Weather", "screen_name": "KansasCityWX", "lang": "en", "location": "Kansas City Metro Area", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Kansas City Metro Area", "followers_count": 1923, "friends_count": 88, "statues_count": 7719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "ddabe75a8f44fe5a", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.765917,38.827176 -94.385522,39.356662") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2015-11-27T10:00:25.000Z"), "id": 670301161564475392, "text": "@ThatGalTaylor do you actually https://t.co/ZrL3ywMqXT", "in_reply_to_status": 670299332227149824, "in_reply_to_user": 184870919, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 184870919 }}, "user": { "id": 251437586, "name": "Aaron Paul", "screen_name": "Aaron_Garland", "lang": "en", "location": "Chicago // Jenks", "create_at": date("2011-02-12"), "description": "Oral Roberts University", "followers_count": 726, "friends_count": 241, "statues_count": 15439 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301161606520832, "text": "LETS GO RANGERS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 441402561, "name": "#PrayforParis", "screen_name": "JohnLiotti1", "lang": "en", "location": "Springfield, MA", "create_at": date("2011-12-19"), "description": "lets go rangers", "followers_count": 308, "friends_count": 538, "statues_count": 22884 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Freehold, NJ", "id": "7ffbe5f2df7fa020", "name": "Freehold", "place_type": "city", "bounding_box": rectangle("-74.319578,40.23751 -74.238132,40.29553") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3425200, "cityName": "Freehold" } }
+{ "create_at": datetime("2015-11-27T10:00:25.000Z"), "id": 670301161744760832, "text": "�� https://t.co/khskpRF7Mf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 64963581, "name": "Tera", "screen_name": "teranicole962", "lang": "en", "location": "null", "create_at": date("2009-08-11"), "description": "null", "followers_count": 106, "friends_count": 107, "statues_count": 1250 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301161916866560, "text": "Two crazy puppies @ The Eagles Nest https://t.co/ydIrO9uzNw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.08550879,36.0459522"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16132521, "name": "Rob Freudenberg", "screen_name": "rfreuden", "lang": "en", "location": "All Over", "create_at": date("2008-09-04"), "description": "Dad, husband, open source evangelist, tinkerer, car dude, dog owner and other stuff. All opinions are my own! Prius' get passed!", "followers_count": 149, "friends_count": 504, "statues_count": 2953 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsborough, NC", "id": "01b9eb2c198acccb", "name": "Hillsborough", "place_type": "city", "bounding_box": rectangle("-79.125303,36.029983 -79.071661,36.101656") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37135, "countyName": "Orange", "cityID": 3731620, "cityName": "Hillsborough" } }
+{ "create_at": datetime("2015-11-27T10:00:25.000Z"), "id": 670301162093023232, "text": "Let's go Lady Vols! https://t.co/NWRCUdXH84", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.925462,35.951643"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2832957561, "name": "Regie Jones", "screen_name": "regie_jones", "lang": "en", "location": "Greeneville, TN", "create_at": date("2014-10-15"), "description": "Real Estate Broker at Realty Executives Greeneville", "followers_count": 155, "friends_count": 441, "statues_count": 55 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thompson-Boling Arena", "id": "07d9ec8f02486000", "name": "Thompson-Boling Arena", "place_type": "poi", "bounding_box": rectangle("-83.92546209999999,35.951642899999996 -83.925462,35.951643") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47093, "countyName": "Knox", "cityID": 4740000, "cityName": "Knoxville" } }
+{ "create_at": datetime("2015-11-27T10:00:25.000Z"), "id": 670301162432765953, "text": "First pair of Yeezy's copped", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 745867716, "name": "Austin", "screen_name": "AustinVanWagner", "lang": "en", "location": "Out Here", "create_at": date("2012-08-08"), "description": "Yee", "followers_count": 602, "friends_count": 292, "statues_count": 12949 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301162739007488, "text": "@MandyCorine hi Mandy. Just read your article \"the style mistake most bloggers make.\" good information and so very true!", "in_reply_to_status": -1, "in_reply_to_user": 495062844, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 495062844 }}, "user": { "id": 23619202, "name": "Tiffany", "screen_name": "tiffanyeatworld", "lang": "en", "location": "United States", "create_at": date("2009-03-10"), "description": "I blog about personal travel experiences, share useful tips + inspire millennial women to live life boldly ✨ get my FREE ebook http://bit.ly/FWTRebook ✨", "followers_count": 571, "friends_count": 284, "statues_count": 15139 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301162973691904, "text": "ARRIVE'S A NEW YEAR...“NICKEL OUT of FIFTEEN CENT” from the album 'Scenic Records Presents Britney' https://t.co/miRuj8SmCD", "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": 605, "friends_count": 2004, "statues_count": 109775 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301163074404352, "text": "@sickasstiger you there rn?", "in_reply_to_status": 670292302795943937, "in_reply_to_user": 2985205252, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2985205252 }}, "user": { "id": 3231358220, "name": "Tony.", "screen_name": "t0niiii_", "lang": "en", "location": "null", "create_at": date("2015-05-30"), "description": "ahaahahhaahhahaa", "followers_count": 130, "friends_count": 142, "statues_count": 353 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301163154219010, "text": "https://t.co/b33pkYQJoN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1942817287, "name": "Dawn McNabb", "screen_name": "lavenderrose777", "lang": "en", "location": "N.Y.", "create_at": date("2013-10-06"), "description": "*~Hi! I love God, my fam., Friends, petpals & country!-God Bless the U.S.A!!-Have a RAINBOW day! I am a singer-songwriter.~*", "followers_count": 288, "friends_count": 1996, "statues_count": 2424 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colonie, NY", "id": "019e50e817947f2d", "name": "Colonie", "place_type": "city", "bounding_box": rectangle("-73.920556,42.702071 -73.73002,42.791154") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36001, "countyName": "Albany", "cityID": 3617332, "cityName": "Colonie" } }
+{ "create_at": datetime("2015-11-27T10:00:25.000Z"), "id": 670301163179409409, "text": "I need to get up and go to the mall", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 399147089, "name": "RIPCHRIS#Thurl", "screen_name": "_SelfMadeMan_", "lang": "en", "location": "somewhere high ", "create_at": date("2011-10-26"), "description": "Money and Freedom !! REAL KILLER MOVING IN SILENCE", "followers_count": 1399, "friends_count": 1103, "statues_count": 22478 }, "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-27T10:00:25.000Z"), "id": 670301163200380929, "text": "�� wth https://t.co/CPIJfwe8S8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 564106423, "name": "NAT♐️", "screen_name": "NATTSTACKS", "lang": "en", "location": "Uptown, DC Born & Raised", "create_at": date("2012-04-26"), "description": "lipstick junkie • ghana • #hu19", "followers_count": 2506, "friends_count": 885, "statues_count": 109272 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301163309240320, "text": "#werd haha https://t.co/mBa6w9445I", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "werd" }}, "user": { "id": 1014486492, "name": "Karenn", "screen_name": "_kayypn", "lang": "en", "location": "Wherever God wants me to be", "create_at": date("2012-12-15"), "description": "Philippians 4:13 No Other Name #Brooklyn #SCMGWP", "followers_count": 116, "friends_count": 95, "statues_count": 5228 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301163338747904, "text": "FLASH SALE! Wolf Moon, Book #1 in Hot Moon Rising, only 99 cents for 3 days. Get it now! https://t.co/W8kDbBKauF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 6453572, "name": "Desiree Holt", "screen_name": "desireeholt", "lang": "en", "location": "Texas", "create_at": date("2007-05-30"), "description": "Loves cowboys, hot alpha males, political thrillers, pizza", "followers_count": 4533, "friends_count": 1568, "statues_count": 29405 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301163510611968, "text": "Check out our #listing in #LaurelHill #FL https://t.co/ca6ChO8WQw #realestate #realtor https://t.co/1XvbMrriJx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.554327,30.854615"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "LaurelHill", "FL", "realestate", "realtor" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6594 }, "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": 12091, "countyName": "Okaloosa" } }
+{ "create_at": datetime("2015-11-27T10:00:25.000Z"), "id": 670301163514929153, "text": "better off by myself ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 489076539, "name": "❣", "screen_name": "_killaD", "lang": "en", "location": "null", "create_at": date("2012-02-10"), "description": "sc// deajahxo // UMES", "followers_count": 6031, "friends_count": 3390, "statues_count": 62382 }, "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-27T10:00:25.000Z"), "id": 670301163577876480, "text": "I don't know what I'm doing wrong. I swear my circle gets smaller and smaller...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 169746942, "name": "Jennifer M Watson", "screen_name": "HenniferWatson", "lang": "en", "location": "Fort Worth, Texas", "create_at": date("2010-07-22"), "description": "Jenn. 22. #girlsthatclimb Kickass single mom with plans to take over the world with my main man, Rylan Cole.", "followers_count": 117, "friends_count": 327, "statues_count": 3884 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301163921674240, "text": "Showers this afternoon, high 67 (19 C). Low 60 (16 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26433972, "name": "Memphis Weather", "screen_name": "MemphisWX", "lang": "en", "location": "Memphis, TN", "create_at": date("2009-03-24"), "description": "Forecasts and advisories for Memphis, TN", "followers_count": 1686, "friends_count": 88, "statues_count": 7631 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301163946799104, "text": "It's gonna be raining all day, and I suck at driving in it. GREAT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 365851179, "name": "ℳ ❁", "screen_name": "mirandaaamarie", "lang": "en", "location": "Midland, TX maybe @ a bar", "create_at": date("2011-08-31"), "description": "laters, baby", "followers_count": 630, "friends_count": 493, "statues_count": 9776 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301164005687297, "text": "@emxmess happy birthday perfect ❤️", "in_reply_to_status": -1, "in_reply_to_user": 2418568824, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2418568824 }}, "user": { "id": 2286100502, "name": "Maddy", "screen_name": "maddy_dieter7", "lang": "en", "location": "@furman_zach ❤️", "create_at": date("2014-01-10"), "description": "pnhs varsity volleyball || spvb || #TrippStrong|| I ❤️ @lillyjohnsonnn", "followers_count": 899, "friends_count": 692, "statues_count": 8037 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301164139913216, "text": "there's nothing quite as surreal as realizing it's your last time attending a home high school football game as a high schooler...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3440871023, "name": "Quinn Collins", "screen_name": "cquinnyc19", "lang": "en", "location": "Tennessee, USA", "create_at": date("2015-08-25"), "description": "i love my God. I love to travel. I love the vols. and i am lucky to be in love with my best friend @Knoah_Rainy", "followers_count": 236, "friends_count": 339, "statues_count": 984 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301164357840896, "text": "Do it for the children ⚫️ https://t.co/3DaPiRYtH3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2432822383, "name": "camren hron", "screen_name": "camrenhron98", "lang": "en", "location": "U.S.A. ", "create_at": date("2014-04-07"), "description": "⚪⚫️️Made in America 8-11-13❤️", "followers_count": 501, "friends_count": 490, "statues_count": 2921 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301164362006528, "text": "Damn Paola just put me on Henny with honey", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 541890448, "name": "Jo$ey", "screen_name": "joseyy_theG", "lang": "en", "location": "null", "create_at": date("2012-03-31"), "description": "Manager: @NicoooGtrrz", "followers_count": 1201, "friends_count": 627, "statues_count": 69461 }, "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-27T10:00:25.000Z"), "id": 670301164467015680, "text": "Even as a Michigander/Michiginian, this is awfully funny... https://t.co/SqvHzRbWFI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155466579, "name": "Nick Ponton", "screen_name": "nick_ponton", "lang": "en", "location": "Kalamazoo, MI", "create_at": date("2010-06-13"), "description": "News photographer, freelance camera op, blogger, web designer. Michigan transplant from Illinois", "followers_count": 885, "friends_count": 1026, "statues_count": 13866 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301164605435904, "text": "#LOLpitt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "LOLpitt" }}, "user": { "id": 595722578, "name": "Tim Buckley", "screen_name": "TimBuckleyWX", "lang": "en", "location": "Greensboro, NC", "create_at": date("2012-05-31"), "description": "WFMY Meteorologist (CBM). Wx geek, beach bum, sports fanatic, Penn Stater for life. Upstate NY transplant. Opinions are mine; all mine! #triad #ncwx #wsnc #gso", "followers_count": 3281, "friends_count": 1664, "statues_count": 26911 }, "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("2015-11-27T10:00:25.000Z"), "id": 670301164848705536, "text": "https://t.co/fPrh0UtaJT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2614318089, "name": "Nqg", "screen_name": "nqg0214", "lang": "en", "location": "null", "create_at": date("2014-06-16"), "description": "♒ aquarius♒", "followers_count": 52, "friends_count": 116, "statues_count": 694 }, "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-27T10:00:26.000Z"), "id": 670301165121363968, "text": "Just endorsed my checks with eyeliner. Never felt so fabulous", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 315339279, "name": "Keeley Murphy", "screen_name": "_TeKeela", "lang": "en", "location": "Maryland", "create_at": date("2011-06-11"), "description": "big butt, bigger heart", "followers_count": 1031, "friends_count": 464, "statues_count": 15770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manchester, MD", "id": "f9815b3ea9abeecf", "name": "Manchester", "place_type": "city", "bounding_box": rectangle("-76.903644,39.636327 -76.865473,39.696654") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24013, "countyName": "Carroll", "cityID": 2449950, "cityName": "Manchester" } }
+{ "create_at": datetime("2015-11-27T10:00:26.000Z"), "id": 670301165251260417, "text": "Really just dropped a 34 track mixtape", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48494511, "name": "Marcus Q", "screen_name": "marcusq22", "lang": "en", "location": "South Florida", "create_at": date("2009-06-18"), "description": "F.$.S.❤ FAU", "followers_count": 466, "friends_count": 324, "statues_count": 15066 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nambe, NM", "id": "00c95cc02668a2d8", "name": "Nambe", "place_type": "city", "bounding_box": rectangle("-105.990864,35.882519 -105.957864,35.910845") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35049, "countyName": "Santa Fe", "cityID": 3550930, "cityName": "Nambe" } }
+{ "create_at": datetime("2015-11-27T10:00:26.000Z"), "id": 670301165544935425, "text": "SHOUTOUT TO @MADMAXATL FOR COMING THROUGH LAST WEEK.... AT #TAKEOVATUESDAYS #LGLOUNGE #DJ #ATLANTA… https://t.co/WbwhGivT03", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.2279892,33.8191528"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TAKEOVATUESDAYS", "LGLOUNGE", "DJ", "ATLANTA" }}, "user_mentions": {{ 47538588 }}, "user": { "id": 27522436, "name": "-DJ--LAZ3R-----", "screen_name": "DJLAZ3R", "lang": "en", "location": "✈ BOSTON ✈ MIAMI ✈ ATL✈ HAITI", "create_at": date("2009-03-29"), "description": "DJ // International // Shutterpixstudio // Haitian-American // konpa // Hip-hop // Dance Hall", "followers_count": 364, "friends_count": 724, "statues_count": 9636 }, "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", "cityID": 1377652, "cityName": "Tucker" } }
+{ "create_at": datetime("2015-11-27T10:00:26.000Z"), "id": 670301165813407744, "text": "@ATTCares @TMobile is offering better service/new devices for same price you're charging us. A 50% service discount would be needed to stay.", "in_reply_to_status": 670297807736184833, "in_reply_to_user": 62643312, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 62643312, 17338082 }}, "user": { "id": 463495544, "name": "Isaiah Joo", "screen_name": "IsaiahJoo", "lang": "en", "location": "Washington, D.C.", "create_at": date("2012-01-13"), "description": "Digital Diplomacy at @StateDept's @IIPState. Ultramarathon Runner. \nICN ✈ORD✈DCA", "followers_count": 547, "friends_count": 1088, "statues_count": 913 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington Heights, IL", "id": "cdf74b55fc0d4600", "name": "Arlington Heights", "place_type": "city", "bounding_box": rectangle("-88.026334,42.028374 -87.944618,42.162938") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1702154, "cityName": "Arlington Heights" } }
+{ "create_at": datetime("2015-11-27T10:00:26.000Z"), "id": 670301166014590976, "text": "@CanesSwag954 @CAN_D_MAN I respect that but you are one person why 8 seats???", "in_reply_to_status": 670300644763389952, "in_reply_to_user": 28575414, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28575414, 30480705 }}, "user": { "id": 200129470, "name": "The Fake Tom Jicha", "screen_name": "FakeTomJicha", "lang": "en", "location": "Bocer", "create_at": date("2010-10-08"), "description": "Bald and Stupid #neilrogers #neilgod", "followers_count": 829, "friends_count": 2000, "statues_count": 20480 }, "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("2015-11-27T10:00:26.000Z"), "id": 670301166278848512, "text": "@Jenna_Marbles i notice that you have red hair you kind of look like akuma the super street fighter 2 turbo character from the arcade game", "in_reply_to_status": -1, "in_reply_to_user": 66699013, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 66699013 }}, "user": { "id": 2264998873, "name": "suniga.christopher", "screen_name": "SunigaSan", "lang": "en", "location": "San Antonio Texas ", "create_at": date("2013-12-27"), "description": "null", "followers_count": 820, "friends_count": 932, "statues_count": 124 }, "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-27T10:00:26.000Z"), "id": 670301166404829188, "text": "Hurrah! (@ Armadillo Post Holes Fence Co. in Dallas, TX) https://t.co/yvcezZ3JRs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.92598963,32.67651599"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 75568616, "name": "Johnny Baker Jr. MLA", "screen_name": "profpepper", "lang": "en", "location": "Gumption Junction, TX", "create_at": date("2009-09-19"), "description": "Prov. 17:22 ♡ Is. 40:31 ♡ You won't find anything in the middle of the road but yellow lines, Bots Dots, and dead armadillos! God bless all ya'll & Texas! Ahaha", "followers_count": 824, "friends_count": 1159, "statues_count": 28326 }, "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("2015-11-27T10:00:26.000Z"), "id": 670301166589321216, "text": "@_EllaaB_ thank you��", "in_reply_to_status": 670298816235573248, "in_reply_to_user": 1283690868, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1283690868 }}, "user": { "id": 77111443, "name": "Birthday Girl♐️", "screen_name": "jaixo__", "lang": "en", "location": "610/215", "create_at": date("2009-09-24"), "description": "got you stuck off the realness❄️ SC: Viva_Jai", "followers_count": 2219, "friends_count": 1158, "statues_count": 73984 }, "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-27T10:00:26.000Z"), "id": 670301166643757058, "text": "@AOL \nIf EVERYONE cancels their @AOL account tomorrow at noon, eastern time, it would send a good message!!! @reider_andrea", "in_reply_to_status": 670300988805218304, "in_reply_to_user": 3038487649, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 66237835, 66237835, 3038487649 }}, "user": { "id": 3038487649, "name": "Andrea Reider", "screen_name": "reider_andrea", "lang": "en", "location": "West Hollywood, California", "create_at": date("2015-02-23"), "description": "I'm a visual artist. I love color, light, and all living things (to the best of my ability!).", "followers_count": 1205, "friends_count": 2241, "statues_count": 5928 }, "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("2015-11-27T10:00:26.000Z"), "id": 670301166731960321, "text": "I'm at @Publix in Orlando, FL https://t.co/4x5VlVZtQH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.44286,28.42645"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14784550 }}, "user": { "id": 379062222, "name": "Derrick Miller", "screen_name": "millerd87", "lang": "en", "location": "null", "create_at": date("2011-09-24"), "description": "null", "followers_count": 920, "friends_count": 2042, "statues_count": 19478 }, "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("2015-11-27T10:00:26.000Z"), "id": 670301166849417216, "text": "@macdac a gdy nie mam korzenia to co mam zrobić :)", "in_reply_to_status": 670300943821447168, "in_reply_to_user": 14337110, "favorite_count": 0, "retweet_count": 0, "lang": "pl", "is_retweet": false, "user_mentions": {{ 14337110 }}, "user": { "id": 1005144679, "name": "Karolina R.", "screen_name": "karolinakusmier", "lang": "pl", "location": "Haarlem/NL/ teraz gdzieś w USA", "create_at": date("2012-12-11"), "description": "Zwykła dziewczyna - matematyk, psycholog", "followers_count": 6463, "friends_count": 6421, "statues_count": 105059 }, "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-27T10:00:26.000Z"), "id": 670301166945742848, "text": "1. #BlackFriday\n2. #WhyINeedYouIn5Words\n3. #HelloStyleChat\n4. #WeDayAtlantic\n5. #WWMO\n\n2015/11/27 11:54 CST #trndnl https://t.co/SHjd3tkINE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.309,56.9547"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "BlackFriday", "WhyINeedYouIn5Words", "HelloStyleChat", "WeDayAtlantic", "WWMO", "trndnl" }}, "user": { "id": 1270239397, "name": "Trendinalia Canada", "screen_name": "trendinaliaCA", "lang": "es", "location": "Canada", "create_at": date("2013-03-15"), "description": "Information about Twitter's Trending Topics in Canada", "followers_count": 1156, "friends_count": 7, "statues_count": 239796 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Division No. 23, Unorganized, Manitoba", "id": "654fd5617c7361d0", "name": "Division No. 23, Unorganized", "place_type": "city", "bounding_box": rectangle("-102.009382,55.158501 -88.989166,60.000283") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2015-11-27T10:00:26.000Z"), "id": 670301167080112128, "text": "Every Friday is black Friday for me ✌ https://t.co/qzfMcYBE3X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2260560157, "name": "Jingyu Lin", "screen_name": "jingyulin_", "lang": "en", "location": "Chicago, IL", "create_at": date("2013-12-24"), "description": "photographing since 2010.", "followers_count": 218, "friends_count": 219, "statues_count": 2995 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elk Grove Village, IL", "id": "227d358f6ed8d29e", "name": "Elk Grove Village", "place_type": "city", "bounding_box": rectangle("-88.0605,41.983375 -87.935116,42.040488") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1723256, "cityName": "Elk Grove Village" } }
+{ "create_at": datetime("2015-11-27T10:00:26.000Z"), "id": 670301167256080384, "text": "Oh my goodness - my dad is NOT a handy man. #homedepotprobz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "homedepotprobz" }}, "user": { "id": 3194668218, "name": "Abigail Montañez", "screen_name": "AMonty2590", "lang": "en", "location": "San Antonio, TX", "create_at": date("2015-05-13"), "description": "25 | Puerto Rican & Mexican | M.A. Admin | Serve + Worship @centralcogsa | Choose Joy | Psalm 16:11", "followers_count": 94, "friends_count": 165, "statues_count": 377 }, "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-27T10:00:26.000Z"), "id": 670301167767953409, "text": "Tell ya jus the way to hit it when you tryna push up 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": 2956902309, "name": "✨Gawwdess✨", "screen_name": "gxlden_myaa", "lang": "en", "location": "null", "create_at": date("2015-01-02"), "description": "null", "followers_count": 225, "friends_count": 329, "statues_count": 4899 }, "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("2015-11-27T10:00:26.000Z"), "id": 670301168124432384, "text": "Does Santa check Twitter for Christmas lists? Asking for a friend", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 183755091, "name": "Alexander Kendall", "screen_name": "Alex7MIH", "lang": "en", "location": "Chicago, IL.", "create_at": date("2010-08-27"), "description": "Lets get wrecked on pop-tarts and sex. 1/3 of @7MIHband | IG: alexinheaven. Pre-order the new EP Side Effects on iTunes or http://7MIH.COM now! •XO•", "followers_count": 28229, "friends_count": 203, "statues_count": 25046 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mountainside, NJ", "id": "c76d79c902d9ffbd", "name": "Mountainside", "place_type": "city", "bounding_box": rectangle("-74.387516,40.664275 -74.331282,40.700078") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3448510, "cityName": "Mountainside" } }
+{ "create_at": datetime("2015-11-27T10:00:26.000Z"), "id": 670301168128630784, "text": "#tdecustadium this afternoon and #NRG tonite @ TDECU Stadium https://t.co/ALKXNJZmW8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.34916667,29.72194444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "tdecustadium", "NRG" }}, "user": { "id": 62968377, "name": "HTOWNBLKGOLFR", "screen_name": "HTOWNBLKGOLFer", "lang": "en", "location": "N 29°31' 0'' / W 95°7' 0''", "create_at": date("2009-08-04"), "description": "Golf , Cigars & Beautiful Smiles are my thing . AKA SMILE FREAK . #Sikarboyzclub member.", "followers_count": 534, "friends_count": 1319, "statues_count": 19494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-27T10:00:26.000Z"), "id": 670301168329818113, "text": "my hair looked different 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": 309876251, "name": "dharmaaray", "screen_name": "dharmaaray", "lang": "en", "location": "null", "create_at": date("2011-06-02"), "description": "null", "followers_count": 974, "friends_count": 601, "statues_count": 25495 }, "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("2015-11-27T10:00:26.000Z"), "id": 670301168359247872, "text": "@Dylan9023 @Davidhnaz Let the Muslim countries take them in rather than moving them thousands of miles away. Saudi Arabia has tons of room.", "in_reply_to_status": 670213840848551936, "in_reply_to_user": 1959526170, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1959526170, 250162737 }}, "user": { "id": 1158014083, "name": "Steve Elliott", "screen_name": "SteveSanAntone", "lang": "en", "location": "San Antonio, TX", "create_at": date("2013-02-07"), "description": "I love my life! Have the best family-a beautiful wife & three gorgeous daughters! I'm an Air Force editor & an Air Force veteran as well as a bass player!", "followers_count": 606, "friends_count": 1113, "statues_count": 2324 }, "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-27T10:00:26.000Z"), "id": 670301168430481408, "text": "good morning, what’s for breakfast? i mean, other than social injustice and capitalist greed?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.05998537,37.95719035"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15291349, "name": "jenny_klug", "screen_name": "jenny_klug", "lang": "en", "location": "san francisco (-ish)", "create_at": date("2008-07-01"), "description": "(latte art and bicycle parts.) \n\nbicycle nerd. \ndrummer por vida.\nworker bee.\npriced out to suburbs.\nreally hates leafblowers.\nit's pronounced klūg.", "followers_count": 255, "friends_count": 220, "statues_count": 12891 }, "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("2015-11-27T10:00:26.000Z"), "id": 670301168531333120, "text": "I'm at @Shallos Antique Restaurant in Indianapolis, IN https://t.co/nXj0MBvIZu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.12418652,39.63801734"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 30416615 }}, "user": { "id": 309636456, "name": "Brooke Lockett", "screen_name": "BrookeNG12", "lang": "en", "location": "Indianapolis, IN", "create_at": date("2011-06-02"), "description": "null", "followers_count": 290, "friends_count": 380, "statues_count": 6367 }, "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("2015-11-27T10:00:26.000Z"), "id": 670301168623480832, "text": "Happy Birthday @_Stephanie_Sims !! Miss you!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 954963625 }}, "user": { "id": 1516691202, "name": "☻", "screen_name": "ColbyRyann", "lang": "en", "location": "null", "create_at": date("2013-06-14"), "description": "Romans 8:18 // @jakewal25 ♡", "followers_count": 1772, "friends_count": 723, "statues_count": 11390 }, "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("2015-11-27T10:00:26.000Z"), "id": 670301168736825344, "text": "when your relatives talk about making pasta for an entire hour #italianproblems", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "italianproblems" }}, "user": { "id": 350004743, "name": "ky-leeee", "screen_name": "thekylienicole", "lang": "en", "location": "new england", "create_at": date("2011-08-06"), "description": "null", "followers_count": 305, "friends_count": 778, "statues_count": 4420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Middletown, NY", "id": "acbf38761374d732", "name": "Middletown", "place_type": "city", "bounding_box": rectangle("-74.461178,41.393907 -74.374751,41.474306") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36071, "countyName": "Orange", "cityID": 3647042, "cityName": "Middletown" } }
+{ "create_at": datetime("2015-11-27T10:00:26.000Z"), "id": 670301168766095360, "text": "6. Karina Wolfe\n7. Brunswick House\n8. Alan Kurdi\n9. Red Sox\n10. Canadian Photography Institute\n\n2015/11/27 11:54 CST https://t.co/SHjd3tkINE", "in_reply_to_status": 670301166945742848, "in_reply_to_user": 1270239397, "favorite_count": 0, "coordinate": point("-98.309,56.9547"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1270239397, "name": "Trendinalia Canada", "screen_name": "trendinaliaCA", "lang": "es", "location": "Canada", "create_at": date("2013-03-15"), "description": "Information about Twitter's Trending Topics in Canada", "followers_count": 1156, "friends_count": 7, "statues_count": 239797 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Division No. 23, Unorganized, Manitoba", "id": "654fd5617c7361d0", "name": "Division No. 23, Unorganized", "place_type": "city", "bounding_box": rectangle("-102.009382,55.158501 -88.989166,60.000283") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2015-11-27T10:00:26.000Z"), "id": 670301168929632256, "text": "I just want 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": 43641405, "name": "Sarah O'Neill", "screen_name": "sbogirly", "lang": "en", "location": "Lake Elsinore, CA", "create_at": date("2009-05-30"), "description": "Twenty-One. Happy. EMT Cert. Gym. #PatriotsNation", "followers_count": 416, "friends_count": 248, "statues_count": 4671 }, "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("2015-11-27T10:00:26.000Z"), "id": 670301169105829888, "text": "The good dinosaur ain't bad tbh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 408947330, "name": "Manny", "screen_name": "MannyROD16", "lang": "en", "location": "null", "create_at": date("2011-11-09"), "description": "null", "followers_count": 608, "friends_count": 802, "statues_count": 64285 }, "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("2015-11-27T10:00:26.000Z"), "id": 670301169235824640, "text": "Our Intergenerational Skype program with students in China is such a blessing! @CourtyardTowers #frontierculture https://t.co/DLlFjv9cp8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "frontierculture" }}, "user_mentions": {{ 3227612336 }}, "user": { "id": 3384927785, "name": "Mailani Fernandez", "screen_name": "CYTActivities", "lang": "en", "location": "Mesa, AZ", "create_at": date("2015-07-20"), "description": "Courtyard Towers Senior Living", "followers_count": 69, "friends_count": 81, "statues_count": 224 }, "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("2015-11-27T10:00:27.000Z"), "id": 670301169324040192, "text": "I want some El tap", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 585118114, "name": "ZarZar", "screen_name": "Sincerely_Nel", "lang": "en", "location": "null", "create_at": date("2012-05-19"), "description": "Toast to clichés in the dark past", "followers_count": 1272, "friends_count": 782, "statues_count": 10590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rocky Mount, NC", "id": "1701e6db682eb647", "name": "Rocky Mount", "place_type": "city", "bounding_box": rectangle("-77.907739,35.890099 -77.715568,36.054453") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37127, "countyName": "Nash", "cityID": 3757500, "cityName": "Rocky Mount" } }
+{ "create_at": datetime("2015-11-27T10:00:27.000Z"), "id": 670301169353232384, "text": "Partly cloudy this afternoon, high 71 (22 C). Low 55 (13 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119697197, "name": "Huntsville Weather", "screen_name": "HuntsvilleWX", "lang": "en", "location": "null", "create_at": date("2010-03-04"), "description": "null", "followers_count": 665, "friends_count": 65, "statues_count": 5942 }, "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("2015-11-27T10:00:27.000Z"), "id": 670301170288623616, "text": "My closest friends can put me 2nd when I put them 1st smh to all you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3327085297, "name": "Sloan Simpson", "screen_name": "SloanSimpson3", "lang": "en", "location": "null", "create_at": date("2015-08-23"), "description": "Jesus is love Jesus is Life", "followers_count": 116, "friends_count": 123, "statues_count": 75 }, "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("2015-11-27T10:00:27.000Z"), "id": 670301170410332161, "text": "Red Starbucks cup, I lift you up #proceedtocoffee (@ Starbucks in Seattle, WA) https://t.co/9kr6bWuv07 https://t.co/nMlZ7sX99Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.28070736,47.66214304"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "proceedtocoffee" }}, "user": { "id": 16828826, "name": "burndive", "screen_name": "burndive", "lang": "en", "location": "Seattle", "create_at": date("2008-10-17"), "description": "Husband, father, programmer, geek, swing dancer, gamer: Christian.", "followers_count": 104, "friends_count": 82, "statues_count": 4864 }, "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-27T10:00:27.000Z"), "id": 670301170418626560, "text": "Gotti dropped that cm8 go peep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 514658394, "name": "King of Kings™", "screen_name": "Iam_Bry22", "lang": "en", "location": "null", "create_at": date("2012-03-04"), "description": "(901) ✈ (402) . “There may be people that have more talent than you, but theres no excuse for anyone to work harder than you do.” – Derek Jeter #EK3 #TKT #DR", "followers_count": 561, "friends_count": 456, "statues_count": 15919 }, "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("2015-11-27T10:00:27.000Z"), "id": 670301170544435201, "text": "on God i dont wanna see no turkey.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 40681203, "name": "BIG Bank Hank", "screen_name": "MichaelStrauss_", "lang": "en", "location": "Trill, Texas", "create_at": date("2009-05-17"), "description": "For All Mines I'll Lay Yours; MOB", "followers_count": 614, "friends_count": 132, "statues_count": 88794 }, "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("2015-11-27T10:00:27.000Z"), "id": 670301170829692928, "text": "When you gf is recording you w/out knowing.\n#caughtofguard #naturalhair @… https://t.co/TN4MfkvdLB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.18319743,37.73671628"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "caughtofguard", "naturalhair" }}, "user": { "id": 1377857341, "name": "Daniel Leslie", "screen_name": "Cobra8030", "lang": "en", "location": "Oakland, CA", "create_at": date("2013-04-24"), "description": "Educator, Community Activist, Youth Supervisor, Aspiring Voice Actor, Art Admirer, Blogger.", "followers_count": 60, "friends_count": 311, "statues_count": 386 }, "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("2015-11-27T10:00:27.000Z"), "id": 670301171068850176, "text": "Happy birthday @amy_lynn_flynn!! Have the best day��❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 401171103 }}, "user": { "id": 601039331, "name": "Stephanie Bono", "screen_name": "bonoo999", "lang": "en", "location": "null", "create_at": date("2012-06-06"), "description": "Zeta Tau Alpha", "followers_count": 587, "friends_count": 555, "statues_count": 3715 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plum, PA", "id": "1b110d80cc29aed9", "name": "Plum", "place_type": "city", "bounding_box": rectangle("-79.820663,40.443524 -79.702189,40.544708") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4261536, "cityName": "Plum" } }
+{ "create_at": datetime("2015-11-27T10:00:27.000Z"), "id": 670301171098230784, "text": "can y'all even go to the bathroom without the other one? �� i mean damnnn ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1616503070, "name": "Ana María", "screen_name": "amclxo_", "lang": "en", "location": "South Florida ", "create_at": date("2013-07-23"), "description": "• 20 • Colombia - Costeña • HJN • #NietoStrong •", "followers_count": 177, "friends_count": 519, "statues_count": 4845 }, "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("2015-11-27T10:00:27.000Z"), "id": 670301171240701952, "text": "This über driver is trash and drives worse than me ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user": { "id": 339908672, "name": "elena", "screen_name": "DayumLilMama", "lang": "en", "location": "san diego", "create_at": date("2011-07-21"), "description": "Stay Solid . http://GuamBomb.com", "followers_count": 3179, "friends_count": 639, "statues_count": 32518 }, "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-27T10:00:27.000Z"), "id": 670301171295256577, "text": "Good morning, giving thanks for this beautiful morning. #oakland #LakeMerritt #OakD2 @ Lake Merritt,… https://t.co/8nYJoOfSXk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.25430168,37.80553634"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "oakland", "LakeMerritt", "OakD2" }}, "user": { "id": 14319867, "name": "Abel Guillen", "screen_name": "Abel_Guillen", "lang": "en", "location": "Oakland, California", "create_at": date("2008-04-06"), "description": "Oakland City Councilmember. Public education champion. Muni Finance. Policy Wonk. Go Bears! Email me about City of Oakland issues: aguillen@oaklandnet.com", "followers_count": 2037, "friends_count": 1823, "statues_count": 5288 }, "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("2015-11-27T10:00:27.000Z"), "id": 670301171647700993, "text": "Yah girls will be mad at a nigga and not talk to him no more but like his Instagram picture", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 343917668, "name": "COWBOYS 8-8", "screen_name": "UnfollowMeIdgaf", "lang": "en", "location": "null", "create_at": date("2011-07-28"), "description": "just average nigga with a small dick I can't rap", "followers_count": 20323, "friends_count": 21857, "statues_count": 195665 }, "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("2015-11-27T10:00:27.000Z"), "id": 670301171832213504, "text": "Shorties need to slow down", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 423033545, "name": "Sept14th #WDLTH", "screen_name": "Flight_Leader", "lang": "en", "location": "Glendale Heights I", "create_at": date("2011-11-27"), "description": "Just keeping it real is the real deal! We don't catch feelings we catch flight #JetLife! #WDLTH!Indiana state class of 2019!", "followers_count": 750, "friends_count": 1235, "statues_count": 14602 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glendale Heights, IL", "id": "389e765d4de59bd2", "name": "Glendale Heights", "place_type": "city", "bounding_box": rectangle("-88.101785,41.895755 -88.048564,41.943267") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1729730, "cityName": "Glendale Heights" } }
+{ "create_at": datetime("2015-11-27T10:00:27.000Z"), "id": 670301172406833152, "text": "ma pushing my buttons", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 928671691, "name": "coach j", "screen_name": "jewwmorgan", "lang": "en", "location": "Boston Red Sox ", "create_at": date("2012-11-05"), "description": "lions don't lose sleep over the opinion of sheep married to hoodie Allen since 2011", "followers_count": 351, "friends_count": 298, "statues_count": 23261 }, "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("2015-11-27T10:00:27.000Z"), "id": 670301172482359297, "text": "@pastorhavila Cubazuela. https://t.co/f47q3ReMSi", "in_reply_to_status": -1, "in_reply_to_user": 987146870, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 987146870 }}, "user": { "id": 3060489088, "name": "Ariel Enrique", "screen_name": "Arielenrique91", "lang": "es", "location": "Florida, USA", "create_at": date("2015-02-24"), "description": "Venezolano.", "followers_count": 33, "friends_count": 72, "statues_count": 911 }, "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("2015-11-27T10:00:27.000Z"), "id": 670301172603990017, "text": "Sometimes I think my bald spot is gonna go away and then I'm like eh probably not", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1016273040, "name": "franklin harrell", "screen_name": "FranklinHarrell", "lang": "en", "location": "null", "create_at": date("2012-12-16"), "description": "Coed Elite • 2015 World Champion", "followers_count": 753, "friends_count": 251, "statues_count": 4860 }, "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("2015-11-27T10:00:27.000Z"), "id": 670301172654325761, "text": "@TWTWsports trade Kluber for Kemp? You are a total idiot.", "in_reply_to_status": 670280411944247297, "in_reply_to_user": 3042386780, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3042386780 }}, "user": { "id": 230313929, "name": "Will Hazen", "screen_name": "Will_Hazen16", "lang": "en", "location": "Medina to Cleveland. ", "create_at": date("2010-12-24"), "description": "thank god for the Cavs. #AllinCLE 4/20/14 CLE & OSU", "followers_count": 337, "friends_count": 219, "statues_count": 11565 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wooster, OH", "id": "459a2284d088a7dd", "name": "Wooster", "place_type": "city", "bounding_box": rectangle("-81.990959,40.758969 -81.877989,40.872411") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39169, "countyName": "Wayne", "cityID": 3986548, "cityName": "Wooster" } }
+{ "create_at": datetime("2015-11-27T10:00:27.000Z"), "id": 670301172675145730, "text": "I'm 0-3 in stores right now. I wanna cry.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3279066494, "name": "sarah emerson", "screen_name": "sarahhemersonn", "lang": "en", "location": "sftx", "create_at": date("2015-07-13"), "description": "•Psalm 37:5•", "followers_count": 416, "friends_count": 528, "statues_count": 1783 }, "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("2015-11-27T10:00:27.000Z"), "id": 670301172771725312, "text": "Let's see if I 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": 564105911, "name": "VLO", "screen_name": "notyoaverage1", "lang": "en", "location": "null", "create_at": date("2012-04-26"), "description": "have you ever heard of me? well my names Viktoria and yes its nice to meet me!! ACT LIKE YOU KNOW", "followers_count": 1105, "friends_count": 1020, "statues_count": 31039 }, "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-27T10:00:27.000Z"), "id": 670301172947795968, "text": "@Kaileyybrook hey you bet old friend!��", "in_reply_to_status": 670301086004023296, "in_reply_to_user": 741661832, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 741661832 }}, "user": { "id": 1662198720, "name": "♕Sensei權Taylor♕", "screen_name": "Kwaunshaw", "lang": "en", "location": "Wichita Falls, TX", "create_at": date("2013-08-11"), "description": "Ninja Art | MSU Mustangs DB", "followers_count": 1024, "friends_count": 1121, "statues_count": 12513 }, "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("2015-11-27T10:00:27.000Z"), "id": 670301173170073600, "text": "@1YoungHaitian I'll think about it .", "in_reply_to_status": 670300656100442112, "in_reply_to_user": 920311940, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 920311940 }}, "user": { "id": 302897020, "name": "Rayglo☯", "screen_name": "rvygo", "lang": "en", "location": "null", "create_at": date("2011-05-21"), "description": "love yourself", "followers_count": 1921, "friends_count": 664, "statues_count": 53399 }, "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("2015-11-27T10:00:27.000Z"), "id": 670301173287665664, "text": "@general_heinz drugs", "in_reply_to_status": 670295149264564225, "in_reply_to_user": 2614454491, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2614454491 }}, "user": { "id": 423058517, "name": "makin'", "screen_name": "Youngxbloodx", "lang": "en", "location": "269 ↔️ 586", "create_at": date("2011-11-27"), "description": "moves", "followers_count": 267, "friends_count": 114, "statues_count": 13037 }, "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("2015-11-27T10:00:27.000Z"), "id": 670301173413511168, "text": "BLACK FRIDAY SPECIAL DEAL!!! ONE DAY ONLY!!!\nAfter gobbling up all that turkey now its time to nom… https://t.co/lZj2rwI5OC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.3002472,34.090271"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48029424, "name": "Chaparral Mtrsprts", "screen_name": "ChaparralMoto", "lang": "en", "location": "San Bernardino, CA", "create_at": date("2009-06-17"), "description": "Motorcycle, ATV dealership", "followers_count": 1744, "friends_count": 287, "statues_count": 3380 }, "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("2015-11-27T10:00:28.000Z"), "id": 670301173988093952, "text": "Love getting randomly sad to the point I can't stop crying", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1365400717, "name": "MOM", "screen_name": "Meghannicole217", "lang": "en", "location": "Minooka", "create_at": date("2013-04-19"), "description": "|MCHS 17| PPC School News Editor| TSA Reporter| Minooka Cross Country| 2/19/13| Photographer| Beatlemanic|instagram Blenck182|", "followers_count": 284, "friends_count": 785, "statues_count": 5009 }, "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("2015-11-27T10:00:28.000Z"), "id": 670301174038470657, "text": "@BankofAmerica and #chicagotraditions https://t.co/RUEoHU3RZe", "in_reply_to_status": -1, "in_reply_to_user": 204881628, "favorite_count": 0, "coordinate": point("-87.68660914,41.87027254"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "chicagotraditions" }}, "user_mentions": {{ 204881628 }}, "user": { "id": 2460463651, "name": "Julie Chavez", "screen_name": "jchax0x0", "lang": "en", "location": "null", "create_at": date("2014-04-23"), "description": "Employed by Bank of America, so comments and views are solely my own.", "followers_count": 67, "friends_count": 67, "statues_count": 142 }, "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-27T10:00:28.000Z"), "id": 670301174399012864, "text": "Partly cloudy this afternoon, high 74 (23 C). Low 58 (14 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119698549, "name": "Mobile Weather", "screen_name": "MobileWX", "lang": "en", "location": "Mobile, Al", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Mobile, Al", "followers_count": 387, "friends_count": 68, "statues_count": 6029 }, "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("2015-11-27T10:00:28.000Z"), "id": 670301174508199936, "text": "@rewhite54g hey how was Ur thanksgiving", "in_reply_to_status": 670033897984532480, "in_reply_to_user": 2601960398, "favorite_count": 0, "coordinate": point("-74.3651626,40.7394434"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2601960398 }}, "user": { "id": 70550778, "name": "Freakiest Virgo", "screen_name": "Quasonwy", "lang": "en", "location": "Newark", "create_at": date("2009-08-31"), "description": "the best ♍ u will ever meet", "followers_count": 718, "friends_count": 1994, "statues_count": 5940 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Millburn, NJ", "id": "1935ab8020a2553b", "name": "Millburn", "place_type": "city", "bounding_box": rectangle("-74.372452,40.713308 -74.284841,40.76105") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3467320, "cityName": "Short Hills" } }
+{ "create_at": datetime("2015-11-27T10:00:28.000Z"), "id": 670301174575165440, "text": "See a virtual tour of our listing on 640 Long Drive #Crestview #FL https://t.co/qJ7zfNZSXF #realestate https://t.co/H2W5zIlti3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.558716,30.768387"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Crestview", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6595 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crestview, FL", "id": "32bb9ace0723336d", "name": "Crestview", "place_type": "city", "bounding_box": rectangle("-86.635021,30.695757 -86.534517,30.805126") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1215475, "cityName": "Crestview" } }
+{ "create_at": datetime("2015-11-27T10:00:28.000Z"), "id": 670301174831194113, "text": "������������ https://t.co/gx9HHxyuv7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 176216530, "name": "Reece", "screen_name": "Reecyy", "lang": "en", "location": "Trinidad&Tobago ♡/Melbourne FL", "create_at": date("2010-08-08"), "description": "20. Florida Tech '2018. Certified Private Pilot ✈️. Working on my Instrument Rating & Commercial Pilot License. #MUFC", "followers_count": 864, "friends_count": 407, "statues_count": 52894 }, "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("2015-11-27T10:00:28.000Z"), "id": 670301174852165632, "text": "@taliamarie77 @GeorgeTroester @lollygagblare @Coicele @cigneutron hope you had a great Thanksgiving", "in_reply_to_status": 670253745322725376, "in_reply_to_user": 35812119, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 35812119, 124417575, 317317459, 2197828584, 213982606 }}, "user": { "id": 483093916, "name": "Yoga Butterfly", "screen_name": "YogaButterfly_", "lang": "en", "location": "Western MA", "create_at": date("2012-02-04"), "description": "#love #art #kindle #vaping #yoga #i2 #NSFW #ADD #autism #aspergers (son) #ASL #blogger online 2 decades!", "followers_count": 9624, "friends_count": 9725, "statues_count": 60540 }, "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("2015-11-27T10:00:28.000Z"), "id": 670301175128920064, "text": "@DatFangirlKait He quickly nodded, obviously excited. \"Yeah! Let's get to it!\"", "in_reply_to_status": 670299950169915392, "in_reply_to_user": 762319093, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 762319093 }}, "user": { "id": 1557093954, "name": "Jay Lucker", "screen_name": "Jay_TotalDrama", "lang": "en", "location": "A place to call my own", "create_at": date("2013-06-29"), "description": "They say love is forever. Your forever is all that I need. @DatFangirlKait ❤️", "followers_count": 2619, "friends_count": 311, "statues_count": 33798 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waukegan, IL", "id": "b819c5d90b780b57", "name": "Waukegan", "place_type": "city", "bounding_box": rectangle("-87.96368,42.305624 -87.802772,42.431936") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1779293, "cityName": "Waukegan" } }
+{ "create_at": datetime("2015-11-27T10:00:28.000Z"), "id": 670301175368056832, "text": "I'm at @ToysRUs in Des Moines, IA https://t.co/qQDkOMaAVm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.59961033,41.52501351"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17308994 }}, "user": { "id": 58682595, "name": "Michael Bird", "screen_name": "IowaBirdman", "lang": "en", "location": "Des Moines, Iowa", "create_at": date("2009-07-20"), "description": "The Birdman is loose on Twitter and will be commenting on movies, games, tv, and life in Iowa... and I'm CEO @Spindustry", "followers_count": 752, "friends_count": 1153, "statues_count": 5593 }, "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("2015-11-27T10:00:28.000Z"), "id": 670301175732834304, "text": "@deuce_papi2 We don't hoop with anybody bro", "in_reply_to_status": 670300629731004422, "in_reply_to_user": 330465376, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 330465376 }}, "user": { "id": 219360791, "name": "Dec.10Th", "screen_name": "LilDave2600", "lang": "en", "location": "null", "create_at": date("2010-11-24"), "description": "Loading... #GRINDForPops ❤ #FreeTae", "followers_count": 1426, "friends_count": 395, "statues_count": 97813 }, "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("2015-11-27T10:00:28.000Z"), "id": 670301176324313088, "text": "@bethanyky97 too warm! haha ��", "in_reply_to_status": 670300908488687616, "in_reply_to_user": 1377043273, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1377043273 }}, "user": { "id": 65412295, "name": "Justin Salzman", "screen_name": "JKSalzman", "lang": "en", "location": "Fayetteville, NC", "create_at": date("2009-08-13"), "description": "You have to remember, fear is not real. It is a product of the thoughts you create. Don't misunderstand me. Danger is very real. Fear is a choice.-Will Smith", "followers_count": 89, "friends_count": 161, "statues_count": 3662 }, "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("2015-11-27T10:00:28.000Z"), "id": 670301176697491456, "text": "Freezing my booty off at the husker game ��❤️❄️ #GBR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GBR" }}, "user": { "id": 1679573347, "name": "Ellie Purple", "screen_name": "ellierosepink", "lang": "en", "location": "nebraska ", "create_at": date("2013-08-17"), "description": "null", "followers_count": 302, "friends_count": 207, "statues_count": 5376 }, "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("2015-11-27T10:00:28.000Z"), "id": 670301176743641088, "text": "Choose Bigtop Collectibles for all your #BlackFriday shopping! We've got something for everyone and EVERYTHING IN OUR STORE IS ON SALE!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BlackFriday" }}, "user": { "id": 1077794400, "name": "Bigtop Collectibles", "screen_name": "bigtop1138", "lang": "en", "location": "12650 W 64th Arvada CO 80004", "create_at": date("2013-01-10"), "description": "We truly do have the largest selection of Star Wars Collectibles in the State of Colorado. (303) 431-3757 http://bit.ly/BigtopEbay | http://bit.ly/BigtopAmazon", "followers_count": 1597, "friends_count": 1133, "statues_count": 1578 }, "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("2015-11-27T10:00:28.000Z"), "id": 670301176764739584, "text": "never pictured myself being with somebody for this long. so blessed. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 632229324, "name": "mad", "screen_name": "madisonlinda_", "lang": "en", "location": "05.06.14", "create_at": date("2012-07-10"), "description": "in love with my bestfriend jbs ❤️", "followers_count": 248, "friends_count": 403, "statues_count": 7049 }, "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("2015-11-27T10:00:28.000Z"), "id": 670301176919814144, "text": "Road Ranger is the real MVP!!! https://t.co/4Uosdej07r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2148026268, "name": "Luis", "screen_name": "CowboycamoLuis", "lang": "en", "location": "Mathis", "create_at": date("2013-10-21"), "description": "Forever Blue#FFA Psalm 23: 1,2-The Lord is my shepeard; I shall not want. He makes me lie down in green pastures. He leads beside still waters.", "followers_count": 144, "friends_count": 328, "statues_count": 1037 }, "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.112926") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48409, "countyName": "San Patricio", "cityID": 4847040, "cityName": "Mathis" } }
+{ "create_at": datetime("2015-11-27T10:00:28.000Z"), "id": 670301176965914624, "text": "Showers/wind this afternoon, high 41 (5 C). Low 37 (3 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119751854, "name": "Chicago Weather", "screen_name": "Chicago_WX", "lang": "en", "location": "Chicago, IL", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Chicago, IL", "followers_count": 1056, "friends_count": 68, "statues_count": 6311 }, "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-27T10:00:28.000Z"), "id": 670301177167376385, "text": "Nana going to have to hand me a one ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 435415983, "name": "D", "screen_name": "Fly_kidd_11", "lang": "en", "location": "The 7 . Pennsylvania ", "create_at": date("2011-12-12"), "description": "#SHIPU⛵️ | #OTG | #RIPBIGBRO", "followers_count": 1919, "friends_count": 957, "statues_count": 45255 }, "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("2015-11-27T10:00:28.000Z"), "id": 670301177188257792, "text": "@jimgroom @shauser whoa. Tarantino-esque", "in_reply_to_status": 670300087982153728, "in_reply_to_user": 3362981, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 3362981, 6088332 }}, "user": { "id": 158207707, "name": "Amy Burvall", "screen_name": "amyburvall", "lang": "en", "location": "Oahu, Hi and San Francisco, CA", "create_at": date("2010-06-21"), "description": "The Cloud is Our Campfire. Dare to Share. I make whimsy happen. VP Academic Affairs @edgemakers History Music Vids @historyteacherz, https://t.co/O7cLIBAjK8", "followers_count": 5659, "friends_count": 3429, "statues_count": 41792 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Heeia, HI", "id": "1dfbbc2b9215bdec", "name": "Heeia", "place_type": "city", "bounding_box": rectangle("-157.826522,21.408367 -157.792604,21.438983") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1513900, "cityName": "Heeia" } }
+{ "create_at": datetime("2015-11-27T10:00:28.000Z"), "id": 670301177247096832, "text": "@gpdigs49 thanks GP ��☺️ miss uu", "in_reply_to_status": 670283663595216897, "in_reply_to_user": 432100732, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 432100732 }}, "user": { "id": 1452517638, "name": "Nick Chillbae", "screen_name": "N_Kilbane30", "lang": "en", "location": "CLE | 420 11 |", "create_at": date("2013-05-23"), "description": "Purveyor of Island Lifestyles... call me goat Ignatius Football '17 Mack Pack #32", "followers_count": 446, "friends_count": 325, "statues_count": 2018 }, "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("2015-11-27T10:00:28.000Z"), "id": 670301177276293120, "text": "literally shopped till I dropped", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2339211128, "name": "gab", "screen_name": "gabbbyyyy___", "lang": "en", "location": "null", "create_at": date("2014-02-11"), "description": "pap south", "followers_count": 435, "friends_count": 405, "statues_count": 1552 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Papillion, NE", "id": "43212288cc83156e", "name": "Papillion", "place_type": "city", "bounding_box": rectangle("-96.078103,41.117927 -95.962936,41.183482") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31153, "countyName": "Sarpy", "cityID": 3138295, "cityName": "Papillion" } }
+{ "create_at": datetime("2015-11-27T10:00:28.000Z"), "id": 670301177444069376, "text": "First Game Tomorrow I Have To 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": 3897429432, "name": "Alice", "screen_name": "AlisonNunez27", "lang": "en", "location": "Pasadena", "create_at": date("2015-10-14"), "description": "Nothing Last Forever.", "followers_count": 131, "friends_count": 117, "statues_count": 2062 }, "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("2015-11-27T10:00:28.000Z"), "id": 670301177460867072, "text": "������ https://t.co/Nyms0XF3Fw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 43225622, "name": "Alex Polyzos", "screen_name": "alexpoly94", "lang": "en", "location": "Yay Area, CA", "create_at": date("2009-05-28"), "description": "#RaiderNation #DubNation", "followers_count": 312, "friends_count": 324, "statues_count": 17561 }, "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("2015-11-27T10:00:28.000Z"), "id": 670301177574203392, "text": "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": 587755694, "name": "❥ ❥ǰ∊ssأʗɑ", "screen_name": "jessicalynn5777", "lang": "en", "location": "☾☮ new york ", "create_at": date("2012-05-22"), "description": "❁ insta: jessicalynn57775.", "followers_count": 414, "friends_count": 553, "statues_count": 13533 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Northport, NY", "id": "9530ea3d29844638", "name": "East Northport", "place_type": "city", "bounding_box": rectangle("-73.35523,40.862419 -73.286426,40.900011") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3622612, "cityName": "East Northport" } }
+{ "create_at": datetime("2015-11-27T10:00:29.000Z"), "id": 670301177683165184, "text": "@BrennaStewart3 @OhMrWonka @itsWillyFerrell @mcdevitt_morgan hahahahahahaha", "in_reply_to_status": 670291828768268288, "in_reply_to_user": 1393793918, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 1393793918, 502531529, 306025835, 3105863804 }}, "user": { "id": 391516924, "name": "TGOD", "screen_name": "TaylorGraham94", "lang": "en", "location": "Scottsdale, AZ", "create_at": date("2011-10-15"), "description": "ASU || think like a proton, always positive", "followers_count": 445, "friends_count": 410, "statues_count": 2799 }, "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("2015-11-27T10:00:29.000Z"), "id": 670301177750421505, "text": "@anthonyVslater to me, the bigger takeaway from that sound byte is his snarling remark on a ring less 4 years in OKC. Sickening.", "in_reply_to_status": 670300004175605760, "in_reply_to_user": 77577780, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 77577780 }}, "user": { "id": 257238016, "name": "Brad Tate", "screen_name": "bradleyjtate", "lang": "en", "location": "Atlanta, Georgia", "create_at": date("2011-02-24"), "description": "Nobody ever went broke underestimating the taste of the American public.", "followers_count": 135, "friends_count": 614, "statues_count": 5625 }, "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("2015-11-27T10:00:29.000Z"), "id": 670301177846755328, "text": "@brittanypinet_ devin didn't want to stop ��", "in_reply_to_status": 670296331743727616, "in_reply_to_user": 3256465247, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3256465247 }}, "user": { "id": 633911981, "name": "Skye", "screen_name": "kendrick_xoxo", "lang": "en", "location": "Biddeford, ME", "create_at": date("2012-07-12"), "description": "@ me", "followers_count": 653, "friends_count": 378, "statues_count": 18491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scarborough, ME", "id": "f39a803949cd2255", "name": "Scarborough", "place_type": "city", "bounding_box": rectangle("-70.401214,43.542566 -70.270921,43.636249") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23005, "countyName": "Cumberland", "cityID": 2366110, "cityName": "Scarborough" } }
+{ "create_at": datetime("2015-11-27T10:00:29.000Z"), "id": 670301178434068480, "text": "@tiriabrill ya smart, ya loyal...", "in_reply_to_status": 670267920942039040, "in_reply_to_user": 4032059773, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4032059773 }}, "user": { "id": 1221349628, "name": "THE CHASERS DC4", "screen_name": "_Rk_215", "lang": "en", "location": "215", "create_at": date("2013-02-26"), "description": "Philadelphia", "followers_count": 169, "friends_count": 672, "statues_count": 6761 }, "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-27T10:00:29.000Z"), "id": 670301178593316864, "text": "I miss @nickolson83 a lot :(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1385650650 }}, "user": { "id": 590213902, "name": "bai", "screen_name": "bjoyna", "lang": "en", "location": "Gilbert, AZ", "create_at": date("2012-05-25"), "description": "null", "followers_count": 1328, "friends_count": 837, "statues_count": 34852 }, "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("2015-11-27T10:00:29.000Z"), "id": 670301178698199040, "text": "The breakup ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1974002876, "name": "✨", "screen_name": "Muni563", "lang": "en", "location": "Oregon, USA", "create_at": date("2013-10-19"), "description": "@chrisbrown ❤️", "followers_count": 3736, "friends_count": 2541, "statues_count": 92657 }, "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("2015-11-27T10:00:29.000Z"), "id": 670301179100794881, "text": "Check out our #listing in #Crestview #FL https://t.co/IGWqkhg4KU #realestate #realtor https://t.co/uQjvGArZPY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.57454,30.777902"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "Crestview", "FL", "realestate", "realtor" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crestview, FL", "id": "32bb9ace0723336d", "name": "Crestview", "place_type": "city", "bounding_box": rectangle("-86.635021,30.695757 -86.534517,30.805126") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1215475, "cityName": "Crestview" } }
+{ "create_at": datetime("2015-11-27T10:00:29.000Z"), "id": 670301179100835840, "text": "This #job might be a great fit for you: Server & Busser - https://t.co/ASEkSjUD1Z #Wilton, CT #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.4415929,41.2077334"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Wilton", "Hiring", "CareerArc" }}, "user": { "id": 41637830, "name": "TMJ-USA Jobs", "screen_name": "tmj_usa_jobs", "lang": "en", "location": "USA", "create_at": date("2009-05-21"), "description": "Follow this account for geo-targeted Other job tweets in USA. Need help? Tweet us at @CareerArc!", "followers_count": 882, "friends_count": 564, "statues_count": 2202 }, "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("2015-11-27T10:00:29.000Z"), "id": 670301179163766784, "text": "@MylesRoll knew you would be the first one to comment", "in_reply_to_status": 670300934111498241, "in_reply_to_user": 493493800, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 493493800 }}, "user": { "id": 129092474, "name": "Bailo The Buffalo", "screen_name": "bigbailey96", "lang": "en", "location": "null", "create_at": date("2010-04-02"), "description": "New Orleans, LA | Houston, TX |Bethany, OK Southern Nazarene | Fullback | #BucNation #JucoProduct | RIP Grandmother| RIP DeeRain |", "followers_count": 1427, "friends_count": 2051, "statues_count": 18920 }, "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("2015-11-27T10:00:29.000Z"), "id": 670301179755139072, "text": "Mostly cloudy this afternoon, high 74 (23 C). Low 59 (15 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119753180, "name": "Jackson Weather", "screen_name": "JacksonWX", "lang": "en", "location": "Jackson, MS", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Jackson, MS", "followers_count": 294, "friends_count": 68, "statues_count": 6260 }, "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("2015-11-27T10:00:29.000Z"), "id": 670301180019388416, "text": "@JamesWolcott \"The Knack... and now to STOP it\"?", "in_reply_to_status": 670299867848179712, "in_reply_to_user": 363809948, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 363809948 }}, "user": { "id": 9218762, "name": "Franklin Harris", "screen_name": "FranklinH3000", "lang": "en", "location": "Alabama", "create_at": date("2007-10-02"), "description": "Love child of Pauline Kael and Joe Bob Briggs. Retweets aren't endorsements except when they are. Even more annoying in person. #critic #gadfly #knowitall", "followers_count": 1383, "friends_count": 1181, "statues_count": 67894 }, "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("2015-11-27T10:00:29.000Z"), "id": 670301180157755392, "text": "@nikostar @StarTrek @gates_mcfadden THESE ARE INCREDIBLE!!!!", "in_reply_to_status": 670299458102550528, "in_reply_to_user": 19428768, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19428768, 130491582, 1137700339 }}, "user": { "id": 2349134814, "name": "Sonya Craig", "screen_name": "SonyaCraig15", "lang": "en", "location": "Austin", "create_at": date("2014-02-17"), "description": "I write Sci-Fi. I do things with words and, you know, paper. Fat Cat won't lose weight - send help!", "followers_count": 5638, "friends_count": 4829, "statues_count": 46641 }, "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-27T10:00:29.000Z"), "id": 670301180182990848, "text": "@drehypeMSU @NajeeSaysWynn @Azixz https://t.co/E59Ilcqelf", "in_reply_to_status": -1, "in_reply_to_user": 2344187256, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2344187256, 393674338, 160372392 }}, "user": { "id": 234219545, "name": "W.W.@.H.", "screen_name": "TrayJuanDon", "lang": "en", "location": "Weest Angeles, Ca", "create_at": date("2011-01-04"), "description": "#CNG #CloudNine. The New West Coast Rachet City, California Musical Artist/Poet", "followers_count": 782, "friends_count": 896, "statues_count": 23248 }, "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("2015-11-27T10:00:29.000Z"), "id": 670301180317327360, "text": "Wore my Auburn jersey shopping all morning in Huntsville and didn't get a single \"roll tide.\" I'm surprised yet impressed.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54388740, "name": "Samantha Moore", "screen_name": "sammimoore13", "lang": "en", "location": "Atlanta • Auburn University ", "create_at": date("2009-07-06"), "description": "hot n fresh out the kitchen", "followers_count": 644, "friends_count": 495, "statues_count": 7112 }, "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("2015-11-27T10:00:29.000Z"), "id": 670301180359233536, "text": "just wanna be laid up w 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": 342207362, "name": "tianaliz", "screen_name": "tianaliz_15", "lang": "en", "location": "null", "create_at": date("2011-07-25"), "description": "18.", "followers_count": 627, "friends_count": 283, "statues_count": 22353 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irondequoit, NY", "id": "31bfc1f1d5544e1b", "name": "Irondequoit", "place_type": "city", "bounding_box": rectangle("-77.623126,43.166743 -77.521784,43.256791") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3637737, "cityName": "Irondequoit" } }
+{ "create_at": datetime("2015-11-27T10:00:29.000Z"), "id": 670301180426326016, "text": "A 2000 Chevrolet C/K 2500 Series was just scanned https://t.co/Pa268hmek6 #vinny_scans #startup #carshopping", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.2,39.79"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "vinny_scans", "startup", "carshopping" }}, "user": { "id": 2361533634, "name": "Vinny Scans", "screen_name": "Vinny_Scans", "lang": "en", "location": "null", "create_at": date("2014-02-25"), "description": "In the market for a #usedcar? Ever wonder what your #car is worth? @Vinny_Says can help.", "followers_count": 783, "friends_count": 1, "statues_count": 442620 }, "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("2015-11-27T10:00:29.000Z"), "id": 670301180430434304, "text": "i was for the most part https://t.co/atQo9w1CDa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 351241298, "name": "sierra", "screen_name": "__sierranicolee", "lang": "en", "location": "somwhere dancing", "create_at": date("2011-08-08"), "description": "in dance there's no such thing as mistakes just unexpected solos | SENIOR '16", "followers_count": 808, "friends_count": 529, "statues_count": 33142 }, "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-27T10:00:29.000Z"), "id": 670301181109866496, "text": "BLACK FRIIIIIIIDAAAYYYY!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1959579655, "name": "♡☁ 19.8k ♡☁", "screen_name": "itscrewsworldTV", "lang": "en", "location": "Houston, TX", "create_at": date("2013-10-13"), "description": "17 | recording artist | song-writer | actress | promoter | model | booking-business inquires contact: crewloveeemusic2@gmail.com | #TheCrewTeam | #RIPRikoV ♡♕.", "followers_count": 19851, "friends_count": 8801, "statues_count": 111052 }, "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("2015-11-27T10:00:29.000Z"), "id": 670301181248319488, "text": "Back off Isis ��", "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": "H | Michael Boyd | good kid", "followers_count": 269, "friends_count": 117, "statues_count": 28674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-27T10:00:29.000Z"), "id": 670301181525295104, "text": "That's Leadership=> RT@EWDolan #Pope Francis: God ‘must never be used to justify hatred and violence’ https://t.co/wL63tFK0S6 …", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Pope" }}, "user_mentions": {{ 29309901 }}, "user": { "id": 470291891, "name": "Donna Boehme", "screen_name": "DonnaCBoehme", "lang": "en", "location": "International", "create_at": date("2012-01-21"), "description": "Compliance+Ethics Strategist. http://Consultant.Global http://CCOX2Columnist.Speaker.Bd Member.RAND Symposia.2015Top 100TrustThought Leaders.2015SCCEInt'lAward", "followers_count": 2630, "friends_count": 2393, "statues_count": 10237 }, "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("2015-11-27T10:00:29.000Z"), "id": 670301181684641792, "text": "Really want to go shopping", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 620333394, "name": "Sydney DeJoie", "screen_name": "SydneyDeJoie", "lang": "en", "location": "null", "create_at": date("2012-06-27"), "description": "LCHS '17 #2 V softball⚡V 1000 Track⚡#3 volleyball ⚡️ instagram: @sydneydejoie Jan 16", "followers_count": 783, "friends_count": 682, "statues_count": 9499 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Peabody, MA", "id": "facb653464828079", "name": "Peabody", "place_type": "city", "bounding_box": rectangle("-71.034586,42.494855 -70.912696,42.570296") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25009, "countyName": "Essex", "cityID": 2552490, "cityName": "Peabody" } }
+{ "create_at": datetime("2015-11-27T10:00:29.000Z"), "id": 670301181697269760, "text": "Just bought me some new uggs ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2701534378, "name": "That Crispy Bitch ", "screen_name": "Ura_Bitchmade", "lang": "en", "location": "Flat Rock, MI", "create_at": date("2014-07-11"), "description": "Sometimes the best cards you got are the ones you haven't played yet. ♣️", "followers_count": 459, "friends_count": 487, "statues_count": 17445 }, "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("2015-11-27T10:00:30.000Z"), "id": 670301182095527936, "text": "See a virtual tour of our listing on 29F Lonnie Jack Drive #Crestview #FL https://t.co/DnCjwzCA7h #realestate https://t.co/igP3Od6ajo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.570782,30.777345"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Crestview", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crestview, FL", "id": "32bb9ace0723336d", "name": "Crestview", "place_type": "city", "bounding_box": rectangle("-86.635021,30.695757 -86.534517,30.805126") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1215475, "cityName": "Crestview" } }
+{ "create_at": datetime("2015-11-27T10:00:30.000Z"), "id": 670301182120693760, "text": "It's the 2015 @NHL #ThanksgivingShowdown an Original Six Matchup. NY Rangers vs. Boston Ursarings on @nbc #NYRvsBOS https://t.co/8iYwgwFNbO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ThanksgivingShowdown", "NYRvsBOS" }}, "user_mentions": {{ 50004938, 26585095 }}, "user": { "id": 2546736998, "name": "#BlackFriday", "screen_name": "RiosRiossp", "lang": "en", "location": "Houston, Texas USA", "create_at": date("2014-06-04"), "description": "Guy with Austim Love Anime, Sports, Video Games, Music, and Anime Fanfiction. #ToonamiFaithful since 1997. PSN: redwings019304", "followers_count": 3170, "friends_count": 1356, "statues_count": 76532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-27T10:00:30.000Z"), "id": 670301182649221120, "text": "Rain this afternoon, high 53 (12 C). Low 40 (4 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119754993, "name": "Waco Weather", "screen_name": "WacoWeather", "lang": "en", "location": "Waco, TX", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Waco, TX", "followers_count": 809, "friends_count": 68, "statues_count": 6240 }, "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.069786,31.639828") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4876000, "cityName": "Waco" } }
+{ "create_at": datetime("2015-11-27T10:00:30.000Z"), "id": 670301183165120512, "text": "@SeeDerAnd perfect", "in_reply_to_status": 670301101996949504, "in_reply_to_user": 192604299, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 192604299 }}, "user": { "id": 29622280, "name": "Peter Rose Holmgren", "screen_name": "peterholmgren", "lang": "en", "location": "Des Moines", "create_at": date("2009-04-07"), "description": "I teach little kids and will try any beer once.", "followers_count": 612, "friends_count": 568, "statues_count": 17513 }, "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("2015-11-27T10:00:30.000Z"), "id": 670301183483990016, "text": "Woke up to find both @troyesivan's NYC gigs had sold out. Life was miserable and then… @PLACEBOWORLD MTV UNPLUGGED.\n\nWHERE'S MY WIG. OMFG.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 53338746, 39991878 }}, "user": { "id": 26284998, "name": "LC", "screen_name": "luiscueto", "lang": "en", "location": "NYC", "create_at": date("2009-03-24"), "description": "Before you know it, it's 1 am.\n\nInglés y español indistintamente. Get with it.", "followers_count": 686, "friends_count": 269, "statues_count": 65039 }, "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-27T10:00:30.000Z"), "id": 670301183743946752, "text": "@AaronQ13Fox @imsolohoe1 pretty sure Aaron is a Stanford guy...dude could be seriously hurt.", "in_reply_to_status": 670298851509596160, "in_reply_to_user": 28844136, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28844136, 200919864 }}, "user": { "id": 18128318, "name": "Bismo Funyuns", "screen_name": "ScoopTheBowler", "lang": "en", "location": "Washington, Kent", "create_at": date("2008-12-14"), "description": "Hey, you know what they say: see a broad to get dat booty yak 'em... ...leg 'er down a smack 'em yak 'em!", "followers_count": 541, "friends_count": 1263, "statues_count": 26566 }, "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("2015-11-27T10:00:30.000Z"), "id": 670301183811063808, "text": "chocolate garments ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 3368850717, "name": "liynaa", "screen_name": "lenniewhy", "lang": "en", "location": "null", "create_at": date("2015-07-10"), "description": "made out of love", "followers_count": 399, "friends_count": 456, "statues_count": 6047 }, "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("2015-11-27T10:00:30.000Z"), "id": 670301184012357632, "text": "@GENRayOdierno @DavidtheAdmiral @NationalGuard @USArmy @Chicago_Police @OIGUSPS @USPS I will have 2 press charges", "in_reply_to_status": 670300956655939584, "in_reply_to_user": 403481070, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 368992130, 81172502, 20600214, 8775672, 24252468, 39301597, 386507775 }}, "user": { "id": 403481070, "name": "MR. TWITT3R", "screen_name": "GIrardisGOD", "lang": "en", "location": "5027 S.Drexel Blvd.Chicago, IL", "create_at": date("2011-11-02"), "description": "where is my spirit? I'm always infinite ft @ light speed. let's exercise growing after age 21 law", "followers_count": 400, "friends_count": 1995, "statues_count": 60410 }, "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-27T10:00:30.000Z"), "id": 670301184339476480, "text": "Happy Gameday everyone. Fight, Tiger ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 37354035, "name": "Josh Smith", "screen_name": "SmeezyJ", "lang": "en", "location": "Jefferson City, MO", "create_at": date("2009-05-02"), "description": "teamwork makes the dream work\n#MIZ ΔΤΔ", "followers_count": 556, "friends_count": 403, "statues_count": 3868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jefferson City, MO", "id": "01cace72701072d0", "name": "Jefferson City", "place_type": "city", "bounding_box": rectangle("-92.312792,38.484463 -92.042823,38.620738") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29051, "countyName": "Cole", "cityID": 2937000, "cityName": "Jefferson City" } }
+{ "create_at": datetime("2015-11-27T10:00:30.000Z"), "id": 670301184532459520, "text": "@jimmychoi_ heck yes", "in_reply_to_status": 670282284671438848, "in_reply_to_user": 3193648332, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3193648332 }}, "user": { "id": 2841602833, "name": "annalysa", "screen_name": "AnnalysaCowie", "lang": "en", "location": "null", "create_at": date("2014-10-05"), "description": "null", "followers_count": 209, "friends_count": 180, "statues_count": 627 }, "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("2015-11-27T10:00:30.000Z"), "id": 670301185069350912, "text": "Showers this afternoon, high 66 (19 C). Low 54 (12 C) tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119755728, "name": "Louisville Weather", "screen_name": "Louisville_WX", "lang": "en", "location": "Louisville, KY", "create_at": date("2010-03-04"), "description": "Forecasts and advisories for Louisville, KY", "followers_count": 1301, "friends_count": 68, "statues_count": 6267 }, "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-27T10:00:30.000Z"), "id": 670301185253904384, "text": "My night was perfect & my morning too ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1499280343, "name": "Jessie Graciano", "screen_name": "graciano_jessie", "lang": "en", "location": "Jerez , Zacatecas", "create_at": date("2013-06-10"), "description": "❤️✨", "followers_count": 124, "friends_count": 115, "statues_count": 2093 }, "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("2015-11-27T10:00:30.000Z"), "id": 670301185656553474, "text": "We are looking for a buyer on 16273 PERDIDO KEY Drive #CityOfPensacola #FL https://t.co/C9oYe4efuP #realestate https://t.co/42ywcMKteX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.48469,30.287163"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CityOfPensacola", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6598 }, "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": 12033, "countyName": "Escambia" } }
+{ "create_at": datetime("2015-11-27T10:00:30.000Z"), "id": 670301185971122176, "text": "\"@SayNoMisha: Future & Drake ima need y'all to give up that JUMPMAN beat. #NoCeilings2 https://t.co/tNIgAorYiZ\" https://t.co/FX0SQLh6Op", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.6684694,29.4514567"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NoCeilings2" }}, "user_mentions": {{ 102828353 }}, "user": { "id": 492802930, "name": "Solomon M.", "screen_name": "SadDadSolo", "lang": "en", "location": "College Station, TX", "create_at": date("2012-02-14"), "description": "JOURNALISM MAJOR • PR Minor • THE FAT R.KELLY • TAMU '17", "followers_count": 1053, "friends_count": 317, "statues_count": 52701 }, "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-27T10:00:30.000Z"), "id": 670301185975291904, "text": "I suffer from shopaholic syndrome", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2543460704, "name": "Katie Hoy", "screen_name": "KatieEHoy", "lang": "en", "location": "null", "create_at": date("2014-06-03"), "description": "happy go lucky", "followers_count": 311, "friends_count": 456, "statues_count": 417 }, "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("2015-11-27T10:00:31.000Z"), "id": 670301186050957312, "text": "@vicrusso97 lying*", "in_reply_to_status": 670067163647123457, "in_reply_to_user": 2196864549, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2196864549 }}, "user": { "id": 600598025, "name": "colin", "screen_name": "Colin_McManus7", "lang": "en", "location": "null", "create_at": date("2012-06-05"), "description": "null", "followers_count": 304, "friends_count": 284, "statues_count": 5212 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walton Park, NY", "id": "936f4f2556937dd0", "name": "Walton Park", "place_type": "city", "bounding_box": rectangle("-74.261763,41.274859 -74.20429,41.333505") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36071, "countyName": "Orange", "cityID": 3678063, "cityName": "Walton Park" } }
+{ "create_at": datetime("2015-11-27T10:00:31.000Z"), "id": 670301186139009024, "text": "Thank the good Lord for great people!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174477134, "name": "Clay Parrish", "screen_name": "Clay_Parrish11", "lang": "en", "location": "null", "create_at": date("2010-08-03"), "description": "I gotta fly to St. Somewhere -Jimmy Buffett #USF", "followers_count": 863, "friends_count": 759, "statues_count": 11714 }, "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("2015-11-27T10:00:31.000Z"), "id": 670301186155679744, "text": "My dad just told me to lick 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": 531650413, "name": "Hailee Avocato", "screen_name": "hailHails", "lang": "en", "location": "null", "create_at": date("2012-03-20"), "description": "•pnw• Nothing special• C/o 2016 •Dammit Hailee• ♛ツ i dont know what im doing•⚓️ Mentally ill and fuckin cool", "followers_count": 231, "friends_count": 160, "statues_count": 8449 }, "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("2015-11-27T10:00:31.000Z"), "id": 670301186168369153, "text": "g n", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2274155218, "name": "migs ♚", "screen_name": "miguel_1429", "lang": "en", "location": "Newark, NJ", "create_at": date("2014-01-10"), "description": "eshs | DOL #13 ⚽️", "followers_count": 698, "friends_count": 294, "statues_count": 39756 }, "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-27T10:00:31.000Z"), "id": 670301186277433345, "text": "Hate being ignored��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2181252750, "name": "Syl", "screen_name": "sylviaa_gil", "lang": "en", "location": "San Antonio, TX", "create_at": date("2013-11-07"), "description": "TWA coed 5 • level 1 coach • softball & track", "followers_count": 349, "friends_count": 397, "statues_count": 3043 }, "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-27T10:00:31.000Z"), "id": 670301186340319233, "text": "@JellyIntoAJam @PFF Doubtful", "in_reply_to_status": 670286880613732353, "in_reply_to_user": 301199121, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 301199121, 87954771 }}, "user": { "id": 56009189, "name": "Bishop", "screen_name": "branbish", "lang": "en", "location": "NaShViLlE, Tn", "create_at": date("2009-07-11"), "description": "Businessman, engineer, music lover, & world explorer. Slaying Conservative nonsense one tweet at a time. Any regretted tweet will be blamed on my young intern.", "followers_count": 181, "friends_count": 582, "statues_count": 10618 }, "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("2015-11-27T10:00:31.000Z"), "id": 670301186399019008, "text": "You know niggas love pretty bitches w/ ambition!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.58032218,39.3603952"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 225625682, "name": "T-Raw", "screen_name": "TyraTheCreator_", "lang": "en", "location": "null", "create_at": date("2010-12-11"), "description": "trust your struggle #MSU19 #polyalumni", "followers_count": 1278, "friends_count": 1047, "statues_count": 23666 }, "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-27T10:00:31.000Z"), "id": 670301186587820032, "text": "I shake it like jello, make the boys say hello", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 458009221, "name": "к α т", "screen_name": "kattprezz", "lang": "en", "location": "ⓞ ⓗ ⓘ ⓞ", "create_at": date("2012-01-07"), "description": "Lost somewhere between Jack Daniels & Jesus", "followers_count": 660, "friends_count": 428, "statues_count": 22569 }, "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-27T10:00:31.000Z"), "id": 670301186776428544, "text": "The holiday snacks my mom won't make any other time of the year >>>", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 564318456, "name": "Paulyeeen ", "screen_name": "paulinemarie55", "lang": "en", "location": "null", "create_at": date("2012-04-26"), "description": "null", "followers_count": 412, "friends_count": 407, "statues_count": 3067 }, "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-27T10:00:31.000Z"), "id": 670301186847858689, "text": "#Gratitude @pearceauto @JoshTropical @MpsSaltsman @omar666781 @JuanaBreeze @nicolah501 @ONTRAKBRADFORD #Quotes https://t.co/82osuk0zXh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Gratitude", "Quotes" }}, "user_mentions": {{ 410426987, 2309607516, 1017703765, 4035909674, 3032943880, 434120825, 3004061963 }}, "user": { "id": 29997206, "name": "Lauri Meizler", "screen_name": "LauriMeizler", "lang": "en", "location": "Boston area", "create_at": date("2009-04-09"), "description": "#Entrepreneur #healthcoach #speaker #blogger supporting you to realize strength #love innerpeace, combat #DomesticAbuse #Crohns mom #WomenWhoInspire #Glutenfree", "followers_count": 44497, "friends_count": 43624, "statues_count": 36911 }, "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("2015-11-27T10:00:31.000Z"), "id": 670301186894004224, "text": "#BoycottBlackFriday https://t.co/oDTVP9exhk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "BoycottBlackFriday" }}, "user": { "id": 2290538707, "name": "Alton Pitre", "screen_name": "TheJayAlligator", "lang": "en", "location": "The Jungles, Los Angeles ✈️✈️", "create_at": date("2014-01-13"), "description": "From jailhouse to Morehouse to the White House! Life's a Jungle but I was raised in one. JuvenileJustice Ambassador @antirecidivism Columnist @jjienews #50Mil", "followers_count": 780, "friends_count": 544, "statues_count": 10153 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newport, SC", "id": "004ae40265bd4dc2", "name": "Newport", "place_type": "city", "bounding_box": rectangle("-81.18872,34.953392 -81.057995,35.087441") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45091, "countyName": "York", "cityID": 4549885, "cityName": "Newport" } }
+{ "create_at": datetime("2015-11-27T10:00:31.000Z"), "id": 670301186948370433, "text": "telling people your problems is pointless. so i won’t be doing that anymore ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 545641253, "name": "juliana santana▪️", "screen_name": "juuulzsantana", "lang": "en", "location": "Madrid, Spain", "create_at": date("2012-04-04"), "description": "nobody special", "followers_count": 2268, "friends_count": 710, "statues_count": 82287 }, "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-27T10:00:31.000Z"), "id": 670301186952699904, "text": "@erin_prohowich you just got that busy life's all. It's all good tho I'll help ya make it through it all", "in_reply_to_status": 670300849625653249, "in_reply_to_user": 1187409150, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1187409150 }}, "user": { "id": 2559447513, "name": "Hayden", "screen_name": "_dennis_menace_", "lang": "en", "location": "null", "create_at": date("2014-05-22"), "description": "CNU '18 / RVA / Balloon animal enthusiast", "followers_count": 248, "friends_count": 240, "statues_count": 2723 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crozet, VA", "id": "3882c456cf4170c2", "name": "Crozet", "place_type": "city", "bounding_box": rectangle("-78.728108,38.051152 -78.651865,38.082472") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51003, "countyName": "Albemarle", "cityID": 5120560, "cityName": "Crozet" } }
+{ "create_at": datetime("2015-11-27T10:00:31.000Z"), "id": 670301187392929792, "text": "Ready to get my new whip��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 336209233, "name": "Léigh Çhung", "screen_name": "kemoskie", "lang": "en", "location": "Port Arthur, TX", "create_at": date("2011-07-15"), "description": "#SHSU Patricia Ann Scypion. | sc: kemoskiee", "followers_count": 2213, "friends_count": 877, "statues_count": 68683 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nederland, TX", "id": "9bc7a8ad8f4a67b6", "name": "Nederland", "place_type": "city", "bounding_box": rectangle("-94.097964,29.946424 -93.964031,30.012307") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48245, "countyName": "Jefferson", "cityID": 4850580, "cityName": "Nederland" } }
+{ "create_at": datetime("2015-11-27T10:00:31.000Z"), "id": 670301187657220096, "text": "They Wildin ���� @WavyRJ https://t.co/fBdUDneqEJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1173753901 }}, "user": { "id": 2606917284, "name": "Chaz_F", "screen_name": "_chazf", "lang": "en", "location": "null", "create_at": date("2014-07-05"), "description": "Young man of faith • Mathew6:33 #greatnessisuponme • ASU • Walter Cronkite Sports brodacasting", "followers_count": 153, "friends_count": 262, "statues_count": 1857 }, "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("2015-11-27T10:00:31.000Z"), "id": 670301187988656128, "text": "@LizzyAMurray it was longer", "in_reply_to_status": 617803589990354944, "in_reply_to_user": 866667745, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 113520677 }}, "user": { "id": 866667745, "name": "Nate Zurawski", "screen_name": "meandernate", "lang": "en", "location": "the choir room", "create_at": date("2012-10-07"), "description": "Hello! I'm in a band @MeanderingCberr The Pun King. A Person Who Dries Their Hands on Their Pants. Future Music Educator. It's fine", "followers_count": 478, "friends_count": 646, "statues_count": 8973 }, "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("2015-11-27T10:00:31.000Z"), "id": 670301188043075584, "text": "@HSMdeMexico @kaidanrobert https://t.co/nnl17eLiYn", "in_reply_to_status": 670097452033290240, "in_reply_to_user": 3222602556, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3222602556, 2742802725 }}, "user": { "id": 2196023482, "name": "rahim mojaver", "screen_name": "rahimmojaver1", "lang": "en", "location": "null", "create_at": date("2013-11-25"), "description": "null", "followers_count": 139, "friends_count": 475, "statues_count": 1038 }, "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("2015-11-27T10:00:31.000Z"), "id": 670301188097617920, "text": "@jarpad i was excited bc it's black friday meaning there's a discount, but it went down to $80 and that's $80 i don't have", "in_reply_to_status": 670300931435556864, "in_reply_to_user": 2562652270, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 310687757 }}, "user": { "id": 2562652270, "name": "madi ❦", "screen_name": "j2ftbands", "lang": "en", "location": "princessjarpad :-) ad biersack", "create_at": date("2014-05-24"), "description": "JARED: we're not the LOSEchesters #AlwaysKeepFighting ;-/ DC", "followers_count": 386, "friends_count": 283, "statues_count": 9192 }, "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("2015-11-27T10:00:31.000Z"), "id": 670301188341022720, "text": "@ObisKanobis @LilPookie yessir -- all is good !! ✌️", "in_reply_to_status": 670300276042170369, "in_reply_to_user": 3040209778, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3040209778, 3232458741 }}, "user": { "id": 285257740, "name": "MrFlorida88", "screen_name": "MrFlorida88", "lang": "en", "location": "Naples, FL", "create_at": date("2011-04-20"), "description": "Calm down , it's only Twitter !", "followers_count": 339, "friends_count": 386, "statues_count": 19001 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marco Island, FL", "id": "398e489d0705ff47", "name": "Marco Island", "place_type": "city", "bounding_box": rectangle("-81.750726,25.90813 -81.673991,25.974477") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12021, "countyName": "Collier", "cityID": 1243083, "cityName": "Marco Island" } }
+{ "create_at": datetime("2015-11-27T10:00:31.000Z"), "id": 670301188361838592, "text": "@Baseballmasta thanks Dennis ����", "in_reply_to_status": 670298814494838785, "in_reply_to_user": 576448086, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 576448086 }}, "user": { "id": 883593284, "name": "Jordan Luthi", "screen_name": "JLUTH21", "lang": "en", "location": "null", "create_at": date("2012-10-15"), "description": "Hutchinson CC commit | God bless America.", "followers_count": 289, "friends_count": 258, "statues_count": 848 }, "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-27T10:00:31.000Z"), "id": 670301188529774592, "text": "Smh it's 2015 almost 2016 and we still calling it BLACK Friday ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1702473876, "name": "〽️atæo Salæo", "screen_name": "MattSaleh13", "lang": "en", "location": "City of Stars", "create_at": date("2013-08-26"), "description": "Feelin fine since 99'|wrestling|BHS| Phd in poonology", "followers_count": 329, "friends_count": 318, "statues_count": 10446 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bergenfield, NJ", "id": "980e30571d9d5f5d", "name": "Bergenfield", "place_type": "city", "bounding_box": rectangle("-74.017383,40.908458 -73.979592,40.936872") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3405170, "cityName": "Bergenfield" } }
+{ "create_at": datetime("2015-11-27T10:00:31.000Z"), "id": 670301188701552641, "text": "See a virtual tour of our listing on 110 Gulf Shore Dr #Destin #FL https://t.co/UrZNyViu3h #realestate https://t.co/5nmLuAG97f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.508545,30.38929"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Destin", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6599 }, "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("2015-11-27T10:00:31.000Z"), "id": 670301188743667712, "text": "Mr.rivlins paradise https://t.co/ywZKXzhR2B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1876893841, "name": "™", "screen_name": "KyleTanzer2000", "lang": "en", "location": "#732", "create_at": date("2013-09-17"), "description": "To never try is the ultimate fail mhs '18 its a movie ⚡️ @ninarrrivera", "followers_count": 589, "friends_count": 574, "statues_count": 26393 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Robertsville, NJ", "id": "1abb3fe80ce0b295", "name": "Robertsville", "place_type": "city", "bounding_box": rectangle("-74.327314,40.315225 -74.260808,40.370606") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3463900, "cityName": "Robertsville" } }
+{ "create_at": datetime("2015-11-27T10:00:31.000Z"), "id": 670301188827447297, "text": "The holiday elves at California Lustre (aka me and @thrashcowboy666) are working round the clock to… https://t.co/Wxxc358YDf", "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, "user": { "id": 3253357357, "name": "California Lustre", "screen_name": "calilustre", "lang": "en", "location": "Palm Springs, CA", "create_at": date("2015-06-22"), "description": "Made-to-order party decor from sunny Palm Springs Make a splash with our handcrafted acrylic drink stirrers, cake toppers & more!", "followers_count": 27, "friends_count": 14, "statues_count": 151 }, "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("2015-11-27T10:00:31.000Z"), "id": 670301188991139841, "text": "She's not a model but she does play the part", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 356395537, "name": "Megan Neville", "screen_name": "Meg_Neville", "lang": "en", "location": "Barbie ", "create_at": date("2011-08-16"), "description": "good girl with a couple of bad habits", "followers_count": 1053, "friends_count": 584, "statues_count": 20368 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Macedonia, OH", "id": "9b8a8b5437a94787", "name": "Macedonia", "place_type": "city", "bounding_box": rectangle("-81.530955,41.276868 -81.463345,41.350133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3945976, "cityName": "Macedonia" } }
+{ "create_at": datetime("2015-11-27T10:00:31.000Z"), "id": 670301189725122560, "text": "shoulda stayed at gearos ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2383744378, "name": "caroline forbes", "screen_name": "jillian_stokes", "lang": "en", "location": "philadelphia ", "create_at": date("2014-03-04"), "description": "I'm average http://jilliannstokess.vsco.co", "followers_count": 1194, "friends_count": 737, "statues_count": 30488 }, "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-27T10:00:31.000Z"), "id": 670301189762846720, "text": "Thanksgiving vs Black Friday https://t.co/PvgLySSpDD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1337614976, "name": "Mr Miagi", "screen_name": "grantblalock", "lang": "en", "location": "The Corner", "create_at": date("2013-04-08"), "description": "Ric Flair. Elton John. & Chris Kyle. // Go Vols", "followers_count": 149, "friends_count": 172, "statues_count": 603 }, "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("2015-11-27T10:00:31.000Z"), "id": 670301189972430848, "text": "@jackyunquatre @Twomey is open today until 5PM. Next time you have a question use the @GetCloseApp! https://t.co/EUACC8rb0g #winehack", "in_reply_to_status": 670297892406472704, "in_reply_to_user": 335538785, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "winehack" }}, "user_mentions": {{ 335538785, 525876061, 2332591328 }}, "user": { "id": 2332591328, "name": "GetClose", "screen_name": "GetCloseApp", "lang": "en", "location": "San Francisco, CA", "create_at": date("2014-02-07"), "description": "Message any business, for any reason and get a quick response! Download at http://appstore.com/getclose or contact us (text only) at (415) 200-2722", "followers_count": 113, "friends_count": 327, "statues_count": 220 }, "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-27T10:00:32.000Z"), "id": 670301190790361088, "text": "Interested in a #Nursing #job near #Monterey, CA? This could be a great fit: https://t.co/JZDykDEkr0 #ORjobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.9131935,36.5787155"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "Monterey", "ORjobs", "Hiring" }}, "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": 52, "friends_count": 27, "statues_count": 49 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monterey, CA", "id": "49af5b43d4963f4c", "name": "Monterey", "place_type": "city", "bounding_box": rectangle("-121.927024,36.573741 -121.830974,36.618826") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 648872, "cityName": "Monterey" } }
+{ "create_at": datetime("2015-11-27T10:00:32.000Z"), "id": 670301190991847425, "text": "Drinking a Pils by @lagunitasbeer @ Boombozz Pizza and Taproom — https://t.co/COU9e2F0cc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.747,38.3287"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1441737372 }}, "user": { "id": 90540586, "name": "Tanner Wortham", "screen_name": "tannerwortham", "lang": "en", "location": "null", "create_at": date("2009-11-16"), "description": "#teamBBN #teamiPhone", "followers_count": 321, "friends_count": 642, "statues_count": 968 }, "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("2015-11-27T10:00:32.000Z"), "id": 670301191675486208, "text": "my mom is dancing and singing to justin bieber in the shower", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2282978036, "name": "alexus", "screen_name": "_felizalexus", "lang": "en", "location": "louisiana", "create_at": date("2014-01-08"), "description": "nhs", "followers_count": 226, "friends_count": 116, "statues_count": 4915 }, "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("2015-11-27T10:00:32.000Z"), "id": 670301191801303041, "text": "Black fridays are crazy. I just waited in line for 3 hours to buy a new release of beer...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2911600659, "name": "Gettinbizzy89", "screen_name": "GBisswurm", "lang": "en", "location": "null", "create_at": date("2014-12-08"), "description": "Been to Busy Lately But I Still Love All of You", "followers_count": 31, "friends_count": 60, "statues_count": 426 }, "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("2015-11-27T10:00:32.000Z"), "id": 670301192174624768, "text": "Darcie: *says something about another Darcie*\nCharlie: but you're Charlie\nDarcie: it's another Darcie\nCharlie: what's her name?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3150235835, "name": "bad juju", "screen_name": "katiemstorey", "lang": "en", "location": "null", "create_at": date("2015-04-08"), "description": "null", "followers_count": 111, "friends_count": 123, "statues_count": 723 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beech Grove, IN", "id": "65603dcc0062a558", "name": "Beech Grove", "place_type": "city", "bounding_box": rectangle("-86.113129,39.694302 -86.052273,39.733898") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18097, "countyName": "Marion", "cityID": 1804204, "cityName": "Beech Grove" } }
+{ "create_at": datetime("2015-11-27T10:00:32.000Z"), "id": 670301192296243200, "text": "Yeah, Black Friday & rampant consumerism is gross but a low income family trying to make their kids' Xmas special with limited means isn't.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 39646859, "name": "Zach Peterson", "screen_name": "MrZachPeterson", "lang": "en", "location": "Chicago/Omaha/Ghost ship", "create_at": date("2009-05-12"), "description": "Future Recluse. Currently a floating orb. @ArgueAndGrieve @TheComedyExpo @GATIpod @VHS_Comedy", "followers_count": 1739, "friends_count": 914, "statues_count": 14540 }, "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("2015-11-27T10:00:32.000Z"), "id": 670301192296275968, "text": "On a whim: A post-bkst, pre-lunch ham & cheese croissant at devocionusa! #williamsburg @ Devocion USA https://t.co/EV9sQTrVRu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.96470279,40.71602304"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "williamsburg" }}, "user": { "id": 18464382, "name": "cynthia sin-yi cheng", "screen_name": "cynetvin", "lang": "en", "location": "present city: NYC", "create_at": date("2008-12-29"), "description": "champagne defines me & i live through ms. bond (@boldmsbond).", "followers_count": 1910, "friends_count": 382, "statues_count": 36533 }, "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-27T10:00:32.000Z"), "id": 670301192744861696, "text": "@Isaiahcarrion aw thanks love ����", "in_reply_to_status": 670300366739603456, "in_reply_to_user": 1016419500, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1016419500 }}, "user": { "id": 3501137114, "name": "OhYayaBells ❄️☃", "screen_name": "yaya15torres", "lang": "en", "location": "null", "create_at": date("2015-09-08"), "description": "blessed / ✨✨✨ ⚽️❤️", "followers_count": 139, "friends_count": 157, "statues_count": 840 }, "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-27T10:00:32.000Z"), "id": 670301192992485376, "text": "@Kenya_1st bitch GM", "in_reply_to_status": -1, "in_reply_to_user": 180236265, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 180236265 }}, "user": { "id": 2881893687, "name": "Jackie", "screen_name": "ImJusJacky", "lang": "en", "location": "InTheHouse", "create_at": date("2014-11-17"), "description": "Like It Or Not I'm Am who I Am!!!", "followers_count": 71, "friends_count": 45, "statues_count": 3265 }, "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("2015-11-27T10:00:32.000Z"), "id": 670301193134977024, "text": "Good afternoon New York! Today's sunset will be at 4:30 pm. Here is the top news at this moment. https://t.co/RYxy9J0VN2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.00001602,40.71669795"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 53932318, "name": "New York Press", "screen_name": "NewYorkCP", "lang": "en", "location": "New York", "create_at": date("2009-07-05"), "description": "Latest news from New York. Updates are frequent.", "followers_count": 6053, "friends_count": 53, "statues_count": 299708 }, "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("2015-11-27T10:00:32.000Z"), "id": 670301193143320576, "text": "I Get, Mad Zooted \nI Got This Bad Bitch I Recruited", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 284785177, "name": "R.I.P Dominque", "screen_name": "OMG_ItssRedd", "lang": "en", "location": "✌", "create_at": date("2011-04-19"), "description": "R.I.P To My Brother Dominque , I LOVE YOU ❤ |BFL Made | \nDTx| M.M.M", "followers_count": 1468, "friends_count": 1033, "statues_count": 23255 }, "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-27T10:00:32.000Z"), "id": 670301193999138816, "text": "Well that happened ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2941482112, "name": "Anthony", "screen_name": "afscsrpino", "lang": "en", "location": "Erie, PA", "create_at": date("2014-12-25"), "description": "swag", "followers_count": 140, "friends_count": 411, "statues_count": 4214 }, "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("2015-11-27T10:00:32.000Z"), "id": 670301194099621890, "text": "We are looking for a buyer on 1338 Cherry Ave #Chipley #FL https://t.co/FbeG5QFFTg #realestate https://t.co/ume9pYbDtG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.544268,30.773062"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Chipley", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6600 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chipley, FL", "id": "15a4f14ad087da60", "name": "Chipley", "place_type": "city", "bounding_box": rectangle("-85.554688,30.758631 -85.524596,30.801333") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12133, "countyName": "Washington", "cityID": 1211975, "cityName": "Chipley" } }
+{ "create_at": datetime("2015-11-27T10:00:32.000Z"), "id": 670301194221395968, "text": "@ReesesPuffDaddy @cnnbrk he's arrested in a week, and will do life! Will the Cop?", "in_reply_to_status": 670274955838230529, "in_reply_to_user": 1183253340, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1183253340, 428333 }}, "user": { "id": 183744088, "name": "RoGrace", "screen_name": "rgrace76", "lang": "en", "location": "West End New Bedford", "create_at": date("2010-08-27"), "description": "#GODFAMILYFRIENDS #GFF Suport Local", "followers_count": 427, "friends_count": 365, "statues_count": 32911 }, "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("2015-11-27T10:00:32.000Z"), "id": 670301194321989634, "text": "Me: my feet are so cold!I'm dying!\n��\nMom:didn't you bring your UGGs? Those are like the warmest shoes ever\n\nMe:��oh yeah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4007555893, "name": "~cutiebear~", "screen_name": "_snow_kay", "lang": "en", "location": "null", "create_at": date("2015-10-24"), "description": "LHS flag squad ~ paws at heart ~", "followers_count": 103, "friends_count": 183, "statues_count": 224 }, "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-27T10:00:33.000Z"), "id": 670301194753982465, "text": "This No Ceilings 2 thumps bro...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3087174463, "name": "Tspilly", "screen_name": "tyler_spillman", "lang": "en", "location": "null", "create_at": date("2015-03-15"), "description": "Just getting started...", "followers_count": 342, "friends_count": 294, "statues_count": 1162 }, "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-27T10:00:33.000Z"), "id": 670301195001401345, "text": "awwww hell ya!!!!!! https://t.co/3r0Oq5PNWT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1578672654, "name": "marcellaaaa", "screen_name": "ezziegirl30", "lang": "en", "location": "somewhere w/ lindsey", "create_at": date("2013-07-08"), "description": "sorry I have soccer/ jimmy❤️", "followers_count": 1083, "friends_count": 987, "statues_count": 15490 }, "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-27T10:00:33.000Z"), "id": 670301195248910336, "text": "Interested in a #Healthcare #job near #Miami, FL? This could be a great fit: https://t.co/NbCTvyJs2U #pediatrics #ultrasoundtech #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.2940652,25.7414669"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "Miami", "pediatrics", "ultrasoundtech", "Hiring" }}, "user": { "id": 42622658, "name": "Miami Childrens Jobs", "screen_name": "MCHCareers", "lang": "en", "location": "Miami, Florida", "create_at": date("2009-05-26"), "description": "Nicklaus Children’s Hospital, formerly Miami Children's Hospital, is S.Fla’s only licensed pediatric specialty hospital.", "followers_count": 1074, "friends_count": 97, "statues_count": 829 }, "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("2015-11-27T10:00:33.000Z"), "id": 670301195303391232, "text": "first thanksgiving where no one got drunk, wut", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 445285566, "name": "Rosie", "screen_name": "rosieposie_WOO", "lang": "en", "location": "Electron Land", "create_at": date("2011-12-23"), "description": "LA 90250 // UC Berkeley - Evolve or go extinct -", "followers_count": 659, "friends_count": 641, "statues_count": 45633 }, "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("2015-11-27T10:00:33.000Z"), "id": 670301195316019200, "text": "@JosephScrimshaw Ever perform in Santa Barbara or at @TheIrvineImprov ?", "in_reply_to_status": 670292325097041921, "in_reply_to_user": 80680964, "favorite_count": 0, "retweet_count": 0, "lang": "no", "is_retweet": false, "user_mentions": {{ 80680964, 128388773 }}, "user": { "id": 4167509860, "name": "Tony Moro", "screen_name": "tonymorosb", "lang": "en", "location": "Dana Point, CA", "create_at": date("2015-11-11"), "description": "golfer, movie lover, video gamer, LOVE action and horror films, Empire Strikes Back is my favorite film ever, born and raised in Santa Barbara, CA.", "followers_count": 84, "friends_count": 247, "statues_count": 1234 }, "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("2015-11-27T10:00:33.000Z"), "id": 670301195605557252, "text": "Snow Days in #creedecolorado @ Creede, Colorado https://t.co/Ib5GsHbilC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-106.926,37.8492"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "creedecolorado" }}, "user": { "id": 108079067, "name": "aaron miltenberger", "screen_name": "aaronmilt", "lang": "en", "location": "null", "create_at": date("2010-01-24"), "description": "Connector, thinker, DnD player, challenger of things, people and systems", "followers_count": 164, "friends_count": 159, "statues_count": 718 }, "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": 8079, "countyName": "Mineral", "cityID": 814765, "cityName": "City of Creede" } }
+{ "create_at": datetime("2015-11-27T10:00:33.000Z"), "id": 670301195689312256, "text": "*Waaaaaaay Better https://t.co/hSgIT1xQBe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2381067475, "name": "Kenny J ❤", "screen_name": "__Jewelss_", "lang": "en", "location": "null", "create_at": date("2014-03-09"), "description": "HTX | Sc : Ka.aay", "followers_count": 779, "friends_count": 598, "statues_count": 19637 }, "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("2015-11-27T10:00:33.000Z"), "id": 670301195756376065, "text": "@Dubaluu haha the best!", "in_reply_to_status": 670295346665291776, "in_reply_to_user": 192752143, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 192752143 }}, "user": { "id": 141069060, "name": "hello~clouds", "screen_name": "anaa_xo", "lang": "en", "location": "Los Angeles, California", "create_at": date("2010-05-06"), "description": "Melt my happiness, some kind of fucked up mess", "followers_count": 758, "friends_count": 742, "statues_count": 37191 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-27T10:00:33.000Z"), "id": 670301195974627329, "text": "why do we value a child's life more than an adults?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 371477612, "name": "cody", "screen_name": "CodySThreadgill", "lang": "en", "location": "null", "create_at": date("2011-09-10"), "description": "null", "followers_count": 507, "friends_count": 307, "statues_count": 12914 }, "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("2015-11-27T10:00:33.000Z"), "id": 670301196121456641, "text": "I'm getting to love my natural hair. It's just a lot to deal 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": 2370488882, "name": "Dsh !", "screen_name": "xDayziaSymoneee", "lang": "en", "location": "null", "create_at": date("2014-03-03"), "description": "null", "followers_count": 604, "friends_count": 574, "statues_count": 7909 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rocky Mount, NC", "id": "1701e6db682eb647", "name": "Rocky Mount", "place_type": "city", "bounding_box": rectangle("-77.907739,35.890099 -77.715568,36.054453") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37127, "countyName": "Nash", "cityID": 3757500, "cityName": "Rocky Mount" } }
+{ "create_at": datetime("2015-11-27T10:00:33.000Z"), "id": 670301196318560257, "text": "Can't beat our Black Friday deal with a free lunch at the truck with Roasted Red Pepper Chicken… https://t.co/nAS44nyaoS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.5574036,33.1876564"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 273695962, "name": "FIVE Tuscaloosa ", "screen_name": "FIVEtuscaloosa", "lang": "en", "location": "tuscaloosa,al 35401", "create_at": date("2011-03-28"), "description": "There are complications in life. Dining out shouldn't be one of them.", "followers_count": 3627, "friends_count": 1676, "statues_count": 810 }, "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-27T10:00:33.000Z"), "id": 670301196582674432, "text": "Wanna do something today ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 442111992, "name": "Ňohě", "screen_name": "nohe_017", "lang": "en", "location": "San Antonio, TX", "create_at": date("2011-12-20"), "description": "Shaila ❤", "followers_count": 443, "friends_count": 279, "statues_count": 17228 }, "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-27T10:00:33.000Z"), "id": 670301197312507904, "text": "@J_Oliver149 \"I make my mom sit in the back seat\"", "in_reply_to_status": 670289963582431232, "in_reply_to_user": 449391915, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 449391915 }}, "user": { "id": 2675857439, "name": "Mason Smith", "screen_name": "MasonThePrinces", "lang": "en", "location": "null", "create_at": date("2014-07-05"), "description": "#Im4thString and a #WannaBe I also invented that's what she said, but I get no credit for it.", "followers_count": 317, "friends_count": 46, "statues_count": 6589 }, "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("2015-11-27T10:00:33.000Z"), "id": 670301197316812800, "text": "@RyanRockswelLOL I emailed ya so I could find out about other things too. How many do you have?", "in_reply_to_status": 670300496733855744, "in_reply_to_user": 946972080, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 946972080 }}, "user": { "id": 336869777, "name": "Mia", "screen_name": "miacorinne", "lang": "en", "location": "Cincinnati, OH", "create_at": date("2011-07-16"), "description": "This is where my tweets live!", "followers_count": 462, "friends_count": 210, "statues_count": 22152 }, "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("2015-11-27T10:00:33.000Z"), "id": 670301197790654464, "text": "*Future https://t.co/hZLEC1O5FT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 523177064, "name": "KolbY✌️", "screen_name": "Kolby2xtimess", "lang": "en", "location": "Dallas, TX", "create_at": date("2012-03-13"), "description": "I Just Be Chillin | #Blinn17 | R.I.P. CT", "followers_count": 918, "friends_count": 495, "statues_count": 22949 }, "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("2015-11-27T10:00:33.000Z"), "id": 670301197803253761, "text": "@iamjadasmith ��", "in_reply_to_status": 670157259373088768, "in_reply_to_user": 2872358017, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2872358017 }}, "user": { "id": 2872358017, "name": "Jada Smith", "screen_name": "iamjadasmith", "lang": "en", "location": "HOUSTON", "create_at": date("2014-10-22"), "description": "18|Weaveologist Snap|Instagram @Iamjadasmith", "followers_count": 858, "friends_count": 246, "statues_count": 9030 }, "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("2015-11-27T10:00:33.000Z"), "id": 670301197828382720, "text": "WANT https://t.co/DJB3EwnyY9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2904114354, "name": "vanessa hernandez", "screen_name": "nessahern2410", "lang": "en", "location": "null", "create_at": date("2014-11-18"), "description": "Fresno state softball commited❤️ Dirtdogs softball", "followers_count": 559, "friends_count": 915, "statues_count": 4781 }, "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("2015-11-27T10:00:33.000Z"), "id": 670301198231035904, "text": "See a virtual tour of our listing on 725 SINCLAIR St #Chipley #FL https://t.co/tK6seq2OeP #realestate https://t.co/vUHyp9jTdc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.54624,30.777874"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Chipley", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chipley, FL", "id": "15a4f14ad087da60", "name": "Chipley", "place_type": "city", "bounding_box": rectangle("-85.554688,30.758631 -85.524596,30.801333") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12133, "countyName": "Washington", "cityID": 1211975, "cityName": "Chipley" } }
+{ "create_at": datetime("2015-11-27T10:00:34.000Z"), "id": 670301198797422593, "text": "The double dub? https://t.co/CAFfyHiWiq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1173753901, "name": "RAyShAuN", "screen_name": "WavyRJ", "lang": "en", "location": "IE,", "create_at": date("2013-02-12"), "description": "@kiannaaamariee ❤️", "followers_count": 1462, "friends_count": 1318, "statues_count": 42801 }, "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-27T10:00:34.000Z"), "id": 670301198948241408, "text": "@DDave808 sure does. I was just curious about how things went", "in_reply_to_status": 670300977589678080, "in_reply_to_user": 3311083398, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3311083398 }}, "user": { "id": 39984376, "name": "Elizabeth Nava", "screen_name": "ERNava", "lang": "en", "location": "Lincoln, NE", "create_at": date("2009-05-14"), "description": "#HUSKERS, #DISNEY, #NHL & Lincoln Stars hockey", "followers_count": 784, "friends_count": 1018, "statues_count": 37255 }, "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("2015-11-27T10:00:34.000Z"), "id": 670301199153807360, "text": "love spending my days with my little sister! :))�� https://t.co/efwhs0iG2s", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 503754453, "name": "shelbi tidwell", "screen_name": "shelbsrocks", "lang": "en", "location": "Weatherford, TX", "create_at": date("2012-02-25"), "description": "@WillPaschal17 is my ride or die", "followers_count": 249, "friends_count": 223, "statues_count": 5198 }, "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("2015-11-27T10:00:34.000Z"), "id": 670301199221026816, "text": "@RE_SetZ @RE_Draftsz yo drafts want to play", "in_reply_to_status": 670298906371170305, "in_reply_to_user": 2784858350, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2784858350, 2961019533 }}, "user": { "id": 2531406546, "name": "#TheComeUp", "screen_name": "Nexify_", "lang": "en", "location": "null", "create_at": date("2014-05-28"), "description": "Im Confident Not Cocky I have lan xp\nPeople still Warrior on Lan", "followers_count": 400, "friends_count": 293, "statues_count": 6421 }, "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("2015-11-27T10:00:34.000Z"), "id": 670301199279763456, "text": "@goddess_statuss Gurrrrrrllllll", "in_reply_to_status": 670300954516934656, "in_reply_to_user": 2809955502, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 2809955502 }}, "user": { "id": 127803431, "name": "Ayanna Harrison", "screen_name": "ayannaMUSIC", "lang": "en", "location": "null", "create_at": date("2010-03-30"), "description": "Aspiring Singer. Tune in to my show on TuneIn Radio at WSLC on Fridays from 6-7pm! Be blessed! http://tunein.com/radio/Sarah-Lawrence-College-Rad", "followers_count": 195, "friends_count": 691, "statues_count": 1733 }, "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("2015-11-27T10:00:34.000Z"), "id": 670301199401250817, "text": "Black Friday shopping round ✌��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1563508154, "name": "Tessa Delger", "screen_name": "tessa_delger", "lang": "en", "location": "null", "create_at": date("2013-07-02"), "description": "SHS", "followers_count": 242, "friends_count": 189, "statues_count": 5403 }, "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("2015-11-27T10:00:34.000Z"), "id": 670301199472586753, "text": "you have school... https://t.co/lA9rj3FWiT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2790398750, "name": "Tariq ugly af", "screen_name": "SavGod_", "lang": "en", "location": "NJ.", "create_at": date("2014-09-04"), "description": "Im just here to talk my shit & be thirsty, your opinion dont matter, stop asking me 4 dick pics... #DONTSHOOTYOURSHOT2K16 #IgnantTribe PEEP MY FAVS", "followers_count": 14435, "friends_count": 6975, "statues_count": 35178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vineland, NJ", "id": "ecc2e1285c7d074f", "name": "Vineland", "place_type": "city", "bounding_box": rectangle("-75.076284,39.401507 -74.945245,39.568715") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34011, "countyName": "Cumberland", "cityID": 3476070, "cityName": "Vineland" } }
+{ "create_at": datetime("2015-11-27T10:00:34.000Z"), "id": 670301199627784192, "text": "when ur bf change ur caption ���� wtf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1453210374, "name": "уαdιdαℓαdιdα✨", "screen_name": "TuNina_Lala", "lang": "en", "location": "where you really wanna be ;)", "create_at": date("2013-05-23"), "description": "my cheek bones higher than your money boi. @nine1sixx ❤️", "followers_count": 407, "friends_count": 377, "statues_count": 17812 }, "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("2015-11-27T10:00:34.000Z"), "id": 670301199661330432, "text": "@HuskerGameday I heard it from an Iowa fan too", "in_reply_to_status": 670278064132235264, "in_reply_to_user": 2728604409, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2728604409 }}, "user": { "id": 286896328, "name": "sɹǝʇsɐɯ ʇʇɐɯ", "screen_name": "MattMasters10", "lang": "en", "location": "Graettinger, Ia", "create_at": date("2011-04-23"), "description": "null", "followers_count": 299, "friends_count": 368, "statues_count": 15541 }, "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("2015-11-27T10:00:34.000Z"), "id": 670301199669682176, "text": "�� el ride mas heavy @ Bush Gardens https://t.co/3ojmKmn4sk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.4198914,28.0328407"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 191840562, "name": "Juanfer Zevallos C.", "screen_name": "JuanferZC", "lang": "en", "location": "Guayaquil - Ecuador", "create_at": date("2010-09-17"), "description": "Salesman | @swellecuador // Alterego : @scratchcolours", "followers_count": 460, "friends_count": 916, "statues_count": 5302 }, "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("2015-11-27T10:00:34.000Z"), "id": 670301199774650368, "text": "@GreyLyttle YOU BETTER BE THERE", "in_reply_to_status": 670256717498679296, "in_reply_to_user": 1347978270, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1347978270 }}, "user": { "id": 866855365, "name": "Lee Olenyik", "screen_name": "LOlenyik", "lang": "en", "location": "null", "create_at": date("2012-10-07"), "description": "the y is silent | veeb | @e_wattzz", "followers_count": 628, "friends_count": 444, "statues_count": 5774 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mechanicsville, VA", "id": "91923d1d1be8347e", "name": "Mechanicsville", "place_type": "city", "bounding_box": rectangle("-77.421421,37.576879 -77.293309,37.669732") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51085, "countyName": "Hanover", "cityID": 5150856, "cityName": "Mechanicsville" } }
+{ "create_at": datetime("2015-11-27T10:00:34.000Z"), "id": 670301199837605888, "text": "I love her so much @miadreid https://t.co/9A9VeuAWup", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1921166846 }}, "user": { "id": 2287112543, "name": "maddie craver", "screen_name": "maddiecraver13", "lang": "en", "location": "nc", "create_at": date("2014-01-16"), "description": "I'm to busy finding myself", "followers_count": 264, "friends_count": 343, "statues_count": 2794 }, "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("2015-11-27T10:00:34.000Z"), "id": 670301199959109632, "text": "relax ���� https://t.co/y5IX28Svwd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2790398750, "name": "Tariq ugly af", "screen_name": "SavGod_", "lang": "en", "location": "NJ.", "create_at": date("2014-09-04"), "description": "Im just here to talk my shit & be thirsty, your opinion dont matter, stop asking me 4 dick pics... #DONTSHOOTYOURSHOT2K16 #IgnantTribe PEEP MY FAVS", "followers_count": 14435, "friends_count": 6975, "statues_count": 35179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vineland, NJ", "id": "ecc2e1285c7d074f", "name": "Vineland", "place_type": "city", "bounding_box": rectangle("-75.076284,39.401507 -74.945245,39.568715") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34011, "countyName": "Cumberland", "cityID": 3476070, "cityName": "Vineland" } }
+{ "create_at": datetime("2015-11-27T10:00:34.000Z"), "id": 670301200382849025, "text": "@Dani_Vazquez10 k✌", "in_reply_to_status": 670300903702855680, "in_reply_to_user": 3365060172, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3365060172 }}, "user": { "id": 932967967, "name": "♡Muñequita Vázquez♡", "screen_name": "M__Vazquez", "lang": "en", "location": "null", "create_at": date("2012-11-07"), "description": "♡ muahhhღ #7⚽️", "followers_count": 309, "friends_count": 570, "statues_count": 10658 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moline, IL", "id": "b8e37f613fedf510", "name": "Moline", "place_type": "city", "bounding_box": rectangle("-90.539878,41.455977 -90.431572,41.518476") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17161, "countyName": "Rock Island", "cityID": 1749867, "cityName": "Moline" } }
+{ "create_at": datetime("2015-11-27T10:00:34.000Z"), "id": 670301200487718912, "text": "@andrianamaria_ it's not unhealthy, you have a specific mind set of what you want, and you shouldn't have to settle.", "in_reply_to_status": 670281191950557184, "in_reply_to_user": 2964814372, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2964814372 }}, "user": { "id": 97945320, "name": "Cool World", "screen_name": "cooley2012", "lang": "en", "location": "Bowling Green, OH", "create_at": date("2009-12-19"), "description": "life's an adventure, make it a fun one✌", "followers_count": 212, "friends_count": 320, "statues_count": 6341 }, "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("2015-11-27T10:00:34.000Z"), "id": 670301201523691520, "text": "@R_GarciaFuerte @inghever @HusseinElgohar3 @lamb041 @angelafabs feliz viernes amigos", "in_reply_to_status": 670294818879242241, "in_reply_to_user": 1925507593, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 1925507593, 91465050, 3227781323, 3041645306, 66722240 }}, "user": { "id": 3783098301, "name": "Reginalaleman@1246@g", "screen_name": "regialeman51", "lang": "es", "location": "Florida, USA", "create_at": date("2015-09-26"), "description": "me siento bendecida .por tanta gente linda que hay en el mundo .lo..unico es tener \nmucha sabiduría ho ..inteligencia ....o tienes que haber estudiado sicología", "followers_count": 968, "friends_count": 201, "statues_count": 12741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gladeview, FL", "id": "bf9a75b27b940e78", "name": "Gladeview", "place_type": "city", "bounding_box": rectangle("-80.258078,25.830731 -80.209318,25.847033") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1225987, "cityName": "Gladeview" } }
+{ "create_at": datetime("2015-11-27T10:00:34.000Z"), "id": 670301201825599488, "text": "When you look in your saved tweet drafts and it's a real eye opener... ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 44564910, "name": "Olivia Braga", "screen_name": "itsOliviaB", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-06-04"), "description": "|| Singer//Songwriter. Chameleon. • |#4up| • #PeachHouse • snapchat//insta: itsoliviab • Bay to LA • itsoliviab@gmail.com ||", "followers_count": 315, "friends_count": 343, "statues_count": 4023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-27T10:00:34.000Z"), "id": 670301201896861696, "text": "Check out our #listing in #Niceville #FL https://t.co/uJhhVYeIjk #realestate #realtor https://t.co/c76reMjXxD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.500849,30.526872"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "Niceville", "FL", "realestate", "realtor" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6602 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Niceville, FL", "id": "f0a5e0df7f75e514", "name": "Niceville", "place_type": "city", "bounding_box": rectangle("-86.50638,30.484446 -86.434634,30.544168") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1248750, "cityName": "Niceville" } }
+{ "create_at": datetime("2015-11-27T10:00:34.000Z"), "id": 670301202161250304, "text": "@Hard2Hate_Me girl here they go ctfu���� https://t.co/RfGOCbVn7K", "in_reply_to_status": 670300599200653312, "in_reply_to_user": 385725734, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 385725734 }}, "user": { "id": 95552992, "name": "⚓️Naj.", "screen_name": "_Kanishaa", "lang": "en", "location": "null", "create_at": date("2009-12-08"), "description": "null", "followers_count": 1970, "friends_count": 868, "statues_count": 65364 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salisbury, MD", "id": "00caf39d503a84e9", "name": "Salisbury", "place_type": "city", "bounding_box": rectangle("-75.71412,38.28923 -75.487032,38.431613") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24045, "countyName": "Wicomico", "cityID": 2469925, "cityName": "Salisbury" } }
+{ "create_at": datetime("2015-11-27T10:00:34.000Z"), "id": 670301202207236096, "text": "I'm joking I need to pass! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1543255142, "name": "Edgar Vega", "screen_name": "Edgar_Vega23", "lang": "en", "location": "Needville, Texas", "create_at": date("2013-06-24"), "description": "Let's put our minds to ease.", "followers_count": 246, "friends_count": 202, "statues_count": 2675 }, "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("2015-11-27T10:00:34.000Z"), "id": 670301202228248576, "text": "when I peep shit 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": 379877473, "name": "CARMZ", "screen_name": "carmen_tijerina", "lang": "en", "location": "the meda", "create_at": date("2011-09-25"), "description": "'murica", "followers_count": 1056, "friends_count": 732, "statues_count": 27321 }, "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("2015-11-27T10:00:34.000Z"), "id": 670301202463252480, "text": "Spem successus alit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.9214185,41.1572965"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 190455087, "name": "Ross", "screen_name": "CorbyRoss", "lang": "en", "location": "Bellevue ⏩ Crete", "create_at": date("2010-09-13"), "description": "My life is pretty much summed up by the fire emoji and bailey.", "followers_count": 694, "friends_count": 348, "statues_count": 8865 }, "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("2015-11-27T10:00:34.000Z"), "id": 670301202609864704, "text": "I miss this so much https://t.co/0qcGubs9NN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1348428709, "name": "Brittani✨", "screen_name": "brittanicaitlin", "lang": "en", "location": "Clovis, California", "create_at": date("2013-04-12"), "description": "19. Scorpio. Alex's Princess", "followers_count": 179, "friends_count": 234, "statues_count": 6630 }, "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("2015-11-27T10:00:34.000Z"), "id": 670301202614059009, "text": "#blackfriday lowest prices for #tint, #remotestarter, #heatedseats, #caraudio #sub package, and… https://t.co/UTMtkOm7zG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.280838,40.6937218"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "blackfriday", "tint", "remotestarter", "heatedseats", "caraudio", "sub" }}, "user": { "id": 3423087237, "name": "AutoEffects", "screen_name": "AutoEffectsUS", "lang": "en", "location": "null", "create_at": date("2015-08-14"), "description": "null", "followers_count": 7, "friends_count": 1, "statues_count": 87 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monaca, PA", "id": "0011ca7c3129a957", "name": "Monaca", "place_type": "city", "bounding_box": rectangle("-80.330375,40.639475 -80.245041,40.694676") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42007, "countyName": "Beaver", "cityID": 4250320, "cityName": "Monaca" } }
+{ "create_at": datetime("2015-11-27T10:00:35.000Z"), "id": 670301202979033088, "text": "Turn your radios up #LA @YoungCalifornia very own @DjCarisma is in the mix and @yesiortiz on the mic on @Power106LA #thsnksmixingweekend", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LA", "thsnksmixingweekend" }}, "user_mentions": {{ 808454347, 20206221, 19564835, 19671932 }}, "user": { "id": 561652097, "name": "#thecruzshow", "screen_name": "Westcoast_jctt", "lang": "en", "location": "null", "create_at": date("2012-04-23"), "description": "i like house music and i like hiphop music and i like going to night clubs and im part of the #jctt and i like hearing @power106la and hearing #thecruzshow", "followers_count": 1170, "friends_count": 2050, "statues_count": 49196 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.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-27T10:00:35.000Z"), "id": 670301203079798784, "text": "@RichHomieCage @asalinas15 @TheHuzlers ��", "in_reply_to_status": 670301002453622784, "in_reply_to_user": 1107825656, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1107825656, 719046120, 2302467404 }}, "user": { "id": 472331180, "name": "Malcolm Z", "screen_name": "ZaveVolak", "lang": "en", "location": "w/ Sarah + Brianne", "create_at": date("2012-01-23"), "description": "Running a 440 after my dreams @sjurbanik14", "followers_count": 909, "friends_count": 887, "statues_count": 21353 }, "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("2015-11-27T10:00:35.000Z"), "id": 670301203301928960, "text": "I think I made a mistake ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 544477497, "name": "Thicknessss", "screen_name": "___PrettyGangg", "lang": "en", "location": "☀️", "create_at": date("2012-04-03"), "description": "FreeMyBrother", "followers_count": 7293, "friends_count": 3067, "statues_count": 139002 }, "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-27T10:00:35.000Z"), "id": 670301203323068419, "text": "#ChristmasGoal kiss my man underneath the mistletoe. ��❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ChristmasGoal" }}, "user": { "id": 2498883678, "name": "Lauren Shoff❥☮❁", "screen_name": "lalalaurrr_22", "lang": "en", "location": "York, PA to the dirty south", "create_at": date("2014-05-16"), "description": "•22 •Dual Elementary Ed Major @ MU •Libra •RJC❤️ •Fishing, Archery, Jeeps •Photography •The Outdoors •Faith", "followers_count": 688, "friends_count": 759, "statues_count": 26126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East York, PA", "id": "6760c07b80cd9bf4", "name": "East York", "place_type": "city", "bounding_box": rectangle("-76.696435,39.950263 -76.659578,39.983265") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42133, "countyName": "York", "cityID": 4222104, "cityName": "East York" } }
+{ "create_at": datetime("2015-11-27T10:00:35.000Z"), "id": 670301203553648640, "text": "These kids are driving me crazy! Serenity now Serenity now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36096413, "name": "Ramon Aguillon", "screen_name": "monchis1998", "lang": "en", "location": "Zionsville, Indiana", "create_at": date("2009-04-28"), "description": "Living in Zionsville, Indiana. Working for Allison Transmission and coaching soccer at Zionsville High School.", "followers_count": 321, "friends_count": 844, "statues_count": 5791 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Zionsville, IN", "id": "3b7be78eb86afa70", "name": "Zionsville", "place_type": "city", "bounding_box": rectangle("-86.358637,39.924127 -86.240696,39.990742") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18011, "countyName": "Boone", "cityID": 1886372, "cityName": "Zionsville" } }
+{ "create_at": datetime("2015-11-27T10:00:35.000Z"), "id": 670301204010942466, "text": "Can enjoy Christmas decorations now! I was out walking 1.25 miles with #Endomondo #endorphins https://t.co/y4jBLkJ3pE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.4837857,40.3744567"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Endomondo", "endorphins" }}, "user": { "id": 278167465, "name": "Kim Frye", "screen_name": "UrbanArbor", "lang": "en", "location": "Chicago, IL", "create_at": date("2011-04-06"), "description": "urban ecologist interested in new models of agriculture; friend of the trees and gardens", "followers_count": 679, "friends_count": 1147, "statues_count": 9055 }, "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("2015-11-27T10:00:35.000Z"), "id": 670301204166148096, "text": "@SithLordEmily why can't I pick both", "in_reply_to_status": 670301007620935680, "in_reply_to_user": 348367463, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 348367463 }}, "user": { "id": 729096949, "name": "rachel", "screen_name": "moonkid93", "lang": "en", "location": "mn", "create_at": date("2012-07-31"), "description": "vote for a pulp free america", "followers_count": 3848, "friends_count": 226, "statues_count": 32096 }, "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("2015-11-27T10:00:35.000Z"), "id": 670301204241620992, "text": "I don't really care about the new bitch, it's that he treats her better than he ever treated me.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3911591362, "name": "Fresh out of fucks.", "screen_name": "86myfucks", "lang": "en", "location": "The Nasti, OH", "create_at": date("2015-10-09"), "description": "Snarky server and dog mom. Always hungry. I have three jobs.", "followers_count": 30, "friends_count": 54, "statues_count": 590 }, "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("2015-11-27T10:00:35.000Z"), "id": 670301204266680321, "text": "Happy birthday to my bff @ChelseaGatess, I hope you have a lovely day and don't remember tonight. I love you Chelly Dogg❤��❤", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 261005339 }}, "user": { "id": 453490639, "name": "Bec", "screen_name": "96beccahamilton", "lang": "en", "location": "Eureka, KS ✌️", "create_at": date("2012-01-02"), "description": "Drunk words are sober thoughts. I love Chuck Samuels", "followers_count": 456, "friends_count": 285, "statues_count": 7636 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eureka, KS", "id": "4fad60ac94100d13", "name": "Eureka", "place_type": "city", "bounding_box": rectangle("-96.306036,37.813107 -96.270671,37.839419") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20073, "countyName": "Greenwood", "cityID": 2021800, "cityName": "Eureka" } }
+{ "create_at": datetime("2015-11-27T10:00:35.000Z"), "id": 670301204325523456, "text": "hate* lol like give ya momma back her rent money girl @queen_sme https://t.co/Fn6sr7iQKf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1472946650 }}, "user": { "id": 1736946595, "name": "kitta", "screen_name": "obvslykitta", "lang": "en", "location": "null", "create_at": date("2013-09-06"), "description": "not ya type.. ig: naakitta", "followers_count": 860, "friends_count": 269, "statues_count": 39704 }, "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("2015-11-27T10:00:35.000Z"), "id": 670301204631695360, "text": "@jennapavioni love you too babe", "in_reply_to_status": 670301112289792000, "in_reply_to_user": 232310441, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 232310441 }}, "user": { "id": 474621801, "name": "captain hook", "screen_name": "Jnichols365", "lang": "en", "location": " the sinners section", "create_at": date("2012-01-25"), "description": "bat brained bruce wayne", "followers_count": 505, "friends_count": 436, "statues_count": 12878 }, "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("2015-11-27T10:00:35.000Z"), "id": 670301204937900035, "text": "������ https://t.co/sWZYVWzIuD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 227724619, "name": "RC", "screen_name": "_prettygirlrave", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2010-12-17"), "description": "null", "followers_count": 2557, "friends_count": 1664, "statues_count": 66979 }, "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("2015-11-27T10:00:35.000Z"), "id": 670301205059473409, "text": "@EliotShorrParks good article. def part of problem", "in_reply_to_status": -1, "in_reply_to_user": 169330397, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 169330397 }}, "user": { "id": 2777071335, "name": "nilesh dadhich", "screen_name": "DadhichNilesh", "lang": "en", "location": "philly", "create_at": date("2014-09-19"), "description": "null", "followers_count": 6, "friends_count": 42, "statues_count": 32 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lansdale, PA", "id": "878041494911ce4f", "name": "Lansdale", "place_type": "city", "bounding_box": rectangle("-75.303353,40.222978 -75.254299,40.261341") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42091, "countyName": "Montgomery", "cityID": 4241432, "cityName": "Lansdale" } }
+{ "create_at": datetime("2015-11-27T10:00:35.000Z"), "id": 670301205503979520, "text": "When the sun shines and they bust nines\nDon't dial 9 plus 1-1\nWhen they run up on your school bus with a two truck you might learn something", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1352813551, "name": "Bulklahoma® Sooner", "screen_name": "BossBrax", "lang": "en", "location": "Norman, Oklahoma", "create_at": date("2013-04-14"), "description": "22. Iron sharpens iron, so one man sharpens another Prov. 27:17 OU '17 #Sooners #RavensNation #Thunderup #NeverEverQuit", "followers_count": 1413, "friends_count": 1065, "statues_count": 29261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Duncan, OK", "id": "98ff6aa065874b12", "name": "Duncan", "place_type": "city", "bounding_box": rectangle("-97.998608,34.460573 -97.921111,34.565247") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40137, "countyName": "Stephens", "cityID": 4021900, "cityName": "Duncan" } }
+{ "create_at": datetime("2015-11-27T10:00:35.000Z"), "id": 670301205667512321, "text": "Not about to sit in the cold to watch Iowa win so let's #BeatIowa pls", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BeatIowa" }}, "user": { "id": 428545080, "name": "Hannah Kenny", "screen_name": "hannah_kenny2", "lang": "en", "location": "Wayne, Nebraska", "create_at": date("2011-12-04"), "description": "Zach Heaton * WSC '19", "followers_count": 602, "friends_count": 369, "statues_count": 27714 }, "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("2015-11-27T10:00:35.000Z"), "id": 670301205923426304, "text": "We are looking for a buyer on 1885 Heartland Dr #FortWaltonBeach #FL https://t.co/RHrPdfgPuL #realestate https://t.co/p1wy2mhUl9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.654727,30.460283"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FortWaltonBeach", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6603 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wright, FL", "id": "6a0c2b0ea86e9213", "name": "Wright", "place_type": "city", "bounding_box": rectangle("-86.672393,30.423666 -86.600428,30.471698") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1278800, "cityName": "Wright" } }
+{ "create_at": datetime("2015-11-27T10:00:35.000Z"), "id": 670301206229573633, "text": "??????? https://t.co/rCsYsNQiZH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1443004663, "name": "mell", "screen_name": "EaasyBuckets", "lang": "en", "location": "compton", "create_at": date("2013-05-19"), "description": "6'5 fool with that ball ...", "followers_count": 1210, "friends_count": 845, "statues_count": 57428 }, "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("2015-11-27T10:00:35.000Z"), "id": 670301206275690496, "text": "Can you recommend anyone for this #job? Assembly Team Lead - https://t.co/VNSAOBWvKB #Sanford, NC #Manufacturing #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.1371655,35.4432775"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Sanford", "Manufacturing", "Hiring", "CareerArc" }}, "user": { "id": 43624785, "name": "TMJ-USA Manuf Jobs", "screen_name": "tmj_usa_manuf", "lang": "en", "location": "USA", "create_at": date("2009-05-30"), "description": "Follow this account for geo-targeted Manufacturing job tweets in USA. Need help? Tweet us at @CareerArc!", "followers_count": 366, "friends_count": 227, "statues_count": 3641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sanford, NC", "id": "25cd0c32e9559c17", "name": "Sanford", "place_type": "city", "bounding_box": rectangle("-79.241516,35.389316 -79.085936,35.559252") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37105, "countyName": "Lee", "cityID": 3759280, "cityName": "Sanford" } }
+{ "create_at": datetime("2015-11-27T10:00:35.000Z"), "id": 670301206389137409, "text": "Fishing! (@ Stream Mobile Services III in Martin, TN) https://t.co/nU3ScWjI6t https://t.co/fSdtIQrrLh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.85085845,36.31102455"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 50712121, "name": "CyberMondayPhones", "screen_name": "ToMMaN2020", "lang": "en", "location": "Dallas, TX", "create_at": date("2009-06-25"), "description": "Delivering Unlimited Data,Text and Talk for $69.80! Unlimited Data at 4GLTESpeed on your phone. No throttling. No need to buy extra data plan. Swap now!", "followers_count": 375, "friends_count": 2022, "statues_count": 9346 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Martin, TN", "id": "7a5e97014f09d4fa", "name": "Martin", "place_type": "city", "bounding_box": rectangle("-88.899282,36.307747 -88.791208,36.373444") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47183, "countyName": "Weakley", "cityID": 4746240, "cityName": "Martin" } }
+{ "create_at": datetime("2015-11-27T10:00:35.000Z"), "id": 670301206485401600, "text": "This #Clerical #job might be a great fit for you: SUPERVISOR OF REVENUE AUDIT - https://t.co/1huwPl5GS8 #Atlanta, GA #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.352759,33.909187"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Clerical", "job", "Atlanta", "Hiring", "CareerArc" }}, "user": { "id": 22985155, "name": "Atlanta Clerical Job", "screen_name": "tmj_atl_cler", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-03-05"), "description": "Follow this account for geo-targeted Office/Clerical job tweets in Atlanta, GA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 490, "friends_count": 236, "statues_count": 116 }, "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("2015-11-27T10:00:35.000Z"), "id": 670301206510702596, "text": "@Judetruth Totally agree. Each game means so much. Makes viewers want to watch", "in_reply_to_status": 670301045390733312, "in_reply_to_user": 353301286, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 353301286 }}, "user": { "id": 374676915, "name": "Mike Nash", "screen_name": "MikeNash15", "lang": "en", "location": "null", "create_at": date("2011-09-16"), "description": "Mercy '16. Media Studies Journalism major. Staff Member @TheJetPress , Social Media and Marketing intern @JetsFanMedia 8-25-11", "followers_count": 883, "friends_count": 497, "statues_count": 31836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shrub Oak, NY", "id": "d5d80193ec83023d", "name": "Shrub Oak", "place_type": "city", "bounding_box": rectangle("-73.843063,41.304525 -73.815822,41.339138") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3667279, "cityName": "Shrub Oak" } }
+{ "create_at": datetime("2015-11-27T10:00:35.000Z"), "id": 670301206669975552, "text": "When ur team is on the verge of a perfect season and @espn keeps throwin shade but you DGAF #OnIowa #GoHawks https://t.co/DzIrPA6bun", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "OnIowa", "GoHawks" }}, "user_mentions": {{ 2557521 }}, "user": { "id": 27870868, "name": "Brendan Power", "screen_name": "bpowerhour", "lang": "en", "location": "Iowa City, IA", "create_at": date("2009-03-31"), "description": "I have flaws. What are they? I sing in the shower. Sometimes I spend too much time volunteering. Occasionally I'll hit somebody with my car.\n@uiowa. ΒΘΠ", "followers_count": 625, "friends_count": 490, "statues_count": 5276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ankeny, IA", "id": "265f5cd93d66b1a1", "name": "Ankeny", "place_type": "city", "bounding_box": rectangle("-93.677966,41.688221 -93.545095,41.777677") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1902305, "cityName": "Ankeny" } }
+{ "create_at": datetime("2015-11-27T10:00:35.000Z"), "id": 670301206804307969, "text": "@HollyHilltopper @WKUFootball they don't have much competition", "in_reply_to_status": 670298511997599744, "in_reply_to_user": 520828167, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 520828167, 1329258864 }}, "user": { "id": 250371875, "name": "Malia Bizer", "screen_name": "maliajean13", "lang": "en", "location": "Lexington-BG", "create_at": date("2011-02-10"), "description": "Junior at WKU sports fanatic & who loves Nike athletic shorts, the sunshine, Bonnaroo, chacos, dogs, & Notre Dame football. Why not me? should be your motto.", "followers_count": 494, "friends_count": 425, "statues_count": 7325 }, "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("2015-11-27T10:00:36.000Z"), "id": 670301207135555584, "text": "@theduhawk que?", "in_reply_to_status": 670300994161373184, "in_reply_to_user": 167610606, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 167610606 }}, "user": { "id": 338652780, "name": "LJ", "screen_name": "Brinky__", "lang": "en", "location": "815", "create_at": date("2011-07-19"), "description": "I'd rather be on the water", "followers_count": 1028, "friends_count": 265, "statues_count": 12528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spring Grove, IL", "id": "00adf35b11bba302", "name": "Spring Grove", "place_type": "city", "bounding_box": rectangle("-88.30152,42.415601 -88.198652,42.495082") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17111, "countyName": "McHenry", "cityID": 1772052, "cityName": "Spring Grove" } }
+{ "create_at": datetime("2015-11-27T10:00:36.000Z"), "id": 670301207257219072, "text": "Per tornare a casa fra sei mesi mi serviranno 10 valigie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 2438032443, "name": "Sofia frassineti", "screen_name": "FrassinetiSofia", "lang": "it", "location": "null", "create_at": date("2014-03-26"), "description": "null", "followers_count": 628, "friends_count": 310, "statues_count": 816 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwich, NY", "id": "c4e29966b85fd874", "name": "Norwich", "place_type": "city", "bounding_box": rectangle("-75.543363,42.503272 -75.476873,42.564814") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36017, "countyName": "Chenango", "cityID": 3653979, "cityName": "Norwich" } }
+{ "create_at": datetime("2015-11-27T10:00:36.000Z"), "id": 670301207341043713, "text": "@DylanAranyos https://t.co/6rsTcC8evO", "in_reply_to_status": -1, "in_reply_to_user": 1422426504, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1422426504 }}, "user": { "id": 1121045376, "name": "Papí", "screen_name": "YeezusDub", "lang": "en", "location": "the situation/crime scene", "create_at": date("2013-01-25"), "description": "Niggaz Want Me Dead | Ima Modern Day MLK @justinbieber", "followers_count": 1591, "friends_count": 913, "statues_count": 84182 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Haven, MI", "id": "001cb5877aca2b2a", "name": "New Haven", "place_type": "city", "bounding_box": rectangle("-82.822107,42.719016 -82.719954,42.750681") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26099, "countyName": "Macomb", "cityID": 2657380, "cityName": "New Haven" } }
+{ "create_at": datetime("2015-11-27T10:00:36.000Z"), "id": 670301208352038916, "text": "First registered human flight on a solar balloon by Tomas Saraceno in White Sands. Photo by Ewen… https://t.co/ncIiFPi6ZL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-106.40583333,32.33555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 261846355, "name": "Nahum", "screen_name": "nahummantra", "lang": "en", "location": "London, England", "create_at": date("2011-03-06"), "description": "Artist / musician. Directs space missions, #KOSMICA and works at @shuntevents, @TheArtsCatalyst & @oelastique. Obsessed with space exploration and secular magic", "followers_count": 2500, "friends_count": 1438, "statues_count": 9822 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Mexico, USA", "id": "71d65c0e6d94efab", "name": "New Mexico", "place_type": "admin", "bounding_box": rectangle("-109.050173,31.332176 -103.002065,37.000294") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35013, "countyName": "Do?a Ana" } }
+{ "create_at": datetime("2015-11-27T10:00:36.000Z"), "id": 670301208406441984, "text": "We're doing the same fucking job I get it man God", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 363911823, "name": "Erick", "screen_name": "ThatFooErick", "lang": "en", "location": "Ontario", "create_at": date("2011-08-28"), "description": "Sadie❤️", "followers_count": 263, "friends_count": 196, "statues_count": 38038 }, "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("2015-11-27T10:00:36.000Z"), "id": 670301208494477312, "text": "#Sales #Job in #Clearfield, UT: Sales Representative (B2B) Base Pay + Comm. 100719 at TruGreen https://t.co/fy736oSg04 #TruGreen #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.0260538,41.1107771"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "Job", "Clearfield", "TruGreen", "Jobs" }}, "user": { "id": 2776752523, "name": "TruGreen Jobs", "screen_name": "TruGreenJobs", "lang": "en", "location": "null", "create_at": date("2014-08-28"), "description": "null", "followers_count": 20, "friends_count": 0, "statues_count": 696 }, "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("2015-11-27T10:00:36.000Z"), "id": 670301208649699328, "text": "We're #hiring! Read about our latest #job opening here: Development Operations Engineer - https://t.co/KB6c7mK9Tt #Woburn, MA #IT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.1522765,42.4792618"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Woburn", "IT" }}, "user": { "id": 46722627, "name": "HealthcareSource", "screen_name": "quality_talent", "lang": "en", "location": "Woburn, MA, USA", "create_at": date("2009-06-12"), "description": "We're the leader in talent management software for healthcare. Our Quality Talent Suite helps healthcare organizations acquire, develop, and retain top talent.", "followers_count": 1859, "friends_count": 931, "statues_count": 5796 }, "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("2015-11-27T10:00:36.000Z"), "id": 670301208721154048, "text": "@PoconoRecord", "in_reply_to_status": 670300945956397056, "in_reply_to_user": 858403350, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 7511312 }}, "user": { "id": 858403350, "name": "Keith R. Stevenson", "screen_name": "PoconoStevenson", "lang": "en", "location": "Monroe County, PA", "create_at": date("2012-10-02"), "description": "Photojournalist for @PoconoRecord covering the Pocono Mountains", "followers_count": 256, "friends_count": 177, "statues_count": 571 }, "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("2015-11-27T10:00:36.000Z"), "id": 670301208821653504, "text": "This is a ace team at Best Buy 220 waiting on the next wave of customers!!!@markjacksonKC5 @JLeRoy01 @SprintCentral https://t.co/xywZOxk2S5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2339360994, 1219311606 }}, "user": { "id": 2747670054, "name": "Janice Elsten", "screen_name": "ElstenJanice", "lang": "en", "location": "null", "create_at": date("2014-08-19"), "description": "null", "followers_count": 22, "friends_count": 35, "statues_count": 26 }, "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-27T10:00:36.000Z"), "id": 670301208951824384, "text": "@JulesSmith412 swagemiah???", "in_reply_to_status": 663782906549116928, "in_reply_to_user": 2968165372, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2968165372 }}, "user": { "id": 3031597393, "name": "sarah thornton", "screen_name": "saraht_11", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2015-02-19"), "description": "August 2, 2015", "followers_count": 153, "friends_count": 174, "statues_count": 1645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jefferson Hills, PA", "id": "1eb10f058c1e11b0", "name": "Jefferson Hills", "place_type": "city", "bounding_box": rectangle("-79.985207,40.252933 -79.878028,40.330002") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4237955, "cityName": "Jefferson Hills" } }
+{ "create_at": datetime("2015-11-27T10:00:36.000Z"), "id": 670301209174085632, "text": "#thickthighs #muscle #bodybuilding #allnatural #quads #delts #biceps #triceps #beastmode #bodacious… https://t.co/cVeP8uYs3c", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.9584198,37.3178482"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "thickthighs", "muscle", "bodybuilding", "allnatural", "quads", "delts", "biceps", "triceps", "beastmode", "bodacious" }}, "user": { "id": 1264890864, "name": "Dr. D. Washington ", "screen_name": "dldwashington3", "lang": "en", "location": "null", "create_at": date("2013-03-13"), "description": "Personal Trainer and Les Mills Instructor: BODYPUMP®, BODYFLOW®, BODYSTEP® and CXWORX™. I'm a researcher, writer, and motivational speaker. Namaste!", "followers_count": 128, "friends_count": 215, "statues_count": 412 }, "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("2015-11-27T10:00:36.000Z"), "id": 670301209220247553, "text": "I love this app I downloaded - waterlogue. Turns your photos into watercolor paintings. @rchosewood @… https://t.co/oxdeKGpKW9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.99916667,40.6975"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 233549190 }}, "user": { "id": 18334887, "name": "Deborah Chosewood", "screen_name": "DChosewood", "lang": "en", "location": "Atlanta suburb", "create_at": date("2008-12-23"), "description": "It is our choices that show what we truly are far more than our abilities-Dumbledore. It is better to raise strong children than to fix broken men -Douglas", "followers_count": 169, "friends_count": 557, "statues_count": 9046 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn Heights, Brooklyn", "id": "4ea7f9f7e686a5c2", "name": "Brooklyn Heights", "place_type": "neighborhood", "bounding_box": rectangle("-74.003908,40.687944 -73.980848,40.701744") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2015-11-27T10:00:36.000Z"), "id": 670301209610162176, "text": "Check out our #listing in #Freeport #FL https://t.co/urSLeBW52T #realestate #realtor https://t.co/ZTcNoQBIkG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.150872,30.491273"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "Freeport", "FL", "realestate", "realtor" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6604 }, "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": 12131, "countyName": "Walton", "cityID": 1224825, "cityName": "Freeport" } }
+{ "create_at": datetime("2015-11-27T10:00:36.000Z"), "id": 670301209807351809, "text": "We're #hiring! Read about our latest #job opening here: Care Manager - https://t.co/uboNudKpmA #HuntingtonBeach, CA #Nursing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.9992265,33.660297"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "HuntingtonBeach", "Nursing" }}, "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": 292, "friends_count": 9, "statues_count": 2870 }, "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("2015-11-27T10:00:36.000Z"), "id": 670301209832521729, "text": "true", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4165577895, "name": "egotistic ✨", "screen_name": "wtfprecious", "lang": "en", "location": "w. kayla❣", "create_at": date("2015-11-11"), "description": "so much ice on i caught the flu baby", "followers_count": 374, "friends_count": 99, "statues_count": 3185 }, "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("2015-11-27T10:00:36.000Z"), "id": 670301210159640576, "text": "@rosaleeuh I hate seeing fighting videos period��", "in_reply_to_status": 670300814259306496, "in_reply_to_user": 257323136, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 257323136 }}, "user": { "id": 3270829069, "name": "Miranda", "screen_name": "mperalta_11", "lang": "en", "location": "null", "create_at": date("2015-07-07"), "description": "The grind never stops.", "followers_count": 312, "friends_count": 201, "statues_count": 3067 }, "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("2015-11-27T10:00:36.000Z"), "id": 670301210306609153, "text": "Summertime shootouts the 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": 713702241, "name": "Chris Franco", "screen_name": "Franco_C34", "lang": "en", "location": "West Haven, CT", "create_at": date("2013-10-13"), "description": "3/31/53-10/23/07 My life for your life man I wouldn't trade it", "followers_count": 452, "friends_count": 202, "statues_count": 1330 }, "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("2015-11-27T10:00:36.000Z"), "id": 670301210398711808, "text": "@amazinvids @nolanhansen49 @Braden_Hansen44 @jakeh2298 @itaIianoo", "in_reply_to_status": 664204772103184384, "in_reply_to_user": 2493427621, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2493427621, 898946870, 753565261, 2790066699, 2576526964 }}, "user": { "id": 1491514616, "name": "glen hansen", "screen_name": "ghansen921", "lang": "en", "location": "Minnesota, USA", "create_at": date("2013-06-07"), "description": "null", "followers_count": 716, "friends_count": 2042, "statues_count": 10494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Princeton, MN", "id": "2d99f3e65f91edd5", "name": "Princeton", "place_type": "city", "bounding_box": rectangle("-93.604276,45.537389 -93.555182,45.594597") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27095, "countyName": "Mille Lacs", "cityID": 2752522, "cityName": "Princeton" } }
+{ "create_at": datetime("2015-11-27T10:00:36.000Z"), "id": 670301210432421889, "text": "@Lex_OnTha_Beach be there in 5", "in_reply_to_status": 670299469276139520, "in_reply_to_user": 224786814, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 224786814 }}, "user": { "id": 796764072, "name": "Cody", "screen_name": "HEELCodyrk", "lang": "en", "location": "....", "create_at": date("2012-09-01"), "description": "Fan of Pittsburgh sports and pro wrestling. Proud republican.", "followers_count": 444, "friends_count": 712, "statues_count": 31497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, PA", "id": "4adb6477a50cec53", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-79.989014,40.360263 -79.96236,40.385779") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4208416, "cityName": "Brentwood" } }
+{ "create_at": datetime("2015-11-27T10:00:36.000Z"), "id": 670301210470195200, "text": "#ROCKSTAR always doing it BIG #TH4NKSGIVING https://t.co/M6EvM08ijy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ROCKSTAR", "TH4NKSGIVING" }}, "user": { "id": 76542056, "name": "ROCKSTAR SATELLITE♛", "screen_name": "rockstarDSM", "lang": "en", "location": "N 41°36' 0'' / W 93°34' 0''", "create_at": date("2009-09-22"), "description": "♛ Home of the World Famous ROCKSTAR satellite! ♛ #ROCKSTAR", "followers_count": 442, "friends_count": 1449, "statues_count": 1709 }, "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("2015-11-27T10:00:36.000Z"), "id": 670301210486919168, "text": "Cowboys y'all can quit this season now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2226964307, "name": "mahmoud ☪", "screen_name": "mahmoudnaser56", "lang": "en", "location": "Egypt", "create_at": date("2013-12-15"), "description": "seeker of knowledge. North African #eaglesnation #stayhumble", "followers_count": 172, "friends_count": 78, "statues_count": 2127 }, "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("2015-11-27T10:00:36.000Z"), "id": 670301210654699520, "text": "That's a bet.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1464421446, "name": "Gary B", "screen_name": "_Bigpuckk", "lang": "en", "location": "Baltimore, MD", "create_at": date("2013-05-28"), "description": "my psn: Puckett1980 @KAY_nadian ✊", "followers_count": 929, "friends_count": 1162, "statues_count": 26342 }, "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("2015-11-27T10:00:36.000Z"), "id": 670301211090751490, "text": "Accident, left lane blocked in #Bloomfield on The Garden State Pkwy SB at The Essex Tolls, stop and go traffic back to X153, delay of 7 mins", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.1843,40.8046"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Bloomfield" }}, "user": { "id": 42640432, "name": "Total Traffic NYC", "screen_name": "TotalTrafficNYC", "lang": "en", "location": "New York City & New Jersey", "create_at": date("2009-05-26"), "description": "Total Traffic Network delivers real-time traffic via in-car and portable navigation, broadcast media, wireless, & Internet. Call Tips Hands-Free to 212-377-7757", "followers_count": 1978, "friends_count": 366, "statues_count": 303764 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, NJ", "id": "dc4ad595a56393ac", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-74.211974,40.76822 -74.162582,40.848801") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex" } }
+{ "create_at": datetime("2015-11-27T10:00:37.000Z"), "id": 670301211560685568, "text": "I love Santa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1609709232, "name": "claire sherfick", "screen_name": "claire_sherfick", "lang": "en", "location": "317", "create_at": date("2013-07-20"), "description": "GUERIN", "followers_count": 321, "friends_count": 239, "statues_count": 1672 }, "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("2015-11-27T10:00:37.000Z"), "id": 670301211581685761, "text": "Going to altitude today ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 259011718, "name": "Şɧąɛ ❕", "screen_name": "QueenBShae_", "lang": "en", "location": "snapchat : its_shaej", "create_at": date("2011-02-28"), "description": "who knew from the concrete , a flower would grow ‼️ #GSU19", "followers_count": 2090, "friends_count": 2301, "statues_count": 41427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rayville, LA", "id": "3cd57561107cd4e6", "name": "Rayville", "place_type": "city", "bounding_box": rectangle("-91.799664,32.449489 -91.733274,32.486038") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22083, "countyName": "Richland", "cityID": 2263680, "cityName": "Rayville" } }
+{ "create_at": datetime("2015-11-27T10:00:37.000Z"), "id": 670301211703144450, "text": "We're #hiring! Click to apply: Interim Director of Nursing - DON - Registered Nurse - RN - LTC - SNF - https://t.co/duT2kAhgtl #Boston, MA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.0597732,42.3584308"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Boston" }}, "user": { "id": 67911583, "name": "Clinical Resources ", "screen_name": "ClinRes", "lang": "en", "location": "Atlanta, Georgia", "create_at": date("2009-08-22"), "description": "A healthcare staffing and professional services firm operating nationwide! / RTs ≠ endorsements / Call us at 404-343-7227! View Jobs: http://t.co/1VWhFiBPTI", "followers_count": 330, "friends_count": 438, "statues_count": 2482 }, "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("2015-11-27T10:00:37.000Z"), "id": 670301211774468096, "text": "Some niggah in line wearing ski googles��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 219625957, "name": "Brown Iverson", "screen_name": "circOnutz", "lang": "en", "location": "snapchat braddahcircs", "create_at": date("2010-11-25"), "description": "null", "followers_count": 285, "friends_count": 333, "statues_count": 11505 }, "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("2015-11-27T10:00:37.000Z"), "id": 670301211808010240, "text": "#Hospitality #Job: Barista (US) at Starbucks https://t.co/9uaVxeuvwN #Veterans #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.2266757,43.1370125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "Job", "Veterans", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 99429565, "name": "TMJ-NYS HRTA Jobs", "screen_name": "tmj_NYS_HRTA", "lang": "en", "location": "Syracuse, NY", "create_at": date("2009-12-25"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Syracuse, NY from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 284, "friends_count": 273, "statues_count": 94 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clay, NY", "id": "00b5e57f592b7f62", "name": "Clay", "place_type": "city", "bounding_box": rectangle("-76.283112,43.115366 -76.159096,43.210789") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36067, "countyName": "Onondaga" } }
+{ "create_at": datetime("2015-11-27T10:00:37.000Z"), "id": 670301212760117248, "text": "This whole situation is cray but it just goes back to the situation with Sharon, Ashley and Faith yrs ago smh https://t.co/ZXW9HEnmcX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 143208958, "name": "*Mrs_AshCakes2u*", "screen_name": "Ms_OnlyMee", "lang": "en", "location": "Carter Lake", "create_at": date("2010-05-12"), "description": "Mother of *1*, Wife.....College Graduate.....POSITIVE working woman...Most importantly lover of GOD!! ~Education NeRd~ °•Libra_Lady•° ♥TheMRS♥ #TeamFollowBack", "followers_count": 270, "friends_count": 546, "statues_count": 15192 }, "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("2015-11-27T10:00:37.000Z"), "id": 670301212856737792, "text": "��mmm who we know? @_LITTLEKIWI https://t.co/plDiHZdBSM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 426971582 }}, "user": { "id": 428337120, "name": "AlmightyJer ✨", "screen_name": "jerrica_tk", "lang": "en", "location": "null", "create_at": date("2011-12-04"), "description": "Chief keef♥️ #SosaHive | Instagram: jerrica.tk | Snapchat: x_jerrica |", "followers_count": 1395, "friends_count": 864, "statues_count": 55982 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beverly Hills, MI", "id": "3d8f3aedb07d9ef7", "name": "Beverly Hills", "place_type": "city", "bounding_box": rectangle("-83.274708,42.50851 -83.203754,42.531894") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2608160, "cityName": "Beverly Hills" } }
+{ "create_at": datetime("2015-11-27T10:00:37.000Z"), "id": 670301212961603584, "text": "@AnthonyBallstar ����❤️❤️", "in_reply_to_status": 670300104021180416, "in_reply_to_user": 1529551106, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1529551106 }}, "user": { "id": 955695434, "name": "Cali", "screen_name": "CCaligrl3", "lang": "en", "location": "null", "create_at": date("2012-11-18"), "description": "idc", "followers_count": 1776, "friends_count": 693, "statues_count": 22247 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bayonet Point, FL", "id": "2d1b0aee7297f004", "name": "Bayonet Point", "place_type": "city", "bounding_box": rectangle("-82.733975,28.299909 -82.642106,28.350175") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12101, "countyName": "Pasco", "cityID": 1204162, "cityName": "Bayonet Point" } }
+{ "create_at": datetime("2015-11-27T10:00:37.000Z"), "id": 670301212961603585, "text": "It's like they knew I was coming! #KCLT https://t.co/vTA9TqUrKj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.9453129,35.2207946"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "KCLT" }}, "user": { "id": 185709045, "name": "Bel Geode (Drew B)", "screen_name": "belgeode", "lang": "en", "location": "Pittsburgh PA", "create_at": date("2010-09-01"), "description": "What is this I don't even... Gamer Geek, Bassist, Lover of Playmates 1987, Artist and Writer... Youtuber. http://www.twitch.tv/belgeode is me.", "followers_count": 395, "friends_count": 422, "statues_count": 11206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bad Daddy's Burger Bar", "id": "07d9d5435d882000", "name": "Bad Daddy's Burger Bar", "place_type": "poi", "bounding_box": rectangle("-80.94254509999999,35.2192759 -80.942545,35.219276") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2015-11-27T10:00:37.000Z"), "id": 670301213120942080, "text": "@ChiAli_ my messages not delivering lol", "in_reply_to_status": 670301155675742208, "in_reply_to_user": 356222118, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 356222118 }}, "user": { "id": 77111443, "name": "Birthday Girl♐️", "screen_name": "jaixo__", "lang": "en", "location": "610/215", "create_at": date("2009-09-24"), "description": "got you stuck off the realness❄️ SC: Viva_Jai", "followers_count": 2219, "friends_count": 1158, "statues_count": 73985 }, "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-27T10:00:37.000Z"), "id": 670301213464899584, "text": "@danno919 @mmcr127 dude I'm telling you they drive so awesome I'm driving mine around as a demo right now", "in_reply_to_status": 670258229159575552, "in_reply_to_user": 36928714, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 36928714, 274277342 }}, "user": { "id": 31392117, "name": "matt lasco", "screen_name": "carguy915", "lang": "en", "location": "Michigan ", "create_at": date("2009-04-15"), "description": "Father to Maxx Speed Huz to jamie lasco MotoX4life / Vice President Auto group / President The acuity group, And the real king of cars !!", "followers_count": 786, "friends_count": 415, "statues_count": 1987 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fenton, MI", "id": "d1e16b409e42e6df", "name": "Fenton", "place_type": "city", "bounding_box": rectangle("-83.80635,42.75267 -83.686497,42.841079") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2627760, "cityName": "Fenton" } }
+{ "create_at": datetime("2015-11-27T10:00:37.000Z"), "id": 670301213515235333, "text": "@zeb1904 @IvyGrey124 so Minnie we tie him to pole and take everything cause that hoodie is swaggie as fudge", "in_reply_to_status": 670300449027850240, "in_reply_to_user": 3194786389, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3194786389, 465270167 }}, "user": { "id": 433697853, "name": "❥ Minnie's Boyfriend", "screen_name": "SmileyGhee_Cx", "lang": "en", "location": "hell", "create_at": date("2011-12-10"), "description": "better off dead.", "followers_count": 344, "friends_count": 178, "statues_count": 10026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waukegan, IL", "id": "b819c5d90b780b57", "name": "Waukegan", "place_type": "city", "bounding_box": rectangle("-87.96368,42.305624 -87.802772,42.431936") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1779293, "cityName": "Waukegan" } }
+{ "create_at": datetime("2015-11-27T10:00:37.000Z"), "id": 670301213544595457, "text": "Child Advocacy or Helicopter Parenting? https://t.co/GWNfAHiTX9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.79470062,32.83620071"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20880486, "name": "Park Cities, TX", "screen_name": "BubbleLife", "lang": "en", "location": "Highland & University Park, TX", "create_at": date("2009-02-14"), "description": "Park Cities, TX (Highland Park & University Park) community news, photos and events. Add your own community news, photos and events using iReporter.", "followers_count": 1231, "friends_count": 2178, "statues_count": 19819 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland Park, TX", "id": "a5e0ff90387af9a1", "name": "Highland Park", "place_type": "city", "bounding_box": rectangle("-96.819215,32.818822 -96.780242,32.843955") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4833824, "cityName": "Highland Park" } }
+{ "create_at": datetime("2015-11-27T10:00:37.000Z"), "id": 670301213737529344, "text": "@sam0ns ��", "in_reply_to_status": 670301096653533184, "in_reply_to_user": 218676744, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 218676744 }}, "user": { "id": 3980761311, "name": "Daddy", "screen_name": "cameronlalong", "lang": "en", "location": "Tampa, FL", "create_at": date("2015-10-17"), "description": "Kassy", "followers_count": 139, "friends_count": 263, "statues_count": 2260 }, "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("2015-11-27T10:00:37.000Z"), "id": 670301214295392256, "text": "Someone do me a huge favor for gas money please", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1696709486, "name": "Sami", "screen_name": "sami_saucedo21", "lang": "en", "location": "Trust all of those instincts. ", "create_at": date("2013-08-24"), "description": "1.6.14 ❤️ Mitch W", "followers_count": 680, "friends_count": 654, "statues_count": 12555 }, "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("2015-11-27T10:00:37.000Z"), "id": 670301214442041344, "text": "Good morning.....��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.7436427,32.469282"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3237278198, "name": "Alex", "screen_name": "anonymous152436", "lang": "en", "location": "null", "create_at": date("2015-06-05"), "description": "@2_onBieber co owns with me!!!! Follow her http://twitter.com/TheGabbieShow/…", "followers_count": 698, "friends_count": 334, "statues_count": 18664 }, "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("2015-11-27T10:00:37.000Z"), "id": 670301214509129728, "text": "If you're a #Nursing professional in #Olympia, WA, check out this #job: https://t.co/55lVCZKiV9 #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.89,47.04"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "Olympia", "job", "Hiring", "CareerArc" }}, "user": { "id": 120647454, "name": "TMJ-WAT Nursing Jobs", "screen_name": "tmj_WAT_NURSING", "lang": "en", "location": "Tacoma, WA", "create_at": date("2010-03-06"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Tacoma, WA. Need help? Tweet us at @CareerArc!", "followers_count": 386, "friends_count": 282, "statues_count": 342 }, "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("2015-11-27T10:00:37.000Z"), "id": 670301214605733888, "text": "@heyjenbartel its so funny seeing these posted, because people are like \"look how bad i was\" but i can see the seeds of brilliance in them.", "in_reply_to_status": 670037373313511426, "in_reply_to_user": 45881767, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 45881767 }}, "user": { "id": 81623503, "name": "Jessi Sheron", "screen_name": "JessiSheron", "lang": "en", "location": "Gaithersburg, MD", "create_at": date("2009-10-11"), "description": "Comic book artist. I'm pretty darn rad. the link below is for my art! my email is Jessisheron@gmail.com", "followers_count": 2310, "friends_count": 1085, "statues_count": 34028 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Potomac, MD", "id": "4374fee986656d2f", "name": "North Potomac", "place_type": "city", "bounding_box": rectangle("-77.26774,39.078493 -77.194035,39.118378") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2456875, "cityName": "North Potomac" } }
+{ "create_at": datetime("2015-11-27T10:00:37.000Z"), "id": 670301214958055424, "text": "Local Art Reviews & Videos Nov. 26 https://t.co/QwVZkqJBFR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.79470062,32.83620071"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20880486, "name": "Park Cities, TX", "screen_name": "BubbleLife", "lang": "en", "location": "Highland & University Park, TX", "create_at": date("2009-02-14"), "description": "Park Cities, TX (Highland Park & University Park) community news, photos and events. Add your own community news, photos and events using iReporter.", "followers_count": 1231, "friends_count": 2178, "statues_count": 19820 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland Park, TX", "id": "a5e0ff90387af9a1", "name": "Highland Park", "place_type": "city", "bounding_box": rectangle("-96.819215,32.818822 -96.780242,32.843955") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4833824, "cityName": "Highland Park" } }
+{ "create_at": datetime("2015-11-27T10:00:37.000Z"), "id": 670301215138250754, "text": "@fox4kc so sad, hopefully they caught it in time", "in_reply_to_status": 670299981975191552, "in_reply_to_user": 6817432, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 6817432 }}, "user": { "id": 3309954102, "name": "Kathleen Seymour", "screen_name": "kathleen66481", "lang": "en", "location": "null", "create_at": date("2015-08-08"), "description": "Happy mom with happy family", "followers_count": 36, "friends_count": 195, "statues_count": 83 }, "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("2015-11-27T10:00:37.000Z"), "id": 670301215306199040, "text": "swag it out https://t.co/wcGZfcjzXI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1232727606, "name": "alyssa ;", "screen_name": "niazisgod", "lang": "en", "location": "#blacklivesmatter", "create_at": date("2013-03-01"), "description": "null", "followers_count": 2498, "friends_count": 2858, "statues_count": 8847 }, "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("2015-11-27T10:00:38.000Z"), "id": 670301215725649921, "text": "1 more hour for #FC3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FC3" }}, "user": { "id": 756182094, "name": "Michel 2 Times ✌️", "screen_name": "Michelnumba9", "lang": "en", "location": "null", "create_at": date("2012-08-13"), "description": "Lightskin living lavish . blessed . 401 thats that 02860 ⛽️ #9️⃣", "followers_count": 390, "friends_count": 303, "statues_count": 798 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pawtucket, RI", "id": "47623c4cf6ca7957", "name": "Pawtucket", "place_type": "city", "bounding_box": rectangle("-71.422117,41.856092 -71.333899,41.8984") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4454640, "cityName": "Pawtucket" } }
+{ "create_at": datetime("2015-11-27T10:00:38.000Z"), "id": 670301215901802496, "text": "2015 isn't over yet theres still about 10 good Ls left for me to suffer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1302109452, "name": "Merry Christmas", "screen_name": "CampaignNoah", "lang": "en", "location": "Akron, OH", "create_at": date("2013-03-25"), "description": "Time Flies", "followers_count": 689, "friends_count": 338, "statues_count": 54057 }, "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("2015-11-27T10:00:38.000Z"), "id": 670301216228831232, "text": "See a virtual tour of our listing on 50 Hidden cove Dr #Valparaiso #FL https://t.co/07Eg9qmKLW #realestate https://t.co/W1wuPXpKy8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.501468,30.503412"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Valparaiso", "FL", "realestate" }}, "user": { "id": 78043094, "name": "ERA American", "screen_name": "ERAAmerican", "lang": "en", "location": "Fort Walton Beach, Florida", "create_at": date("2009-09-28"), "description": "Changing lives through real estate since 1981. #ERAamerican #ERAlife #ERAjobs", "followers_count": 340, "friends_count": 287, "statues_count": 6605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valparaiso, FL", "id": "01c3fa285ac61b1d", "name": "Valparaiso", "place_type": "city", "bounding_box": rectangle("-86.529249,30.455681 -86.47863,30.535059") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1273675, "cityName": "Valparaiso" } }
+{ "create_at": datetime("2015-11-27T10:00:38.000Z"), "id": 670301216367185920, "text": "Spirit lead me where my trust is without borders", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2716068389, "name": "Juliet rose", "screen_name": "Juliet_Rose01", "lang": "en", "location": "San Antonio, TX", "create_at": date("2014-07-17"), "description": "blessed.", "followers_count": 736, "friends_count": 348, "statues_count": 9922 }, "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-27T10:00:38.000Z"), "id": 670301216371552258, "text": "Playoffs Week 3 Waco Area Matchups https://t.co/fWe0wf5Z0F", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.79470062,32.83620071"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20880486, "name": "Park Cities, TX", "screen_name": "BubbleLife", "lang": "en", "location": "Highland & University Park, TX", "create_at": date("2009-02-14"), "description": "Park Cities, TX (Highland Park & University Park) community news, photos and events. Add your own community news, photos and events using iReporter.", "followers_count": 1231, "friends_count": 2178, "statues_count": 19821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland Park, TX", "id": "a5e0ff90387af9a1", "name": "Highland Park", "place_type": "city", "bounding_box": rectangle("-96.819215,32.818822 -96.780242,32.843955") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4833824, "cityName": "Highland Park" } }
+{ "create_at": datetime("2015-11-27T10:00:38.000Z"), "id": 670301216400924676, "text": "@ryanschafer26 this tweet is so gay", "in_reply_to_status": 670299329886859264, "in_reply_to_user": 280323873, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 280323873 }}, "user": { "id": 801146822, "name": "cash", "screen_name": "ashlyne34", "lang": "en", "location": "prolly w holl, lex, syd & ab ", "create_at": date("2012-09-03"), "description": "Harvard 20'", "followers_count": 1470, "friends_count": 488, "statues_count": 15889 }, "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-27T10:00:38.000Z"), "id": 670301216493060097, "text": "Interested in a #Nursing #job near #Westfield, MA? This could be a great fit: https://t.co/lHMxsqvnk1 #RN #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.7592109,42.11807"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "Westfield", "RN", "Hiring" }}, "user": { "id": 2688167298, "name": "Noble Hospital Jobs", "screen_name": "JobsatNobleHosp", "lang": "en", "location": "Westfield, MA", "create_at": date("2014-07-28"), "description": "Noble Hospital is a regional community hospital, providing a broad range of expert services to Greater Westfield and surrounding areas.", "followers_count": 14, "friends_count": 0, "statues_count": 124 }, "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("2015-11-27T10:00:38.000Z"), "id": 670301216669171713, "text": "When you look at John abugahtass thanksgiving picture��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1250534689, "name": "TheOutSpokenClozPerv", "screen_name": "bSunRize", "lang": "en", "location": "null", "create_at": date("2013-03-07"), "description": "A melancholic comedian who plans to change the world. One grammatically incorrect step at a time.", "followers_count": 108, "friends_count": 146, "statues_count": 1403 }, "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("2015-11-27T10:00:38.000Z"), "id": 670301216799232000, "text": "Why is my setup so pretty? https://t.co/NRs33EC8O7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119559561, "name": "Melvin", "screen_name": "JamesLongoria", "lang": "en", "location": "null", "create_at": date("2010-03-03"), "description": "Loozyana, magical cards. PC gaming superior. Check out the stream.", "followers_count": 159, "friends_count": 326, "statues_count": 1459 }, "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("2015-11-27T10:00:38.000Z"), "id": 670301216874721280, "text": "When someone asks me if I want another drink https://t.co/A8c6QanMvD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 248481997, "name": "Noah George", "screen_name": "NoahGeorge1", "lang": "en", "location": "Champaign", "create_at": date("2011-02-06"), "description": "Pilot. Sports fanatic. Dude", "followers_count": 419, "friends_count": 340, "statues_count": 19177 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hamburg, PA", "id": "00dd27f51c5d2551", "name": "West Hamburg", "place_type": "city", "bounding_box": rectangle("-76.018402,40.538581 -75.98426,40.572249") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42011, "countyName": "Berks", "cityID": 4283120, "cityName": "West Hamburg" } }
+{ "create_at": datetime("2015-11-27T10:00:38.000Z"), "id": 670301216891473920, "text": "I think I'm going to get my septum repierced", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 982491655, "name": "simply stellaaa✨", "screen_name": "dopeout_somdah", "lang": "en", "location": "| HIP Phat | HTX |", "create_at": date("2012-12-01"), "description": "| The Lord is my savior | #24 | SHSU'19 |", "followers_count": 917, "friends_count": 663, "statues_count": 27446 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.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-27T10:00:38.000Z"), "id": 670301217231347712, "text": "@serendipitydc1 @thebeautyfiles2 @jazzcoffeehouse live tonite @8:00pm serendipity Jazz Coffee House https://t.co/RZju4WulWl", "in_reply_to_status": -1, "in_reply_to_user": 3432015982, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3432015982, 949870724, 3323742406, 2314519681 }}, "user": { "id": 3431976387, "name": "Juanita Dillard", "screen_name": "facejobmedia", "lang": "en", "location": "null", "create_at": date("2015-08-19"), "description": "null", "followers_count": 26, "friends_count": 84, "statues_count": 66 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheverly, MD", "id": "77a6c282d3f41d1d", "name": "Cheverly", "place_type": "city", "bounding_box": rectangle("-76.929458,38.912321 -76.895956,38.939587") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2416550, "cityName": "Cheverly" } }
+{ "create_at": datetime("2015-11-27T10:00:38.000Z"), "id": 670301217311088641, "text": "@Aerionnnnn no tf you don't��", "in_reply_to_status": 670296398760419328, "in_reply_to_user": 2855110745, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2855110745 }}, "user": { "id": 2763662669, "name": "Tor-Tor & Ni-Ni✨", "screen_name": "TorionnC", "lang": "en", "location": "Clev. ", "create_at": date("2014-09-06"), "description": "D.", "followers_count": 498, "friends_count": 420, "statues_count": 25558 }, "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("2015-11-27T10:00:38.000Z"), "id": 670301217369657344, "text": "Sioux Falls SD Police Department not just steal IP from women it aggravate sex harass worse stole program designed for military veterans.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 579163841, "name": "Observation Team1", "screen_name": "walkwithmany", "lang": "en", "location": "Sioux Falls SD", "create_at": date("2012-05-13"), "description": "Don't be pushed by your problems; be led by your dreams - author unknown", "followers_count": 4780, "friends_count": 4925, "statues_count": 126218 }, "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("2015-11-27T10:00:38.000Z"), "id": 670301217508036608, "text": "i just want my big face rose gold MK watch ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2722476915, "name": "☯", "screen_name": "hn_carter09", "lang": "en", "location": "Port Arthur, TX", "create_at": date("2014-07-23"), "description": "May God bless the woman deep within me..the woman I'm trying to be.", "followers_count": 736, "friends_count": 799, "statues_count": 17379 }, "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("2015-11-27T10:00:38.000Z"), "id": 670301217793269760, "text": "Want to work in #LakeForest, CA? View our latest opening: https://t.co/slX0CRU9qY #SupplyChain #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.689218,33.6469661"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LakeForest", "SupplyChain", "Job", "Jobs", "Hiring" }}, "user": { "id": 870560696, "name": "SpectrumBrandsJobs", "screen_name": "WorkAtSpectrum", "lang": "en", "location": "null", "create_at": date("2012-10-09"), "description": "Spectrum Brands hires and rewards great people who understand the importance of a strong culture that values integrity, commitment, teamwork, and diversity.", "followers_count": 278, "friends_count": 6, "statues_count": 2598 }, "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("2015-11-27T10:00:38.000Z"), "id": 670301217851990017, "text": "#security #Job in #Indianapolis, IN: Lead Parental Escort at Simon https://t.co/DaSQGg8ZKL #Simon #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.1580423,39.7683765"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "security", "Job", "Indianapolis", "Simon", "Jobs", "Hiring" }}, "user": { "id": 118581799, "name": "TMJ-IND Security Job", "screen_name": "tmj_IND_secure", "lang": "en", "location": "Indianapolis, IN", "create_at": date("2010-02-28"), "description": "Follow this account for geo-targeted Security job tweets in Indianapolis, IN from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 287, "friends_count": 263, "statues_count": 13 }, "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("2015-11-27T10:00:38.000Z"), "id": 670301218078629888, "text": "mall any1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 749589260, "name": "jazzy", "screen_name": "simplyjazzy__", "lang": "en", "location": "Pleasantville, NY", "create_at": date("2012-08-10"), "description": "searching for a reason to care. Pace Soccer 19'", "followers_count": 1369, "friends_count": 914, "statues_count": 112300 }, "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-27T10:00:38.000Z"), "id": 670301218376413185, "text": "@StaceGots lol", "in_reply_to_status": 670299316091752448, "in_reply_to_user": 17742722, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 17742722 }}, "user": { "id": 47534846, "name": "janice", "screen_name": "chestnuthell", "lang": "en", "location": "boston", "create_at": date("2009-06-15"), "description": "There are only two tragedies in life: one is not getting what one wants, and the other is getting it. - Oscar Wilde (1854-1900)", "followers_count": 2323, "friends_count": 2437, "statues_count": 144622 }, "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("2015-11-27T10:00:38.000Z"), "id": 670301218414206976, "text": "The amount of candles I got from Bath and Body Works today is unhealthy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3296234443, "name": "Brianna Caridi", "screen_name": "brianna_caridi", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2015-07-26"), "description": ":): BCHSvolleyball", "followers_count": 93, "friends_count": 133, "statues_count": 113 }, "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("2015-11-27T10:00:38.000Z"), "id": 670301218565156865, "text": "There's Winter LIVE! https://t.co/XmhfMQB49W", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4260471815, "name": "Klarysa Kaseman", "screen_name": "KlarysaKaseman", "lang": "en", "location": "null", "create_at": date("2015-11-23"), "description": "Andy Grammer. Dancing With The Stars. Cozi Zuehlsdorff. Sabrina Carpenter. Stewie. @AndyMyers202 DOLPHINS! Winter And Hope. CMA. Always here. Food. Follow me ;)", "followers_count": 41, "friends_count": 213, "statues_count": 291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Exeter, NH", "id": "0a5eca4e60fe6f35", "name": "Exeter", "place_type": "city", "bounding_box": rectangle("-71.001194,42.956811 -70.887799,42.997041") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33015, "countyName": "Rockingham", "cityID": 3325300, "cityName": "Exeter" } }
+{ "create_at": datetime("2015-11-27T10:00:38.000Z"), "id": 670301218598756352, "text": "Once I Lose Interest We Can Never Be Nothing...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2331168103, "name": "@Mulah.__", "screen_name": "red_almightyz", "lang": "en", "location": "where you wanna be", "create_at": date("2014-02-06"), "description": "Baton Rouge To Houston✈️", "followers_count": 598, "friends_count": 867, "statues_count": 6363 }, "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("2015-11-27T10:00:38.000Z"), "id": 670301218640531458, "text": "' see it's getting intense ' ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2915966244, "name": "Birthday Boy !", "screen_name": "vinny_vin__10", "lang": "en", "location": "null", "create_at": date("2014-12-01"), "description": "happily taken", "followers_count": 218, "friends_count": 273, "statues_count": 4813 }, "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-27T10:00:38.000Z"), "id": 670301218737123328, "text": "A string on my finger to remind my Sophanistas: coupon code USAmade for 40% off all online and studio… https://t.co/F8yuFVTR16", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.1193314,41.7531204"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 32424083, "name": "Sophia Forero", "screen_name": "sophiaforero", "lang": "en", "location": "Naperville, IL", "create_at": date("2009-04-17"), "description": "Mosaic jeweler #feelbeautiful", "followers_count": 362, "friends_count": 323, "statues_count": 4229 }, "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("2015-11-27T10:00:38.000Z"), "id": 670301219139620864, "text": "New Arrivals at Neptunes!! #emoji #newarrivals #happyemoji @ Neptunes Kids Swimwear & Women Boutique https://t.co/LWvmUX06uv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.3108978,25.7265339"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "emoji", "newarrivals", "happyemoji" }}, "user": { "id": 269538383, "name": "Joan", "screen_name": "neptuneskidsswi", "lang": "en", "location": "Miami", "create_at": date("2011-03-20"), "description": "null", "followers_count": 160, "friends_count": 248, "statues_count": 8339 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glenvar Heights, FL", "id": "f546db03397f6e09", "name": "Glenvar Heights", "place_type": "city", "bounding_box": rectangle("-80.334426,25.690519 -80.292923,25.733912") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1226100, "cityName": "Glenvar Heights" } }
+{ "create_at": datetime("2015-11-27T10:00:38.000Z"), "id": 670301219206877184, "text": "sucks being me sometimes...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1390633512, "name": "• mel •", "screen_name": "melindatnie", "lang": "en", "location": "Raleigh, NC", "create_at": date("2013-04-29"), "description": "be heartless or be heartbroken", "followers_count": 340, "friends_count": 205, "statues_count": 11443 }, "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("2015-11-27T10:00:39.000Z"), "id": 670301219739574273, "text": "@JMRamos1701D John I know you love Rush but when are you going to get on the @LuciferonFOX bandwagon!! @tomellis17 is so fab in it.", "in_reply_to_status": -1, "in_reply_to_user": 68698703, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 68698703, 3060210854, 106753602 }}, "user": { "id": 2722866009, "name": "Jules", "screen_name": "JulieMcgonagle", "lang": "en", "location": "Kenmore, NY", "create_at": date("2014-07-24"), "description": "Converse wearing,chocolate chomping,geek in glasses. wishing @TomEllis17 a happy birthday month. TomEgirl LuciFan DonnieGirl BritroyalBabe Whovian", "followers_count": 914, "friends_count": 1015, "statues_count": 23091 }, "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("2015-11-27T10:00:39.000Z"), "id": 670301219949305857, "text": "Happy Black Friday ...... I Guess ������������ @ Latina 100.3 FM Radio https://t.co/0cyxLCBxTh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.4051819,41.8035202"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18014545, "name": "djtravieso007", "screen_name": "DjTravieso007", "lang": "en", "location": "Providence ri ", "create_at": date("2008-12-09"), "description": "Locutor de radio & dj , Producer , Mc , Promoter", "followers_count": 858, "friends_count": 666, "statues_count": 4390 }, "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-27T10:00:39.000Z"), "id": 670301220079312901, "text": "@theoriginalpous we still are ��", "in_reply_to_status": 670283139286069249, "in_reply_to_user": 1489552002, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1489552002 }}, "user": { "id": 615868898, "name": "Sarah Arnaud", "screen_name": "sarah_arnaud", "lang": "en", "location": "null", "create_at": date("2012-06-22"), "description": "null", "followers_count": 452, "friends_count": 247, "statues_count": 5221 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Peachtree City, GA", "id": "fab01c4d14587744", "name": "Peachtree City", "place_type": "city", "bounding_box": rectangle("-84.626167,33.335655 -84.522236,33.449361") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13113, "countyName": "Fayette", "cityID": 1359724, "cityName": "Peachtree City" } }
+{ "create_at": datetime("2015-11-27T10:00:39.000Z"), "id": 670301220196626432, "text": "I have so much clothes it's retarded", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 410180270, "name": "Khalil O", "screen_name": "Khalildaddy", "lang": "en", "location": "Fresno//Sacramento Trapartment", "create_at": date("2011-11-11"), "description": "What's Gerber. Palestinian and Egyptian", "followers_count": 1146, "friends_count": 789, "statues_count": 59995 }, "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-27T10:00:39.000Z"), "id": 670301220247072770, "text": "@MpacTheGreat my cribbo, I'm about to head to the field house in like 15mins though", "in_reply_to_status": 670300958866448385, "in_reply_to_user": 517884639, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 517884639 }}, "user": { "id": 2280701306, "name": "saucyJP", "screen_name": "JP1_Era", "lang": "en", "location": "slums of suwanee", "create_at": date("2014-01-07"), "description": "Dab Connoisseur #FreeTee 2Love #UglyNiggasOnTheComeUp", "followers_count": 878, "friends_count": 692, "statues_count": 19823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Suwanee, GA", "id": "7b01e60536219287", "name": "Suwanee", "place_type": "city", "bounding_box": rectangle("-84.107404,34.001413 -84.024808,34.096528") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13135, "countyName": "Gwinnett", "cityID": 1374936, "cityName": "Suwanee" } }
+{ "create_at": datetime("2015-11-27T10:00:39.000Z"), "id": 670301220305821696, "text": "Listen to Disturbed - Forsaken by user699384 #np on #SoundCloud\nhttps://t.co/sosltQqy4e is Danielle's troll boy Xmas https://t.co/JrbcgNG63O", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "np", "SoundCloud" }}, "user": { "id": 4137128894, "name": "Henry Fairchild", "screen_name": "moodkiller113", "lang": "en", "location": "1 foot in hell 1 in heaven!", "create_at": date("2015-11-05"), "description": "the person your mother warn you about!", "followers_count": 814, "friends_count": 1645, "statues_count": 2275 }, "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("2015-11-27T10:00:39.000Z"), "id": 670301220637044736, "text": "@slimcurls u must been having a bad morning", "in_reply_to_status": 670301009126625281, "in_reply_to_user": 1047286484, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1047286484 }}, "user": { "id": 983457463, "name": "Dougie", "screen_name": "__its__dougie", "lang": "en", "location": "Fairfield Ca ", "create_at": date("2012-12-01"), "description": "null", "followers_count": 194, "friends_count": 279, "statues_count": 798 }, "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("2015-11-27T10:00:39.000Z"), "id": 670301220855136256, "text": "Join the Guckenheimer team! See our latest #Hospitality #job opening here: https://t.co/aF265OgvVJ #SouthSanFrancisco, CA #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.4077498,37.654656"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "SouthSanFrancisco", "Hiring" }}, "user": { "id": 731027906, "name": "Guckenheimer Careers", "screen_name": "GuckenheimerJob", "lang": "en", "location": "null", "create_at": date("2012-08-01"), "description": "We are a high end food service company providing restaurant quality food in corporate settings. We are looking to fill a variety of positions across the US.", "followers_count": 131, "friends_count": 2, "statues_count": 336 }, "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("2015-11-27T10:00:39.000Z"), "id": 670301220960096256, "text": "Headed to #southcoastplaza to band out my guy @DRESINATRA got the mix going up right now #92.3 #lashit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "southcoastplaza", "lashit" }}, "user_mentions": {{ 19584818 }}, "user": { "id": 1744516554, "name": "Jiovanni Major Jett", "screen_name": "THEREALGEEMAJOR", "lang": "en", "location": "los angeles", "create_at": date("2013-09-07"), "description": "arranger | Songwriter | Engineer .....Just a tuned soul making hits ...No telling where im headed #everytimeyouhearmymusic #MajorArtistent", "followers_count": 5056, "friends_count": 1206, "statues_count": 6187 }, "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("2015-11-27T10:00:39.000Z"), "id": 670301221266153472, "text": "Bitches In The Section.... \nAll They Wanna Do... \nFuck A Real Rich Nigga In The Crew!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 228141887, "name": "❄️PRE$$URE❄️", "screen_name": "Mr_IceWater", "lang": "en", "location": "Pink Hole Palace ", "create_at": date("2010-12-18"), "description": "21 ⛄️.. #ISwear Thi$ Pre$$ure Life$tyle Ain't For Everybody!! #75BlockBoy #ONTHEP", "followers_count": 904, "friends_count": 358, "statues_count": 119451 }, "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-27T10:00:39.000Z"), "id": 670301221442486272, "text": "Fuck u lol https://t.co/5VBXxn6a6o", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 121286525, "name": "FlawlessBxtch", "screen_name": "MushyBaybee", "lang": "en", "location": "Houstalantavegas", "create_at": date("2010-03-08"), "description": "Good Enough To Eat ** ;)", "followers_count": 636, "friends_count": 363, "statues_count": 11291 }, "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("2015-11-27T10:00:39.000Z"), "id": 670301221719310336, "text": "@01bhudes luv you Hudey :)))", "in_reply_to_status": 670285100215570437, "in_reply_to_user": 940006848, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 940006848 }}, "user": { "id": 1452517638, "name": "Nick Chillbae", "screen_name": "N_Kilbane30", "lang": "en", "location": "CLE | 420 11 |", "create_at": date("2013-05-23"), "description": "Purveyor of Island Lifestyles... call me goat Ignatius Football '17 Mack Pack #32", "followers_count": 446, "friends_count": 325, "statues_count": 2019 }, "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("2015-11-27T10:00:39.000Z"), "id": 670301221836615680, "text": "the light of my life https://t.co/6oMK8UJE1B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2466490854, "name": "laney", "screen_name": "DoppDelaney", "lang": "en", "location": "Scottsdale, AZ", "create_at": date("2014-04-27"), "description": "null", "followers_count": 289, "friends_count": 288, "statues_count": 372 }, "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("2015-11-27T10:00:39.000Z"), "id": 670301222113517568, "text": "@Theyh8kyle, What you Do ����", "in_reply_to_status": 670244731499233280, "in_reply_to_user": 827092626, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 827092626 }}, "user": { "id": 313432823, "name": ", ✨", "screen_name": "___Miyahhh", "lang": "en", "location": "Bestfriend was here✌", "create_at": date("2011-06-08"), "description": "His Girlfriend Sent You Here Right ? ' Lil Jeff❤My Everything 8.12.14 ❤️", "followers_count": 935, "friends_count": 592, "statues_count": 31383 }, "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-27T10:00:39.000Z"), "id": 670301222222602240, "text": "@CierraStoner51 Sounds like a good plan....", "in_reply_to_status": 670274567122763776, "in_reply_to_user": 327552187, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 327552187 }}, "user": { "id": 425252107, "name": "TERESA STONER", "screen_name": "TERESASTONER", "lang": "en", "location": "null", "create_at": date("2011-11-30"), "description": "null", "followers_count": 39, "friends_count": 78, "statues_count": 176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cavetown, MD", "id": "7423dca579d08f8d", "name": "Cavetown", "place_type": "city", "bounding_box": rectangle("-77.623937,39.634946 -77.564356,39.656319") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24043, "countyName": "Washington", "cityID": 2414225, "cityName": "Cavetown" } }
+{ "create_at": datetime("2015-11-27T10:00:39.000Z"), "id": 670301222302150657, "text": "4eva gr8ful 4 this big boogie right here.don't know wth I'd do without ya��thanks 4 everything!@PrincessHolivia https://t.co/tZLvi9gTv5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3022526883, "name": "lex", "screen_name": "alexacunat", "lang": "en", "location": "null", "create_at": date("2015-02-06"), "description": "livin'||FL", "followers_count": 218, "friends_count": 192, "statues_count": 1400 }, "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("2015-11-27T10:00:39.000Z"), "id": 670301222318936064, "text": "Interested in a #job near #Portland, OR? This could be a great fit: https://t.co/QdHrltdxJr #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.6762071,45.5234515"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Portland", "Hiring", "CareerArc" }}, "user": { "id": 38053186, "name": "TMJ-PDX Jobs", "screen_name": "tmj_pdx_jobs", "lang": "en", "location": "Portland, OR", "create_at": date("2009-05-05"), "description": "Follow this account for geo-targeted Other job tweets in Portland, OR from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 597, "friends_count": 536, "statues_count": 228 }, "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("2015-11-27T10:00:39.000Z"), "id": 670301222717419520, "text": "When are they gonna make a car that runs on water? My bank account was wondering.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 101618733, "name": "Haleyyy", "screen_name": "haleystarrett", "lang": "en", "location": "null", "create_at": date("2010-01-03"), "description": "ARKANSAS STATE CHI OMEGA❤️", "followers_count": 1060, "friends_count": 1030, "statues_count": 7116 }, "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("2015-11-27T10:00:39.000Z"), "id": 670301222734307329, "text": "#Cultura Solo 12% de Diputados del PRI presentaron iniciativas en septiembre. Video https://t.co/QkKIasx7gq RT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "Cultura" }}, "user": { "id": 3280268953, "name": "Parlamento y Debate", "screen_name": "ParlayDebate", "lang": "es", "location": "Mexico, ME", "create_at": date("2015-07-14"), "description": "Información Parlamentaria para masters, analistas, expertos y novice researcher. Difundimos lo que medios tradicionales no publican. #Follow", "followers_count": 2856, "friends_count": 3283, "statues_count": 2738 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mexico, PA", "id": "005ad92252b34585", "name": "Mexico", "place_type": "city", "bounding_box": rectangle("-77.367717,40.533752 -77.352142,40.55465") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42067, "countyName": "Juniata", "cityID": 4248904, "cityName": "Mexico" } }
+{ "create_at": datetime("2015-11-27T10:00:39.000Z"), "id": 670301222759301121, "text": "This article is not credible, it's a blog. People blog fake stuff all the time keep spinning your stories... https://t.co/nd3tCC4yqH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 126459718, "name": "Conservative Kevin", "screen_name": "commonsensekev", "lang": "en", "location": "Texas", "create_at": date("2010-03-25"), "description": "Husband, father, veteran, who loves the America our founding fathers built. Common sense and conservative values are parallel. #CruzCrew", "followers_count": 299, "friends_count": 335, "statues_count": 3358 }, "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-27T10:00:39.000Z"), "id": 670301223128535041, "text": "But goddAMN HER VOICE", "in_reply_to_status": 670300406518521856, "in_reply_to_user": 1496964042, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1496964042, "name": "Hales", "screen_name": "halliejanee", "lang": "en", "location": "Florida ☀", "create_at": date("2013-06-09"), "description": "Greater things are coming", "followers_count": 709, "friends_count": 269, "statues_count": 37139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Melbourne, FL", "id": "e49edeea1a3526b1", "name": "West Melbourne", "place_type": "city", "bounding_box": rectangle("-80.706103,28.03518 -80.637804,28.103875") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12009, "countyName": "Brevard", "cityID": 1276500, "cityName": "West Melbourne" } }
+{ "create_at": datetime("2015-11-27T10:00:39.000Z"), "id": 670301223182974976, "text": "YES ADELE! https://t.co/qR9cF0CGoD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 178394704, "name": "Lindsay Ireland", "screen_name": "iron369", "lang": "en", "location": "null", "create_at": date("2010-08-14"), "description": "• somewhere dreaming of owning a pet goat •", "followers_count": 375, "friends_count": 329, "statues_count": 8616 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfax, IA", "id": "005f681b2b60da2e", "name": "Fairfax", "place_type": "city", "bounding_box": rectangle("-91.793712,41.910765 -91.740476,41.942499") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19113, "countyName": "Linn", "cityID": 1926400, "cityName": "Fairfax" } }
+{ "create_at": datetime("2015-11-27T10:00:39.000Z"), "id": 670301223216545792, "text": "@mitskileaks @dlbogosian ¯\\_(ツ)_/¯", "in_reply_to_status": 670300542414004224, "in_reply_to_user": 2285540658, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2285540658, 22708618 }}, "user": { "id": 1644112537, "name": "♡", "screen_name": "xsharktooth", "lang": "en", "location": "null", "create_at": date("2013-08-03"), "description": "null", "followers_count": 173, "friends_count": 140, "statues_count": 13151 }, "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-27T10:00:39.000Z"), "id": 670301223426367488, "text": "https://t.co/pO23I13pPI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2614318089, "name": "Nqg", "screen_name": "nqg0214", "lang": "en", "location": "null", "create_at": date("2014-06-16"), "description": "♒ aquarius♒", "followers_count": 52, "friends_count": 116, "statues_count": 695 }, "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-27T10:00:39.000Z"), "id": 670301223753351168, "text": "Interested in a #Sales #job near #Bedford, NH? This could be a great fit: https://t.co/qnYOEiZNY9 #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.5159015,42.9464737"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "job", "Bedford", "Hiring" }}, "user": { "id": 218766987, "name": "ConcordNH Sales ", "screen_name": "tmj_NHC_sales", "lang": "en", "location": "Concord, NH", "create_at": date("2010-11-22"), "description": "Follow this account for geo-targeted Sales job tweets in Concord, NH from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 338, "friends_count": 338, "statues_count": 35 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bedford, NH", "id": "01db791b17fcd999", "name": "Bedford", "place_type": "city", "bounding_box": rectangle("-71.615121,42.896802 -71.453297,42.982829") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33011, "countyName": "Hillsborough" } }
+{ "create_at": datetime("2015-11-27T10:00:40.000Z"), "id": 670301224214794240, "text": "It's Friday ���� ready for some drinks ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1695471295, "name": "teeeeee♍️", "screen_name": "SmithLatecia", "lang": "en", "location": "null", "create_at": date("2013-08-23"), "description": "tee, young independent go getta, spoiled , chase is my world, living life", "followers_count": 1659, "friends_count": 2007, "statues_count": 34239 }, "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("2015-11-27T10:00:40.000Z"), "id": 670301224231690240, "text": "I can't believe I work all break��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 431293702, "name": "mia", "screen_name": "MiiLo823", "lang": "en", "location": "null", "create_at": date("2011-12-07"), "description": "Psychology Major✏️ CAL U '17 follow me on instagram @miiaaa_23", "followers_count": 1260, "friends_count": 748, "statues_count": 47165 }, "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("2015-11-27T10:00:40.000Z"), "id": 670301224261058560, "text": "I'm at White Rose Diner in Linden, NJ https://t.co/jhkvyIO0zo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.23934312,40.64313594"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 74828817, "name": "Michael W. ✈", "screen_name": "MichaelWTravels", "lang": "en", "location": "Brooklyn,NY", "create_at": date("2009-09-16"), "description": "I love all aspects of travel. From planning trips to finding great deals to exploring the world & taking photos along the way!", "followers_count": 1252, "friends_count": 432, "statues_count": 23073 }, "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("2015-11-27T10:00:40.000Z"), "id": 670301224348946433, "text": "@wintersnowolf just responded. The badge looks great, thanks ^_^", "in_reply_to_status": 670298035730128897, "in_reply_to_user": 24379132, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24379132 }}, "user": { "id": 2989734524, "name": "Moogie", "screen_name": "MoogieCougar", "lang": "en", "location": "Rancho Santa Fe, CA", "create_at": date("2015-01-19"), "description": "Furry | Musician | Photographer | Fun guy | Straight | Always SFW http://facebook.com/moogie.cougar", "followers_count": 79, "friends_count": 470, "statues_count": 205 }, "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-27T10:00:40.000Z"), "id": 670301224680480768, "text": "All I do when I come home is go bowling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3002666280, "name": "Jill", "screen_name": "julieschut", "lang": "en", "location": "Big Rapids, MI", "create_at": date("2015-01-29"), "description": "be the person your dog thinks you are", "followers_count": 101, "friends_count": 198, "statues_count": 523 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wyoming, MI", "id": "77b8f2664985738a", "name": "Wyoming", "place_type": "city", "bounding_box": rectangle("-85.782295,42.85341 -85.635295,42.942348") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26081, "countyName": "Kent", "cityID": 2688940, "cityName": "Wyoming" } }
+{ "create_at": datetime("2015-11-27T10:00:40.000Z"), "id": 670301224684556288, "text": "You can KMA lolololol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 502200665, "name": "k", "screen_name": "Kamryndcotton", "lang": "en", "location": "JolietWestJVCheer ", "create_at": date("2012-02-24"), "description": "all day momma #BLACKLIVESMATTER", "followers_count": 467, "friends_count": 647, "statues_count": 10767 }, "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("2015-11-27T10:00:40.000Z"), "id": 670301224709849090, "text": "Goin in on this Black Friday!! ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 713265463, "name": "Erick ©", "screen_name": "Al_Cano92", "lang": "en", "location": "Chicago,IL ", "create_at": date("2012-07-23"), "description": "Pay the cost to be the BOSS. Gemini. Living Young,Wild n Free. #KCCO", "followers_count": 138, "friends_count": 124, "statues_count": 3137 }, "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("2015-11-27T10:00:40.000Z"), "id": 670301224814698496, "text": "@ChulissaMonster #Leftovers #latino https://t.co/3d6sIJBes3", "in_reply_to_status": -1, "in_reply_to_user": 409464224, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Leftovers", "latino" }}, "user_mentions": {{ 409464224 }}, "user": { "id": 1076973967, "name": "Keith Dippold", "screen_name": "kdippol", "lang": "en", "location": "New Orleans, LA", "create_at": date("2013-01-10"), "description": "null", "followers_count": 66, "friends_count": 115, "statues_count": 456 }, "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("2015-11-27T10:00:40.000Z"), "id": 670301224885837824, "text": "Oml he got bring some fire to the table https://t.co/uVGcd9KvbG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3304268792, "name": "mula_boyひ", "screen_name": "zirslum", "lang": "en", "location": "Dickinson, TX.", "create_at": date("2015-08-02"), "description": "we all get two assets,24 hours in a day and the power to make choices(no excuse just produce) born to lose,BUILT TO WIN", "followers_count": 283, "friends_count": 203, "statues_count": 3093 }, "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("2015-11-27T10:00:40.000Z"), "id": 670301225162661889, "text": "dis side \ndis side \ndis side", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 411286724, "name": "11 | 28", "screen_name": "basedkamer", "lang": "en", "location": "null", "create_at": date("2011-11-12"), "description": "#hailsouth17 / your favorite young ambitious east african girl ✨", "followers_count": 1174, "friends_count": 876, "statues_count": 19756 }, "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("2015-11-27T10:00:40.000Z"), "id": 670301225326391296, "text": "I'm at Exit 25 Turnpike North in Miami, FL https://t.co/fat0cathI2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.38359148,25.75954022"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1038145110, "name": "robertkayakoparan", "screen_name": "CrmRobert", "lang": "tr", "location": "null", "create_at": date("2012-12-26"), "description": "null", "followers_count": 51, "friends_count": 78, "statues_count": 744 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University Park, FL", "id": "883ee3f98abb082b", "name": "University Park", "place_type": "city", "bounding_box": rectangle("-80.385477,25.732128 -80.351653,25.761752") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1273287, "cityName": "University Park" } }
+{ "create_at": datetime("2015-11-27T10:00:40.000Z"), "id": 670301225456308224, "text": "@creamedkearn oh god that unforgiving lighting lord have mercy on our souls... I seriously may be dying", "in_reply_to_status": 670300888255197184, "in_reply_to_user": 998750976, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 998750976 }}, "user": { "id": 1373517402, "name": "Brie", "screen_name": "RiddleBrieThis", "lang": "en", "location": "null", "create_at": date("2013-04-22"), "description": "null", "followers_count": 149, "friends_count": 248, "statues_count": 4514 }, "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("2015-11-27T10:00:40.000Z"), "id": 670301225489817600, "text": "@HollywoodRome lol he finesses, mad niggas try to", "in_reply_to_status": 670300604108021760, "in_reply_to_user": 29723500, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 29723500 }}, "user": { "id": 157412121, "name": "Random Dreadiii", "screen_name": "JefePioneer", "lang": "en", "location": "Start Out Local End Up Global ", "create_at": date("2010-06-19"), "description": "Tune Bandoiologist Discovering Greatness\n\n#Pioneer$ #MouthinAwF #HardWorkEnT #FullCreativeControl \n\nhttp://itunes.apple.com/us/album/rude-…", "followers_count": 818, "friends_count": 535, "statues_count": 31805 }, "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-27T10:00:40.000Z"), "id": 670301225598906369, "text": "last night basically, #happythanksgiving https://t.co/O86BKyIyCN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "happythanksgiving" }}, "user": { "id": 162790200, "name": "i dont even Cara ☀️✨", "screen_name": "carabear902", "lang": "en", "location": "Beckley; #BTown , WV", "create_at": date("2010-07-04"), "description": "'16, WWHS Varsity Softball. ⚾️ God is number ☝ matthew 5:38-42 ❤️ worlds friendliest bully.✊ Apparently no one gives a Cara . #UberFan #FaketasticFour", "followers_count": 1234, "friends_count": 496, "statues_count": 61976 }, "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("2015-11-27T10:00:40.000Z"), "id": 670301225838125056, "text": "\"They're not built like us!\" #spiderstrong #WarReady thank you again @CJemalHorton! https://t.co/j47ziJyfSg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "spiderstrong", "WarReady" }}, "user_mentions": {{ 44809474 }}, "user": { "id": 347530873, "name": "Carla B. Black", "screen_name": "Chewydst", "lang": "en", "location": "null", "create_at": date("2011-08-02"), "description": "C.O.G., BB & LAX Mom, Principal of the '12 US News & World Report top 10 HS in NC~ IB World School/ Pubic Safety Academy #WSSURAM #GOSPIDERS", "followers_count": 1615, "friends_count": 940, "statues_count": 19186 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Concord, NC", "id": "ec689e69912965d0", "name": "Concord", "place_type": "city", "bounding_box": rectangle("-80.766298,35.321152 -80.495965,35.482811") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37025, "countyName": "Cabarrus", "cityID": 3714100, "cityName": "Concord" } }
+{ "create_at": datetime("2015-11-27T10:00:40.000Z"), "id": 670301225959579648, "text": "If you're a #ProjectMgmt professional in #Philadelphia, PA, check out this #job: https://t.co/ErYp98E54u #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.1749671,39.9531865"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ProjectMgmt", "Philadelphia", "job", "Hiring" }}, "user": { "id": 22393668, "name": "TMJ-PHL IT PM Jobs", "screen_name": "tmj_phl_itpm", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2009-03-01"), "description": "Follow this account for geo-targeted IT-PM/BA job tweets in Philadelphia, PA from TweetMyJobs. Need help? Tweet us at @TweetMyJobs!", "followers_count": 246, "friends_count": 201, "statues_count": 46 }, "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-27T10:00:40.000Z"), "id": 670301226190278656, "text": "I just want 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": 2281492082, "name": "Jo$h Martinez ️➰", "screen_name": "Juan_josh20", "lang": "en", "location": "Stone Oak, San Antonio", "create_at": date("2014-01-07"), "description": "taken", "followers_count": 1310, "friends_count": 1012, "statues_count": 14931 }, "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-27T10:00:40.000Z"), "id": 670301226337202180, "text": "#bliss #blackfriday #solostatus @ Vero Beach, Florida https://t.co/yjKg1K5l8Y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.3911,27.6418"), "retweet_count": 0, "lang": "pt", "is_retweet": false, "hashtags": {{ "bliss", "blackfriday", "solostatus" }}, "user": { "id": 35214983, "name": "Amber M. Borgen", "screen_name": "awallabee23", "lang": "en", "location": "Gaithersburg, Maryland", "create_at": date("2009-04-25"), "description": "HTTR!! IG and Snapchat - awallabee23", "followers_count": 219, "friends_count": 317, "statues_count": 1322 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vero Beach, FL", "id": "3cf05613c1dd6dbd", "name": "Vero Beach", "place_type": "city", "bounding_box": rectangle("-80.463302,27.587008 -80.346166,27.674983") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12061, "countyName": "Indian River", "cityID": 1274150, "cityName": "Vero Beach" } }
+{ "create_at": datetime("2015-11-27T10:00:40.000Z"), "id": 670301226513391616, "text": "@AyooJariah Happy Birthday ��", "in_reply_to_status": -1, "in_reply_to_user": 2388927708, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2388927708 }}, "user": { "id": 164884613, "name": "Brappy", "screen_name": "Wassgud_B", "lang": "en", "location": "null", "create_at": date("2010-07-09"), "description": "#RIPGrandad #CentralStateUniversity", "followers_count": 2445, "friends_count": 1934, "statues_count": 42460 }, "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("2015-11-27T10:00:40.000Z"), "id": 670301226987159552, "text": "#Creed @ Showcase Cinemas Warwick https://t.co/r4R4SKdPSW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.49596034,41.66365955"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Creed" }}, "user": { "id": 34811996, "name": "Nicholas Delmenico", "screen_name": "NickDelmenico", "lang": "en", "location": "Rhode Island", "create_at": date("2009-04-23"), "description": "producer & line producer @RIFilmGuys / Delta Chi / Swiss American / LRI '14 / all star foodie / rookie runner", "followers_count": 542, "friends_count": 793, "statues_count": 586 }, "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("2015-11-27T10:00:40.000Z"), "id": 670301227037519876, "text": "qué admitan a sus errores y lloren", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2387257286, "name": "mamiii", "screen_name": "suggaarrr", "lang": "en", "location": "null", "create_at": date("2014-03-13"), "description": "soy de Durango, el estado más chingon ⠀⠀⠀⠀⠀⠀ ⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀", "followers_count": 1987, "friends_count": 1535, "statues_count": 8039 }, "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("2015-11-27T10:00:40.000Z"), "id": 670301227041857536, "text": "@BartoloFargo @SchlossmanGF You can make it regional after seeding 16, seeding 16 meant to balance the brackets.", "in_reply_to_status": 670294466075332608, "in_reply_to_user": 24933126, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24933126, 30335064 }}, "user": { "id": 25472083, "name": "Jeff Kolpack", "screen_name": "FGOSPORTSWRITER", "lang": "en", "location": "Fargo, ND", "create_at": date("2009-03-19"), "description": "Forum reporter; co-host Kolpack & Izzo WDAY-AM Saturday morning radio show; host, WDAY Golf Show.", "followers_count": 4415, "friends_count": 264, "statues_count": 8321 }, "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("2015-11-27T10:00:40.000Z"), "id": 670301227553566720, "text": "Officially done using @Outlook as my main email account. I'm convinced they have someone deleting all of my important emails. Hello @gmail.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 605805760, 38679388 }}, "user": { "id": 379846624, "name": "London Williams", "screen_name": "lalalundun", "lang": "en", "location": "Hampton University | PR Major ", "create_at": date("2011-09-25"), "description": "Aspiring. Spontaneous. Confident. Passionate. Determined. Ambitious. PR | Pop Culture | Fashion | Social media for Brand id | SP", "followers_count": 964, "friends_count": 1020, "statues_count": 14476 }, "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("2015-11-27T10:00:40.000Z"), "id": 670301227662491648, "text": "i love wearing brown", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3368850717, "name": "liynaa", "screen_name": "lenniewhy", "lang": "en", "location": "null", "create_at": date("2015-07-10"), "description": "made out of love", "followers_count": 399, "friends_count": 456, "statues_count": 6048 }, "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" } }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/btree-btree-search-wo-query-filter.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/btree-btree-search-wo-query-filter.aql
new file mode 100644
index 0000000..520f59f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/btree-btree-search-wo-query-filter.aql
@@ -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 dataverse test;
+
+create type tTweet as closed {
+  id: int32,
+  location: point,
+  message: string,
+  create_at: datetime,
+  misc: string
+}
+
+create dataset dsTweet(tTweet) primary key id with filter on create_at;
+
+create index misc_index on dsTweet(misc) type btree;
+
+for $t in dataset dsTweet
+where $t.misc = "xyz"
+return $t
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/btree-btree-search.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/btree-btree-search.aql
new file mode 100644
index 0000000..b37b1ad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/btree-btree-search.aql
@@ -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 test if exists;
+create dataverse test;
+use dataverse test;
+
+create type tTweet as closed {
+  id: int32,
+  location: point,
+  message: string,
+  create_at: datetime,
+  misc: string
+}
+
+create dataset dsTweet(tTweet) primary key id with filter on create_at;
+
+create index misc_index on dsTweet(misc) type btree;
+
+let $ts_start := datetime("2015-11-11T00:00:00Z")
+let $ts_end := datetime("2015-12-18T23:59:59Z")
+
+for $t in dataset dsTweet
+where $t.create_at >= $ts_start and $t.create_at < $ts_end
+  and $t.misc = "xyz"
+return $t
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/inverted-btree-search-wo-query-filter.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/inverted-btree-search-wo-query-filter.aql
new file mode 100644
index 0000000..92226ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/inverted-btree-search-wo-query-filter.aql
@@ -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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type tTweet as closed {
+  id: int32,
+  location: point,
+  message: string,
+  create_at: datetime,
+  misc: string
+}
+
+create dataset dsTweet(tTweet) primary key id with filter on create_at;
+
+create index ft_index on dsTweet(message) type fulltext;
+
+let $keyword := "hello"
+
+for $t in dataset dsTweet
+where ftcontains($t.message, $keyword)
+return $t
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/inverted-btree-search.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/inverted-btree-search.aql
new file mode 100644
index 0000000..61f7e08
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/inverted-btree-search.aql
@@ -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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type tTweet as closed {
+  id: int32,
+  location: point,
+  message: string,
+  create_at: datetime,
+  misc: string
+}
+
+create dataset dsTweet(tTweet) primary key id with filter on create_at;
+
+create index ft_index on dsTweet(message) type fulltext;
+
+let $ts_start := datetime("2015-11-11T00:00:00Z")
+let $ts_end := datetime("2015-12-18T23:59:59Z")
+let $keyword := "hello"
+
+for $t in dataset dsTweet
+where $t.create_at >= $ts_start and $t.create_at < $ts_end
+  and ftcontains($t.message, $keyword)
+return $t
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/multi-index-btree-search-wo-query-filter.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/multi-index-btree-search-wo-query-filter.aql
new file mode 100644
index 0000000..cd8639e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/multi-index-btree-search-wo-query-filter.aql
@@ -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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type tTweet as closed {
+  id: int32,
+  location: point,
+  message: string,
+  create_at: datetime,
+  misc: string
+}
+
+create dataset dsTweet(tTweet) primary key id with filter on create_at;
+
+create index ft_index on dsTweet(message) type fulltext;
+create index misc_index on dsTweet(misc) type btree;
+create index location_index on dsTweet(location) type rtree;
+
+let $region := create-rectangle(create-point(-128.43,20.29), create-point(-64.26,54.56))
+let $keyword := "hello"
+
+for $t in dataset dsTweet
+where  $t.misc < "xzy"
+  and spatial-intersect($t.location, $region)
+  and ftcontains($t.message, $keyword)
+return $t
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/multi-index-btree-search.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/multi-index-btree-search.aql
new file mode 100644
index 0000000..c50fa1f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/multi-index-btree-search.aql
@@ -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 test if exists;
+create dataverse test;
+use dataverse test;
+
+create type tTweet as closed {
+  id: int32,
+  location: point,
+  message: string,
+  create_at: datetime,
+  misc: string
+}
+
+create dataset dsTweet(tTweet) primary key id with filter on create_at;
+
+create index ft_index on dsTweet(message) type fulltext;
+create index misc_index on dsTweet(misc) type btree;
+create index location_index on dsTweet(location) type rtree;
+
+let $region := create-rectangle(create-point(-128.43,20.29), create-point(-64.26,54.56))
+let $ts_start := datetime("2015-11-11T00:00:00Z")
+let $ts_end := datetime("2015-12-18T23:59:59Z")
+let $keyword := "hello"
+
+for $t in dataset dsTweet
+where $t.create_at >= $ts_start and $t.create_at < $ts_end
+  and $t.misc < "xzy"
+  and spatial-intersect($t.location, $region)
+  and ftcontains($t.message, $keyword)
+return $t
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/rtree-btree-search-wo-query-filter.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/rtree-btree-search-wo-query-filter.aql
new file mode 100644
index 0000000..556fddd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/rtree-btree-search-wo-query-filter.aql
@@ -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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type tTweet as closed {
+  id: int32,
+  location: point,
+  message: string,
+  create_at: datetime,
+  misc: string
+}
+
+create dataset dsTweet(tTweet) primary key id with filter on create_at;
+
+create index location_index on dsTweet(location) type rtree;
+
+let $region := create-rectangle(create-point(-128.43,20.29), create-point(-64.26,54.56))
+
+for $t in dataset dsTweet
+where spatial-intersect($t.location, $region)
+return $t
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/rtree-btree-search.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/rtree-btree-search.aql
new file mode 100644
index 0000000..5b4bcb2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/filter/rtree-btree-search.aql
@@ -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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type tTweet as closed {
+  id: int32,
+  location: point,
+  message: string,
+  create_at: datetime,
+  misc: string
+}
+
+create dataset dsTweet(tTweet) primary key id with filter on create_at;
+
+create index location_index on dsTweet(location) type rtree;
+
+let $region := create-rectangle(create-point(-128.43,20.29), create-point(-64.26,54.56))
+let $ts_start := datetime("2015-11-11T00:00:00Z")
+let $ts_end := datetime("2015-12-18T23:59:59Z")
+
+for $t in dataset dsTweet
+where $t.create_at >= $ts_start and $t.create_at < $ts_end
+  and spatial-intersect($t.location, $region)
+return $t
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/btree-btree-search-wo-query-filter.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/btree-btree-search-wo-query-filter.plan
new file mode 100644
index 0000000..d7667d6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/btree-btree-search-wo-query-filter.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 [$$9(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/filter/btree-btree-search.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/btree-btree-search.plan
new file mode 100644
index 0000000..0d9a257
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/btree-btree-search.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|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-wo-query-filter.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-wo-query-filter.plan
new file mode 100644
index 0000000..1b99cd6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-wo-query-filter.plan
@@ -0,0 +1,13 @@
+-- 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 [$$8(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- SINGLE_PARTITION_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search.plan
new file mode 100644
index 0000000..7b03ff5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search.plan
@@ -0,0 +1,15 @@
+-- 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 [$$18(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SINGLE_PARTITION_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/multi-index-btree-search-wo-query-filter.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/multi-index-btree-search-wo-query-filter.plan
new file mode 100644
index 0000000..8c799fa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/multi-index-btree-search-wo-query-filter.plan
@@ -0,0 +1,33 @@
+-- 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|
+              -- INTERSECT  |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|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SINGLE_PARTITION_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$33(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|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/multi-index-btree-search.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/multi-index-btree-search.plan
new file mode 100644
index 0000000..fa69d3b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/multi-index-btree-search.plan
@@ -0,0 +1,35 @@
+-- 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|
+                  -- INTERSECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$31(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 [$$33(ASC)]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- SINGLE_PARTITION_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$42(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|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/rtree-btree-search-wo-query-filter.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/rtree-btree-search-wo-query-filter.plan
new file mode 100644
index 0000000..59443b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/rtree-btree-search-wo-query-filter.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 [$$20(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|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/rtree-btree-search.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/rtree-btree-search.plan
new file mode 100644
index 0000000..8f346d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/rtree-btree-search.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 [$$30(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|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/multi-indexes/btree-rtree-ngram-intersect-with-filter.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/multi-indexes/btree-rtree-ngram-intersect-with-filter.plan
index 87a6f42..ecb9eff 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/multi-indexes/btree-rtree-ngram-intersect-with-filter.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/multi-indexes/btree-rtree-ngram-intersect-with-filter.plan
@@ -7,31 +7,29 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+                  -- INTERSECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INTERSECT  |PARTITIONED|
+                      -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |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 [$$31(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$40(ASC)]  |PARTITIONED|
+                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$40(ASC)]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- RTREE_SEARCH  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- RTREE_SEARCH  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.1.ddl.aql
new file mode 100644
index 0000000..70af710
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.1.ddl.aql
@@ -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.
+ */
+/*
+ * This test case verify if the filter optimization rule is still correct when there
+ * is one deletion on some of the component. The deleted value shouldn't be returned.
+ *
+ * 1. create the dataset Tweet that ingested by a feed.
+ * 2. delete one tweet
+ * 3. start the feed again to make the previous component flush to disk
+ * 4. send the query to check deleted record.
+ */
+
+drop dataverse test if exists;
+create dataverse test if not exists;
+use dataverse test
+
+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 Tweet(typeTweet) primary key id
+using compaction policy prefix (("max-mergable-component-size"="32768"),("max-tolerance-component-count"="32"))
+with filter on create_at;
+
+create index text_idx if not exists on Tweet("text") type fulltext;
+create index state_idx if not exists on Tweet(geo_tag.stateID) type btree;
+
+create feed TweetFeed using socket_adapter
+(
+    ("sockets"="127.0.0.1:10001"),
+    ("address-type"="IP"),
+    ("type-name"="typeTweet"),
+    ("format"="adm")
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.10.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.10.query.aql
new file mode 100644
index 0000000..077ef70
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.10.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.
+ */
+/*
+ * Search the deleted record
+ */
+use dataverse test;
+
+for $m in dataset('Tweet')
+where ftcontains($m.'text', ["Campus", "Martius", "Park"])
+return $m.id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.2.update.aql
new file mode 100644
index 0000000..148c5c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.2.update.aql
@@ -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 dataverse test;
+set wait-for-completion-feed "false";
+
+connect feed TweetFeed to dataset Tweet;
+
+start feed TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.3.server.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.3.server.aql
new file mode 100644
index 0000000..22fbc4a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.3.server.aql
@@ -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  : Create a socket feed with a client that pushes
+ * 1000 records. The feed is connected to a dataset that is then
+ * queried for the data.
+ * Expected Res : Success
+ */
+
+start client 10001 file-client 127.0.0.1 ../asterix-app/data/twitter/real.adm 1000 100 900
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.4.sleep.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.4.sleep.aql
new file mode 100644
index 0000000..17e7e67
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.4.sleep.aql
@@ -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.
+ */
+/*
+ * sleep 5 seconds
+ */
+5000
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.5.server.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.5.server.aql
new file mode 100644
index 0000000..39f1216
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.5.server.aql
@@ -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.
+ */
+/*
+ * stop the feed client
+ */
+ stop 10001
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.6.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.6.update.aql
new file mode 100644
index 0000000..6989de1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.6.update.aql
@@ -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 dataverse test;
+delete $d from dataset Tweet where $d.id = 668945640186101761;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.7.server.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.7.server.aql
new file mode 100644
index 0000000..4c139d8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.7.server.aql
@@ -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.
+ */
+/*
+ * Description  : Continue ingest 500 records
+ */
+
+start client 10001 file-client 127.0.0.1 ../asterix-app/data/twitter/real.2.adm 500 50 1000
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.8.sleep.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.8.sleep.aql
new file mode 100644
index 0000000..c0e90c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.8.sleep.aql
@@ -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.
+ */
+
+5000
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.9.server.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.9.server.aql
new file mode 100644
index 0000000..aacaeaf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/delete/delete.9.server.aql
@@ -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.
+ */
+
+ stop 10001
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.1.ddl.aql
new file mode 100644
index 0000000..4adc491
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.1.ddl.aql
@@ -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.
+ */
+/*
+ * This test case verify if the filter optimization rule is still correct when there
+ * is one upsert on some of the component. The new value should be returned.
+ *
+ * 1. create the dataset Tweet that ingested by a feed.
+ * 2. update one tweet
+ * 3. start the feed again to make the previous component flush to disk
+ * 4. send the query by the old value to see if any record returns
+ */
+
+drop dataverse test if exists;
+create dataverse test if not exists;
+use dataverse test
+
+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 Tweet(typeTweet) primary key id
+using compaction policy prefix (("max-mergable-component-size"="32768"),("max-tolerance-component-count"="32"))
+with filter on create_at;
+
+create index text_idx if not exists on Tweet("text") type btree;
+create index state_idx if not exists on Tweet(geo_tag.stateID) type btree;
+
+create feed TweetFeed using socket_adapter
+(
+    ("sockets"="127.0.0.1:10001"),
+    ("address-type"="IP"),
+    ("type-name"="typeTweet"),
+    ("format"="adm")
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.10.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.10.query.aql
new file mode 100644
index 0000000..7ec75ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.10.query.aql
@@ -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.
+ */
+/*
+ * Select the updated record through the btree access
+ */
+use dataverse test;
+
+for $m in dataset('Tweet')
+where $m.'text'= "Just posted a photo @ Campus Martius Park https://t.co/5Ax4E2CdWZ"
+return $m.geo_tag.stateID
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.11.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.11.ddl.aql
new file mode 100644
index 0000000..f12a2b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.11.ddl.aql
@@ -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/filters/upsert/upsert.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.2.update.aql
new file mode 100644
index 0000000..21b9b03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.2.update.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.
+ */
+/*
+ * start the feed
+ */
+use dataverse test;
+set wait-for-completion-feed "false";
+
+connect feed TweetFeed to dataset Tweet;
+
+start feed TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.3.server.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.3.server.aql
new file mode 100644
index 0000000..22fbc4a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.3.server.aql
@@ -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  : Create a socket feed with a client that pushes
+ * 1000 records. The feed is connected to a dataset that is then
+ * queried for the data.
+ * Expected Res : Success
+ */
+
+start client 10001 file-client 127.0.0.1 ../asterix-app/data/twitter/real.adm 1000 100 900
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.4.sleep.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.4.sleep.aql
new file mode 100644
index 0000000..73f6185
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.4.sleep.aql
@@ -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.
+ */
+/*
+ * sleep 5s
+ */
+5000
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.5.server.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.5.server.aql
new file mode 100644
index 0000000..9c9197b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.5.server.aql
@@ -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.
+ */
+/*
+ * stop feed client
+ */
+ stop 10001
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.6.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.6.update.aql
new file mode 100644
index 0000000..08e524f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.6.update.aql
@@ -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.
+ */
+/*
+ * update the record to change it's stateID
+ */
+use dataverse test;
+
+upsert into dataset Tweet (
+{ "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", "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": 0, "stateName": "Michigan",
+    "countyID": 26163, "countyName": "Wayne",
+    "cityID": 2622000, "cityName": "Detroit"
+  }
+}
+)
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.7.server.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.7.server.aql
new file mode 100644
index 0000000..9405846
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.7.server.aql
@@ -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.
+ */
+/*
+ * Description  : Continue ingest 10,000 records
+ */
+
+start client 10001 file-client 127.0.0.1 ../asterix-app/data/twitter/real.2.adm 500 50 1000
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.8.sleep.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.8.sleep.aql
new file mode 100644
index 0000000..c0e90c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.8.sleep.aql
@@ -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.
+ */
+
+5000
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.9.server.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.9.server.aql
new file mode 100644
index 0000000..aacaeaf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/filters/upsert/upsert.9.server.aql
@@ -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.
+ */
+
+ stop 10001
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/intersection-with-filter/tinysocial-intersect.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/intersection-with-filter/tinysocial-intersect.1.ddl.aql
new file mode 100644
index 0000000..a1f531d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/intersection-with-filter/tinysocial-intersect.1.ddl.aql
@@ -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 dataverse TinySocial;
+
+create type TwitterUserType as open {
+        screen-name: string,
+        lang: string,
+        friends_count: int64,
+        statuses_count: int64,
+        name: string,
+        followers_count: int64,
+        sender-location: point
+}
+
+create type TweetMessageType as closed {
+        tweetid: int64,
+        user: TwitterUserType,
+        send-time: datetime,
+        referred-topics: {{ string }},
+        message-text: string
+}
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid with filter on send-time;
+
+create index twTimeIdx on TweetMessages(send-time) type btree;
+create index twLocationIdx on TweetMessages(user.sender-location) type rtree;
+create index twMessage on TweetMessages(message-text) type ngram(2);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/intersection-with-filter/tinysocial-intersect.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/intersection-with-filter/tinysocial-intersect.2.update.aql
new file mode 100644
index 0000000..9923ff4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/intersection-with-filter/tinysocial-intersect.2.update.aql
@@ -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 dataverse TinySocial;
+
+load dataset TweetMessages using localfs
+(("path"="asterix_nc1://data/tinysocial/twm-nested.adm"),("format"="adm"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/intersection-with-filter/tinysocial-intersect.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/intersection-with-filter/tinysocial-intersect.3.query.aql
new file mode 100644
index 0000000..392008c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/index-selection/intersection-with-filter/tinysocial-intersect.3.query.aql
@@ -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 dataverse TinySocial;
+
+let $ts := datetime("2010-12-12T00:00:00Z")
+let $region := create-rectangle(create-point(0.0,0.0),create-point(100.0,100.0))
+let $keyword := "verizon"
+for $t in dataset TweetMessages
+where $t.send-time > $ts
+    and spatial-intersect($t.user.sender-location, $region)
+    and contains($t.message-text, $keyword)
+return $t
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.1.ddl.sqlpp
new file mode 100644
index 0000000..fcd4f57
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.1.ddl.sqlpp
@@ -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.
+ */
+/*
+ * This test case verify if the filter optimization rule is still correct when there
+ * is one deletion on some of the component. The deleted value shouldn't be returned.
+ *
+ * 1. create the dataset Tweet that ingested by a feed.
+ * 2. delete one tweet
+ * 3. start the feed again to make the previous component flush to disk
+ * 4. send the query to check deleted record.
+ */
+
+drop dataverse test if exists;
+create dataverse test if not exists;
+use test;
+
+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 Tweet(typeTweet) primary key id
+using compaction policy prefix (("max-mergable-component-size"="32768"),("max-tolerance-component-count"="32"))
+with filter on create_at;
+
+create index text_idx if not exists on Tweet(`text`) type fulltext;
+create index state_idx if not exists on Tweet(geo_tag.stateID) type btree;
+
+create feed TweetFeed using socket_adapter
+(
+    ("sockets"="127.0.0.1:10001"),
+    ("address-type"="IP"),
+    ("type-name"="typeTweet"),
+    ("format"="adm")
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.10.query.sqlpp
new file mode 100644
index 0000000..dd08fd5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.10.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 value m.id from Tweet m
+WHERE ftcontains(m.`text`, ["Campus", "Martius", "Park"]);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.11.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.11.ddl.sqlpp
new file mode 100644
index 0000000..da9bd3a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.11.ddl.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  : Create a socket feed with a client that pushes
+ * 10 records. The feed is connected to a dataset that is then
+ * queried for the data.
+ * Expected Res : Success
+ * Date         : 24th Feb 2016
+ */
+
+drop dataverse test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.2.update.sqlpp
new file mode 100644
index 0000000..d65d404
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.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;
+set `wait-for-completion-feed` "false";
+connect feed TweetFeed to dataset Tweet;
+
+start feed TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.3.server.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.3.server.sqlpp
new file mode 100644
index 0000000..22fbc4a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.3.server.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  : Create a socket feed with a client that pushes
+ * 1000 records. The feed is connected to a dataset that is then
+ * queried for the data.
+ * Expected Res : Success
+ */
+
+start client 10001 file-client 127.0.0.1 ../asterix-app/data/twitter/real.adm 1000 100 900
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.4.sleep.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.4.sleep.sqlpp
new file mode 100644
index 0000000..c0e90c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.4.sleep.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.
+ */
+
+5000
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.5.server.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.5.server.sqlpp
new file mode 100644
index 0000000..aacaeaf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.5.server.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.
+ */
+
+ stop 10001
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.6.update.sqlpp
new file mode 100644
index 0000000..cc12612
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.6.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;
+DELETE FROM Tweet t WHERE t.id = 668945640186101761;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.7.server.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.7.server.sqlpp
new file mode 100644
index 0000000..9405846
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.7.server.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.
+ */
+/*
+ * Description  : Continue ingest 10,000 records
+ */
+
+start client 10001 file-client 127.0.0.1 ../asterix-app/data/twitter/real.2.adm 500 50 1000
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.8.sleep.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.8.sleep.sqlpp
new file mode 100644
index 0000000..c0e90c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.8.sleep.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.
+ */
+
+5000
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.9.server.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.9.server.sqlpp
new file mode 100644
index 0000000..aacaeaf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/delete/delete.9.server.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.
+ */
+
+ stop 10001
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.1.ddl.sqlpp
new file mode 100644
index 0000000..417f841
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.1.ddl.sqlpp
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * This test case verify if the filter optimization rule is still correct when there
+ * is one upsert on some of the component. The new value should be returned.
+ *
+ * 1. create the dataset Tweet that ingested by a feed.
+ * 2. update one tweet
+ * 3. start the feed again to make the previous component flush to disk
+ * 4. send the query by the old value to see if any record returns
+ */
+
+
+drop dataverse test if exists;
+create dataverse test if not exists;
+use test;
+
+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 Tweet(typeTweet) primary key id
+using compaction policy prefix (("max-mergable-component-size"="32768"),("max-tolerance-component-count"="32"))
+with filter on create_at;
+
+create index text_idx if not exists on Tweet(`text`) type btree;
+create index state_idx if not exists on Tweet(geo_tag.stateID) type btree;
+
+create feed TweetFeed using socket_adapter
+(
+    ("sockets"="127.0.0.1:10001"),
+    ("address-type"="IP"),
+    ("type-name"="typeTweet"),
+    ("format"="adm")
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.10.query.sqlpp
new file mode 100644
index 0000000..4846e8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.10.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 value m.geo_tag.stateID from Tweet m
+WHERE m.`text` = "Just posted a photo @ Campus Martius Park https://t.co/5Ax4E2CdWZ"
+;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.11.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.11.ddl.sqlpp
new file mode 100644
index 0000000..da9bd3a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.11.ddl.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  : Create a socket feed with a client that pushes
+ * 10 records. The feed is connected to a dataset that is then
+ * queried for the data.
+ * Expected Res : Success
+ * Date         : 24th Feb 2016
+ */
+
+drop dataverse test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.2.update.sqlpp
new file mode 100644
index 0000000..54a8a2c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.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;
+
+set `wait-for-completion-feed` "false";
+
+connect feed TweetFeed to dataset Tweet;
+
+start feed TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.3.server.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.3.server.sqlpp
new file mode 100644
index 0000000..22fbc4a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.3.server.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  : Create a socket feed with a client that pushes
+ * 1000 records. The feed is connected to a dataset that is then
+ * queried for the data.
+ * Expected Res : Success
+ */
+
+start client 10001 file-client 127.0.0.1 ../asterix-app/data/twitter/real.adm 1000 100 900
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.4.sleep.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.4.sleep.sqlpp
new file mode 100644
index 0000000..c0e90c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.4.sleep.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.
+ */
+
+5000
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.5.server.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.5.server.sqlpp
new file mode 100644
index 0000000..aacaeaf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.5.server.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.
+ */
+
+ stop 10001
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.6.update.sqlpp
new file mode 100644
index 0000000..2204437
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.6.update.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.
+ */
+
+use test;
+
+upsert into Tweet (
+{ "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", "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": 0, "stateName": "Michigan",
+    "countyID": 26163, "countyName": "Wayne",
+    "cityID": 2622000, "cityName": "Detroit"
+  }
+}
+)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.7.server.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.7.server.sqlpp
new file mode 100644
index 0000000..4c139d8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.7.server.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.
+ */
+/*
+ * Description  : Continue ingest 500 records
+ */
+
+start client 10001 file-client 127.0.0.1 ../asterix-app/data/twitter/real.2.adm 500 50 1000
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.8.sleep.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.8.sleep.sqlpp
new file mode 100644
index 0000000..c0e90c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.8.sleep.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.
+ */
+
+5000
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.9.server.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.9.server.sqlpp
new file mode 100644
index 0000000..aacaeaf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/upsert/upsert.9.server.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.
+ */
+
+ stop 10001
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/delete/delete.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/delete/delete.1.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/delete/delete.1.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/upsert/upsert.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/upsert/upsert.1.adm
new file mode 100644
index 0000000..c227083
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/upsert/upsert.1.adm
@@ -0,0 +1 @@
+0
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-filter/intersection.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-filter/intersection.1.adm
new file mode 100644
index 0000000..44cc08a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-filter/intersection.1.adm
@@ -0,0 +1 @@
+{ "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.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index be6827f..e986046 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -3051,6 +3051,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="index-selection">
+      <compilation-unit name="intersection-with-filter">
+        <output-dir compare="Text">intersection-with-filter</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
       <compilation-unit name="intersection_with_nodegroup">
         <output-dir compare="Text">intersection</output-dir>
       </compilation-unit>
@@ -7229,6 +7234,16 @@
         <output-dir compare="Text">nested-filter-equality-predicate</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="upsert">
+        <output-dir compare="Text">upsert</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="delete">
+        <output-dir compare="Text">delete</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="json">
     <test-case FilePath="json">
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 4574c66..6a6ea4b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -8511,6 +8511,16 @@
         <output-dir compare="Text">nested-filter-equality-predicate</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="upsert">
+        <output-dir compare="Text">upsert</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="delete">
+        <output-dir compare="Text">delete</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="json">
     <test-case FilePath="json">
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 68c7e22..e2b1761 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
@@ -120,7 +120,7 @@
                 int[] maxFilterFieldIndexes = createFilterIndexes(maxFilterVars, opSchema);
                 return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true,
                         false, ((DatasetDataSource) dataSource).getDataset(), primaryIndex.getIndexName(), null, null,
-                        true, true, minFilterFieldIndexes, maxFilterFieldIndexes);
+                        true, true, false, minFilterFieldIndexes, maxFilterFieldIndexes);
             default:
                 throw new AlgebricksException("Unknown datasource type");
         }
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 e0cfc28..3b70ea9 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
@@ -427,8 +427,8 @@
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildBtreeRuntime(JobSpecification jobSpec,
             IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, JobGenContext context, boolean retainInput,
             boolean retainMissing, Dataset dataset, String indexName, int[] lowKeyFields, int[] highKeyFields,
-            boolean lowKeyInclusive, boolean highKeyInclusive, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes)
-            throws AlgebricksException {
+            boolean lowKeyInclusive, boolean highKeyInclusive, boolean propagateFilter, int[] minFilterFieldIndexes,
+            int[] maxFilterFieldIndexes) throws AlgebricksException {
         boolean isSecondary = true;
         try {
             Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
@@ -456,7 +456,7 @@
                 btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields, highKeyFields,
                         lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
                         context.getMissingWriterFactory(), searchCallbackFactory, minFilterFieldIndexes,
-                        maxFilterFieldIndexes, false);
+                        maxFilterFieldIndexes, propagateFilter);
             } else {
                 btreeSearchOp = new ExternalBTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
                         highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput,
@@ -473,7 +473,8 @@
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildRtreeRuntime(JobSpecification jobSpec,
             List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
             JobGenContext context, boolean retainInput, boolean retainMissing, Dataset dataset, String indexName,
-            int[] keyFields, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes) throws AlgebricksException {
+            int[] keyFields, boolean propagateFilter, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes)
+            throws AlgebricksException {
         try {
             int numPrimaryKeys = dataset.getPrimaryKeys().size();
             Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
@@ -498,7 +499,7 @@
             if (dataset.getDatasetType() == DatasetType.INTERNAL) {
                 rtreeSearchOp = new RTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, keyFields, true, true,
                         indexDataflowHelperFactory, retainInput, retainMissing, context.getMissingWriterFactory(),
-                        searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes, false);
+                        searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes, propagateFilter);
             } else {
                 // Create the operator
                 rtreeSearchOp = new ExternalRTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, keyFields, true, true,
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractScanOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractScanOperator.java
index 9cd7138..5d6f40c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractScanOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractScanOperator.java
@@ -36,6 +36,11 @@
         return variables;
     }
 
+    public List<LogicalVariable> getScanVariables() {
+        return variables;
+    }
+
+
     public void setVariables(List<LogicalVariable> variables) {
         this.variables = variables;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
index 8a2981d..a8e6b44 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
@@ -35,12 +35,24 @@
     protected List<LogicalVariable> minFilterVars;
     protected List<LogicalVariable> maxFilterVars;
 
+    protected boolean propagateIndexFilter;
+
     public AbstractUnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
             List<Object> variableTypes, boolean propagateInput) {
         super(variables, expression);
         this.expression = expression;
         this.variableTypes = variableTypes;
         this.propagateInput = propagateInput;
+        this.propagateIndexFilter = false;
+    }
+
+    @Override
+    public List<LogicalVariable> getScanVariables() {
+        if (propagateIndexFilter) {
+            return variables.subList(0, variables.size() - 2);
+        } else {
+            return variables;
+        }
     }
 
     public List<Object> getVariableTypes() {
@@ -98,4 +110,27 @@
         return additionalFilteringExpressions;
     }
 
+    public void markPropagageIndexFilter() {
+        this.propagateIndexFilter = true;
+    }
+
+    public boolean propagateIndexFilter() {
+        return this.propagateIndexFilter;
+    }
+
+    public LogicalVariable getPropagateIndexMinFilterVar() {
+        if (propagateIndexFilter) {
+            return variables.get(variables.size() - 2);
+        } else {
+            return null;
+        }
+    }
+
+    public LogicalVariable getPropagateIndexMaxFilterVar() {
+        if (propagateIndexFilter) {
+            return variables.get(variables.size() - 1);
+        } else {
+            return null;
+        }
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IntersectOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IntersectOperator.java
index e64be2b..114fde0 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IntersectOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IntersectOperator.java
@@ -19,7 +19,9 @@
 
 package org.apache.hyracks.algebricks.core.algebra.operators.logical;
 
+import java.util.ArrayList;
 import java.util.List;
+import java.util.stream.Collectors;
 
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
@@ -32,22 +34,45 @@
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 import org.apache.hyracks.algebricks.core.config.AlgebricksConfig;
+import org.apache.hyracks.api.exceptions.ErrorCode;
 
 public class IntersectOperator extends AbstractLogicalOperator {
 
     private final List<List<LogicalVariable>> inputVars;
+    private final List<List<LogicalVariable>> compareVars;
     private final List<LogicalVariable> outputVars;
+    private List<List<LogicalVariable>> extraVars;
 
-    public IntersectOperator(List<LogicalVariable> outputVars, List<List<LogicalVariable>> inputVars)
+    public IntersectOperator(List<LogicalVariable> outputVars, List<List<LogicalVariable>> compareVars)
             throws AlgebricksException {
-        if (outputVars.size() != inputVars.get(0).size()) {
-            throw new AlgebricksException("The number of output variables is different with the input variable number");
+        this(outputVars, compareVars,
+                compareVars.stream().map(vars -> new ArrayList<LogicalVariable>()).collect(Collectors.toList()));
+    }
+
+    public IntersectOperator(List<LogicalVariable> outputVars, List<List<LogicalVariable>> compareVars,
+            List<List<LogicalVariable>> extraVars) throws AlgebricksException {
+        int numCompareFields = compareVars.get(0).size();
+        if (compareVars.stream().anyMatch(vlist -> vlist.size() != numCompareFields)) {
+            throw AlgebricksException.create(ErrorCode.INVALID_INPUT_PARAMETER);
         }
-        if (inputVars.stream().anyMatch(vlist -> vlist.size() != outputVars.size())) {
-            throw new AlgebricksException("The schemas of input variables are not consistent");
+        int numExtraFields = extraVars.get(0).size();
+        if (extraVars.stream().anyMatch(vlist -> vlist.size() != numExtraFields)) {
+            throw AlgebricksException.create(ErrorCode.INVALID_INPUT_PARAMETER);
         }
-        this.outputVars = outputVars;
-        this.inputVars = inputVars;
+        if (outputVars.size() != numCompareFields + numExtraFields) {
+            throw AlgebricksException.create(ErrorCode.INVALID_INPUT_PARAMETER);
+        }
+
+        this.outputVars = new ArrayList<>(outputVars);
+        this.compareVars = new ArrayList<>(compareVars);
+        this.inputVars = new ArrayList<>(compareVars.size());
+        for (List<LogicalVariable> vars : compareVars) {
+            this.inputVars.add(new ArrayList<>(vars));
+        }
+        for (int i = 0; i < extraVars.size(); i++) {
+            this.inputVars.get(i).addAll(extraVars.get(i));
+        }
+        this.extraVars = extraVars;
     }
 
     @Override
@@ -86,14 +111,20 @@
         IVariableTypeEnvironment typeEnv = ctx.getOutputTypeEnvironment(inputs.get(0).getValue());
 
         for (int i = 1; i < inputs.size(); i++) {
-            checkTypeConsistency(typeEnv, inputVars.get(0), ctx.getOutputTypeEnvironment(inputs.get(i).getValue()),
-                    inputVars.get(i));
+            checkTypeConsistency(typeEnv, compareVars.get(0), ctx.getOutputTypeEnvironment(inputs.get(i).getValue()),
+                    compareVars.get(i));
         }
 
-        IVariableTypeEnvironment env = new NonPropagatingTypeEnvironment(ctx.getExpressionTypeComputer(),
-                ctx.getMetadataProvider());
-        for (int i = 0; i < outputVars.size(); i++) {
-            env.setVarType(outputVars.get(i), typeEnv.getVarType(inputVars.get(0).get(i)));
+        IVariableTypeEnvironment env =
+                new NonPropagatingTypeEnvironment(ctx.getExpressionTypeComputer(), ctx.getMetadataProvider());
+        int i = 0;
+        for (; i < compareVars.get(0).size(); i++) {
+            env.setVarType(outputVars.get(i), typeEnv.getVarType(compareVars.get(0).get(i)));
+        }
+        if (extraVars != null) {
+            for (int k = 0; k < extraVars.get(0).size(); k++) {
+                env.setVarType(outputVars.get(i + k), typeEnv.getVarType(extraVars.get(0).get(k)));
+            }
         }
         return typeEnv;
     }
@@ -103,11 +134,19 @@
     }
 
     public int getNumInput() {
-        return inputVars.size();
+        return compareVars.size();
+    }
+
+    public List<LogicalVariable> getCompareVariables(int inputIndex) {
+        return compareVars.get(inputIndex);
+    }
+
+    public List<List<LogicalVariable>> getExtraVariables() {
+        return extraVars;
     }
 
     public List<LogicalVariable> getInputVariables(int inputIndex) {
-        return inputVars.get(inputIndex);
+        return this.inputVars.get(inputIndex);
     }
 
     private void checkTypeConsistency(IVariableTypeEnvironment expected, List<LogicalVariable> expectedVariables,
@@ -116,8 +155,8 @@
             Object expectedType = expected.getVarType(expectedVariables.get(i));
             Object actualType = actual.getVarType(actualVariables.get(i));
             if (!expectedType.equals(actualType)) {
-                AlgebricksConfig.ALGEBRICKS_LOGGER
-                        .warning("Type of two variables are not equal." + expectedVariables.get(i) + " is of type: "
+                AlgebricksConfig.ALGEBRICKS_LOGGER.warning(
+                        "Type of two variables are not equal." + expectedVariables.get(i) + " is of type: "
                                 + expectedType + actualVariables.get(i) + " is of type: " + actualType);
             }
         }
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 89e2423..e7fb6c0 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
@@ -51,7 +51,7 @@
     // this operator propagates all input variables.
     @Override
     public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
-        IVariableTypeEnvironment env = null;
+        IVariableTypeEnvironment env;
         if (propagateInput) {
             env = createPropagatingAllInputsTypeEnvironment(ctx);
         } else {
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 5f43c3e..0baffc9 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
@@ -69,13 +69,13 @@
         for (int i = 0; i < intersectOp.getNumInput(); i++) {
             List<ILocalStructuralProperty> localProps = new ArrayList<>();
             List<OrderColumn> orderColumns = new ArrayList<>();
-            for (LogicalVariable column : intersectOp.getInputVariables(i)) {
+            for (LogicalVariable column : intersectOp.getCompareVariables(i)) {
                 orderColumns.add(new OrderColumn(column, OrderOperator.IOrder.OrderKind.ASC));
             }
             localProps.add(new LocalOrderProperty(orderColumns));
             IPartitioningProperty pp = null;
             if (intersectOp.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.PARTITIONED) {
-                Set<LogicalVariable> partitioningVariables = new HashSet<>(intersectOp.getInputVariables(i));
+                Set<LogicalVariable> partitioningVariables = new HashSet<>(intersectOp.getCompareVariables(i));
                 pp = new UnorderedPartitionedProperty(partitioningVariables, null);
             }
             pv[i] = new StructuralPropertiesVector(pp, localProps);
@@ -108,37 +108,48 @@
     @Override
     public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
             IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
-                    throws AlgebricksException {
+            throws AlgebricksException {
         // logical op should have checked all the mismatch issues.
         IntersectOperator logicalOp = (IntersectOperator) op;
         int nInput = logicalOp.getNumInput();
         int[][] compareFields = new int[nInput][];
 
-        IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
-                logicalOp.getInputVariables(0), context.getTypeEnvironment(op), context);
+        IBinaryComparatorFactory[] comparatorFactories = JobGenHelper
+                .variablesToAscBinaryComparatorFactories(logicalOp.getCompareVariables(0),
+                        context.getTypeEnvironment(op), context);
 
         INormalizedKeyComputerFactoryProvider nkcfProvider = context.getNormalizedKeyComputerFactoryProvider();
         INormalizedKeyComputerFactory nkcf = null;
 
         if (nkcfProvider != null) {
-            Object type = context.getTypeEnvironment(op).getVarType(logicalOp.getInputVariables(0).get(0));
+            Object type = context.getTypeEnvironment(op).getVarType(logicalOp.getCompareVariables(0).get(0));
             if (type != null) {
                 nkcf = nkcfProvider.getNormalizedKeyComputerFactory(type, true);
             }
         }
 
         for (int i = 0; i < logicalOp.getNumInput(); i++) {
-            compareFields[i] = JobGenHelper.variablesToFieldIndexes(logicalOp.getInputVariables(i), inputSchemas[i]);
+            compareFields[i] = JobGenHelper.variablesToFieldIndexes(logicalOp.getCompareVariables(i), inputSchemas[i]);
+        }
+
+        int[][] extraFields = null;
+        if (logicalOp.getExtraVariables() != null) {
+            extraFields = new int[logicalOp.getNumInput()][];
+            for (int i = 0; i < logicalOp.getNumInput(); i++) {
+                extraFields[i] =
+                        JobGenHelper.variablesToFieldIndexes(logicalOp.getExtraVariables().get(i), inputSchemas[i]);
+            }
         }
 
         IOperatorDescriptorRegistry spec = builder.getJobSpec();
-        RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema,
-                context);
+        RecordDescriptor recordDescriptor =
+                JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
 
-        IntersectOperatorDescriptor opDescriptor = null;
+        IntersectOperatorDescriptor opDescriptor;
         try {
-            opDescriptor = new IntersectOperatorDescriptor(spec, nInput, compareFields, nkcf, comparatorFactories,
-                    recordDescriptor);
+            opDescriptor =
+                    new IntersectOperatorDescriptor(spec, nInput, compareFields, extraFields, nkcf, comparatorFactories,
+                            recordDescriptor);
         } catch (HyracksException e) {
             throw new AlgebricksException(e);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/intersect/IntersectOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/intersect/IntersectOperatorDescriptor.java
index fd7a8cb..5cc854c 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/intersect/IntersectOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/intersect/IntersectOperatorDescriptor.java
@@ -35,6 +35,7 @@
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 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.HyracksException;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
@@ -47,56 +48,70 @@
 
 /**
  * This intersection operator is to get the common elements from multiple way inputs.
- * It will only produce the projected fields which are used for comparison.
+ * It will produce the projected fields which are used for comparison and also the extra fields that could
+ * come with the record from any input
  */
 public class IntersectOperatorDescriptor extends AbstractOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final int[][] projectFields;
+    private final int[][] compareFields;
+    private final int[][] extraFields;
     private final INormalizedKeyComputerFactory firstKeyNormalizerFactory;
     private final IBinaryComparatorFactory[] comparatorFactory;
 
     /**
      * @param spec
      * @param nInputs                   Number of inputs
-     * @param compareAndProjectFields   The project field list of each input.
+     * @param compareFields             The compare field list of each input.
      *                                  All the fields order should be the same with the comparatorFactories
+     * @param extraFields               Extra field that
      * @param firstKeyNormalizerFactory Normalizer for the first comparison key.
      * @param comparatorFactories       A list of comparators for each field
      * @param recordDescriptor
      * @throws HyracksException
      */
-    public IntersectOperatorDescriptor(IOperatorDescriptorRegistry spec, int nInputs, int[][] compareAndProjectFields,
-            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
-            RecordDescriptor recordDescriptor) throws HyracksException {
+    public IntersectOperatorDescriptor(IOperatorDescriptorRegistry spec, int nInputs, int[][] compareFields,
+            int[][] extraFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
+            IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) throws HyracksException {
         super(spec, nInputs, 1);
         outRecDescs[0] = recordDescriptor;
 
-        validateParameters(compareAndProjectFields, comparatorFactories);
+        validateParameters(compareFields, comparatorFactories, extraFields);
 
-        this.projectFields = compareAndProjectFields;
+        this.compareFields = compareFields;
+        this.extraFields = extraFields;
         this.firstKeyNormalizerFactory = firstKeyNormalizerFactory;
         this.comparatorFactory = comparatorFactories;
     }
 
-    private void validateParameters(int[][] compareAndProjectFields, IBinaryComparatorFactory[] comparatorFactories)
-            throws HyracksException {
+    private void validateParameters(int[][] compareFields, IBinaryComparatorFactory[] comparatorFactories,
+            int[][] extraFields) throws HyracksException {
 
-        int firstLength = compareAndProjectFields[0].length;
-        for (int[] fields : compareAndProjectFields) {
+        int firstLength = compareFields[0].length;
+        for (int[] fields : compareFields) {
             if (fields.length != firstLength) {
-                throw new HyracksException("The given input comparison fields is not equal");
+                throw HyracksException.create(ErrorCode.INVALID_INPUT_PARAMETER);
             }
             for (int fid : fields) {
                 if (fid < 0) {
-                    throw new HyracksException("Invalid field index in given comparison fields array");
+                    throw HyracksException.create(ErrorCode.INVALID_INPUT_PARAMETER);
                 }
             }
         }
 
         if (firstLength != comparatorFactories.length) {
-            throw new HyracksException("The size of given fields is not equal with the number of comparators");
+            throw HyracksException.create(ErrorCode.INVALID_INPUT_PARAMETER);
         }
+
+        if (extraFields != null) {
+            firstLength = extraFields[0].length;
+            for (int[] fields : extraFields) {
+                if (fields.length != firstLength) {
+                    throw HyracksException.create(ErrorCode.INVALID_INPUT_PARAMETER);
+                }
+            }
+        }
+
     }
 
     @Override
@@ -125,7 +140,7 @@
             for (int i = 0; i < inputRecordDesc.length; i++) {
                 inputRecordDesc[i] = recordDescProvider.getInputRecordDescriptor(getActivityId(), i);
             }
-            return new IntersectOperatorNodePushable(ctx, inputArity, inputRecordDesc, projectFields,
+            return new IntersectOperatorNodePushable(ctx, inputArity, inputRecordDesc, compareFields, extraFields,
                     firstKeyNormalizerFactory, comparatorFactory);
         }
     }
@@ -135,7 +150,8 @@
         private enum ACTION {FAILED, CLOSE}
 
         private final int inputArity;
-        private final int[][] projectFields;
+        private final int[][] compareFields;
+        private final int[][] allProjectFields;
         private final BitSet consumed;
         private final int[] tupleIndexMarker;
         private final FrameTupleAccessor[] refAccessor;
@@ -147,16 +163,32 @@
         private boolean done = false;
 
         public IntersectOperatorNodePushable(IHyracksTaskContext ctx, int inputArity,
-                RecordDescriptor[] inputRecordDescriptors, int[][] projectFields,
+                RecordDescriptor[] inputRecordDescriptors, int[][] compareFields, int[][] extraFields,
                 INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactory)
                 throws HyracksDataException {
 
             this.inputArity = inputArity;
-            this.projectFields = projectFields;
+            this.compareFields = compareFields;
+
+            int[][] projectedFields = compareFields;
+            if (extraFields != null) {
+                projectedFields = new int[inputArity][];
+                for (int input = 0; input < inputArity; input++) {
+                    projectedFields[input] = new int[compareFields[input].length + extraFields[input].length];
+                    int j = 0;
+                    for (; j < compareFields[input].length; j++) {
+                        projectedFields[input][j] = compareFields[input][j];
+                    }
+                    for (int k = 0; k < extraFields[input].length; k++) {
+                        projectedFields[input][j + k] = extraFields[input][k];
+                    }
+                }
+            }
+            this.allProjectFields = projectedFields;
             this.firstKeyNormalizerComputer =
                     firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory.createNormalizedKeyComputer();
 
-            comparators = new IBinaryComparator[projectFields[0].length];
+            comparators = new IBinaryComparator[compareFields[0].length];
             for (int i = 0; i < comparators.length; i++) {
                 comparators[i] = comparatorFactory[i].createBinaryComparator();
             }
@@ -241,8 +273,9 @@
                                 continue;
                             }
                             while (tupleIndexMarker[i] < refAccessor[i].getTupleCount()) {
-                                int cmp = compare(i, refAccessor[i], tupleIndexMarker[i], maxInput,
-                                        refAccessor[maxInput], tupleIndexMarker[maxInput]);
+                                int cmp =
+                                        compare(i, refAccessor[i], tupleIndexMarker[i], maxInput, refAccessor[maxInput],
+                                                tupleIndexMarker[maxInput]);
                                 if (cmp == 0) {
                                     match++;
                                     break;
@@ -260,7 +293,7 @@
                         }
                         if (match == inputArity) {
                             FrameUtils.appendProjectionToWriter(writer, appender, refAccessor[maxInput],
-                                    tupleIndexMarker[maxInput], projectFields[maxInput]);
+                                    tupleIndexMarker[maxInput], allProjectFields[maxInput]);
                             for (int i = 0; i < inputArity; i++) {
                                 tupleIndexMarker[i]++;
                                 if (tupleIndexMarker[i] >= refAccessor[i].getTupleCount()) {
@@ -291,11 +324,11 @@
 
                     for (int i = 0; i < comparators.length; i++) {
                         int cmp = comparators[i].compare(frameTupleAccessor1.getBuffer().array(),
-                                frameTupleAccessor1.getAbsoluteFieldStartOffset(tid1, projectFields[input1][i]),
-                                frameTupleAccessor1.getFieldLength(tid1, projectFields[input1][i]),
+                                frameTupleAccessor1.getAbsoluteFieldStartOffset(tid1, compareFields[input1][i]),
+                                frameTupleAccessor1.getFieldLength(tid1, compareFields[input1][i]),
                                 frameTupleAccessor2.getBuffer().array(),
-                                frameTupleAccessor2.getAbsoluteFieldStartOffset(tid2, projectFields[input2][i]),
-                                frameTupleAccessor2.getFieldLength(tid2, projectFields[input2][i]));
+                                frameTupleAccessor2.getAbsoluteFieldStartOffset(tid2, compareFields[input2][i]),
+                                frameTupleAccessor2.getFieldLength(tid2, compareFields[input2][i]));
 
                         if (cmp != 0) {
                             return cmp;
@@ -308,8 +341,8 @@
                     return firstKeyNormalizerComputer == null ?
                             0 :
                             firstKeyNormalizerComputer.normalize(frameTupleAccessor1.getBuffer().array(),
-                                    frameTupleAccessor1.getAbsoluteFieldStartOffset(tid1, projectFields[inputId1][0]),
-                                    frameTupleAccessor1.getFieldLength(tid1, projectFields[inputId1][0]));
+                                    frameTupleAccessor1.getAbsoluteFieldStartOffset(tid1, compareFields[inputId1][0]),
+                                    frameTupleAccessor1.getFieldLength(tid1, compareFields[inputId1][0]));
                 }
 
                 private int findMaxInput() throws HyracksDataException {
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IntersectOperatorDescriptorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IntersectOperatorDescriptorTest.java
index 0c49588..6729713 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IntersectOperatorDescriptorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IntersectOperatorDescriptorTest.java
@@ -108,7 +108,7 @@
     public void testNormalOperatorInitialization() throws HyracksException {
 
         IntersectOperatorDescriptor operatorDescriptor = new IntersectOperatorDescriptor(mockRegistry, nInputs,
-                compareFields, normalizedKeyFactory, comparatorFactory, outRecordDescriptor);
+                compareFields, null, normalizedKeyFactory, comparatorFactory, outRecordDescriptor);
 
         assertEquals(nInputs, operatorDescriptor.getInputArity());
     }
@@ -158,7 +158,7 @@
     private void executeAndVerifyResult(List<IFrame>[] inputFrames, List<Object[]> answer) throws Exception {
         IntersectOperatorDescriptor.IntersectOperatorNodePushable pushable =
                 new IntersectOperatorDescriptor.IntersectOperatorNodePushable(ctx, nInputs, inputRecordDescriptor,
-                        compareFields, null, comparatorFactory);
+                        compareFields, null, null, comparatorFactory);
         assertEquals(nInputs, pushable.getInputArity());
 
         IFrameWriter[] writers = new IFrameWriter[nInputs];